42
42
import io .trino .spi .connector .RelationColumnsMetadata ;
43
43
import io .trino .spi .connector .RelationCommentMetadata ;
44
44
import io .trino .spi .connector .RetryMode ;
45
+ import io .trino .spi .connector .RowChangeParadigm ;
45
46
import io .trino .spi .connector .SaveMode ;
46
47
import io .trino .spi .connector .SchemaNotFoundException ;
47
48
import io .trino .spi .connector .SchemaTableName ;
48
49
import io .trino .spi .connector .TableFunctionApplicationResult ;
49
50
import io .trino .spi .connector .TableNotFoundException ;
51
+ import io .trino .spi .expression .Constant ;
50
52
import io .trino .spi .function .table .ConnectorTableFunctionHandle ;
51
53
import io .trino .spi .predicate .TupleDomain ;
52
54
import io .trino .spi .statistics .ComputedStatistics ;
78
80
import static io .trino .spi .connector .RelationColumnsMetadata .forTable ;
79
81
import static io .trino .spi .connector .RelationCommentMetadata .forRelation ;
80
82
import static io .trino .spi .connector .RetryMode .NO_RETRIES ;
83
+ import static io .trino .spi .connector .RowChangeParadigm .CHANGE_ONLY_UPDATED_COLUMNS ;
81
84
import static io .trino .spi .connector .SaveMode .REPLACE ;
82
85
import static java .lang .String .format ;
83
86
import static java .util .Locale .ENGLISH ;
@@ -285,6 +288,7 @@ public Optional<ConstraintApplicationResult<ConnectorTableHandle>> applyFilter(C
285
288
286
289
String clusteringKeyPredicates = "" ;
287
290
TupleDomain <ColumnHandle > unenforcedConstraint ;
291
+ Boolean includesAllClusteringColumnsAndHasAllEqPredicates = null ;
288
292
if (partitionResult .unpartitioned () || partitionResult .indexedColumnPredicatePushdown ()) {
289
293
// When the filter is missing at least one of the partition keys or when the table is not partitioned,
290
294
// use the raw unenforced constraint of the partitionResult
@@ -297,6 +301,7 @@ public Optional<ConstraintApplicationResult<ConnectorTableHandle>> applyFilter(C
297
301
partitionResult .unenforcedConstraint ());
298
302
clusteringKeyPredicates = clusteringPredicatesExtractor .getClusteringKeyPredicates ();
299
303
unenforcedConstraint = clusteringPredicatesExtractor .getUnenforcedConstraints ();
304
+ includesAllClusteringColumnsAndHasAllEqPredicates = clusteringPredicatesExtractor .includesAllClusteringColumnsAndHasAllEqPredicates ();
300
305
}
301
306
302
307
Optional <List <CassandraPartition >> currentPartitions = handle .getPartitions ();
@@ -313,7 +318,8 @@ public Optional<ConstraintApplicationResult<ConnectorTableHandle>> applyFilter(C
313
318
handle .getTableName (),
314
319
Optional .of (partitionResult .partitions ()),
315
320
// TODO this should probably be AND-ed with handle.getClusteringKeyPredicates()
316
- clusteringKeyPredicates )),
321
+ clusteringKeyPredicates ,
322
+ includesAllClusteringColumnsAndHasAllEqPredicates )),
317
323
unenforcedConstraint ,
318
324
constraint .getExpression (),
319
325
false ));
@@ -470,39 +476,103 @@ private static boolean isHiddenIdColumn(CassandraColumnHandle columnHandle)
470
476
}
471
477
472
478
@ Override
473
- public ConnectorMergeTableHandle beginMerge (ConnectorSession session , ConnectorTableHandle tableHandle , Map < Integer , Collection < ColumnHandle >> updateCaseColumns , RetryMode retryMode )
479
+ public RowChangeParadigm getRowChangeParadigm (ConnectorSession session , ConnectorTableHandle tableHandle )
474
480
{
475
- throw new TrinoException ( NOT_SUPPORTED , "Delete without primary key or partition key is not supported" ) ;
481
+ return CHANGE_ONLY_UPDATED_COLUMNS ;
476
482
}
477
483
484
+ /**
485
+ * Attempt to push down an update operation into the connector. If a connector
486
+ * can execute an update for the table handle on its own, it should return a
487
+ * table handle, which will be passed back to {@link #executeUpdate} during
488
+ * query executing to actually execute the update.
489
+ */
478
490
@ Override
479
- public ColumnHandle getMergeRowIdColumnHandle (ConnectorSession session , ConnectorTableHandle tableHandle )
491
+ public Optional < ConnectorTableHandle > applyUpdate (ConnectorSession session , ConnectorTableHandle tableHandle , Map < ColumnHandle , Constant > assignments )
480
492
{
481
- return new CassandraColumnHandle ("$update_row_id" , 0 , CassandraTypes .TEXT , false , false , false , true );
493
+ CassandraNamedRelationHandle table = ((CassandraTableHandle ) tableHandle ).getRequiredNamedRelation ();
494
+ if (cassandraSession .isMaterializedView (table .getSchemaTableName ())) {
495
+ throw new TrinoException (NOT_SUPPORTED , "Updating materialized views not yet supported" );
496
+ }
497
+
498
+ CassandraNamedRelationHandle handle = ((CassandraTableHandle ) tableHandle ).getRequiredNamedRelation ();
499
+ List <CassandraPartition > partitions = handle .getPartitions ()
500
+ .orElseThrow (() -> new TrinoException (NOT_SUPPORTED , "Updating without partition key is not supported" ));
501
+ if (partitions .isEmpty ()) {
502
+ // there are no records of a given partition key
503
+ throw new TrinoException (NOT_SUPPORTED , "Updating without partition key is not supported" );
504
+ }
505
+ if (!handle .getIncludesAllClusteringColumnsAndHasAllEqPredicates ()) {
506
+ throw new TrinoException (NOT_SUPPORTED , "Updating without all clustering keys or with non-eq predicates is not supported" );
507
+ }
508
+
509
+ Map <String , String > assignmentsMap = new HashMap <>();
510
+ for (Map .Entry <ColumnHandle , Constant > entry : assignments .entrySet ()) {
511
+ CassandraColumnHandle column = (CassandraColumnHandle ) entry .getKey ();
512
+ if (isHiddenIdColumn (column )) {
513
+ throw new TrinoException (NOT_SUPPORTED , "Updating the hidden id column is not supported" );
514
+ }
515
+ Object value = entry .getValue ().getValue ();
516
+ if (value == null ) {
517
+ throw new TrinoException (NOT_SUPPORTED , "Updating columns to null is not supported" );
518
+ }
519
+ String cqlLiteral = cassandraTypeManager .toCqlLiteral (column .cassandraType (), value ); // validate that the value can be converted to the cassandra type
520
+ assignmentsMap .put (column .name (), cqlLiteral );
521
+ }
522
+
523
+ return Optional .of (new CassandraUpdateTableHandle (tableHandle , assignmentsMap ));
482
524
}
483
525
526
+ /**
527
+ * Execute the update operation on the handle returned from {@link #applyUpdate}.
528
+ */
484
529
@ Override
485
- public Optional < ConnectorTableHandle > applyDelete (ConnectorSession session , ConnectorTableHandle handle )
530
+ public OptionalLong executeUpdate (ConnectorSession session , ConnectorTableHandle tableHandle )
486
531
{
487
- return Optional .of (handle );
532
+ CassandraUpdateTableHandle updateTableHandle = ((CassandraUpdateTableHandle ) tableHandle );
533
+ CassandraTableHandle cassandraTableHandle = (CassandraTableHandle ) updateTableHandle .tableHandle ();
534
+ CassandraNamedRelationHandle namedRelationHandle = cassandraTableHandle .getRequiredNamedRelation ();
535
+ for (String cql : CassandraCqlUtils .getUpdateQueries (namedRelationHandle , updateTableHandle .assignments ())) {
536
+ cassandraSession .execute (cql );
537
+ }
538
+ return OptionalLong .empty ();
488
539
}
489
540
490
541
@ Override
491
- public OptionalLong executeDelete (ConnectorSession session , ConnectorTableHandle deleteHandle )
542
+ public ColumnHandle getMergeRowIdColumnHandle (ConnectorSession session , ConnectorTableHandle tableHandle )
543
+ {
544
+ return new CassandraColumnHandle ("$update_row_id" , 0 , CassandraTypes .TEXT , false , false , false , true );
545
+ }
546
+
547
+ @ Override
548
+ public Optional <ConnectorTableHandle > applyDelete (ConnectorSession session , ConnectorTableHandle deleteHandle )
492
549
{
493
550
CassandraNamedRelationHandle handle = ((CassandraTableHandle ) deleteHandle ).getRequiredNamedRelation ();
494
551
List <CassandraPartition > partitions = handle .getPartitions ()
495
552
.orElseThrow (() -> new TrinoException (NOT_SUPPORTED , "Deleting without partition key is not supported" ));
496
553
if (partitions .isEmpty ()) {
497
554
// there are no records of a given partition key
498
- return OptionalLong .empty ();
555
+ return Optional .empty ();
499
556
}
557
+ return Optional .of (deleteHandle );
558
+ }
559
+
560
+ @ Override
561
+ public OptionalLong executeDelete (ConnectorSession session , ConnectorTableHandle deleteHandle )
562
+ {
563
+ CassandraNamedRelationHandle handle = ((CassandraTableHandle ) deleteHandle ).getRequiredNamedRelation ();
500
564
for (String cql : CassandraCqlUtils .getDeleteQueries (handle )) {
501
565
cassandraSession .execute (cql );
502
566
}
503
567
return OptionalLong .empty ();
504
568
}
505
569
570
+ @ Override
571
+ public ConnectorMergeTableHandle beginMerge (ConnectorSession session , ConnectorTableHandle tableHandle , Map <Integer , Collection <ColumnHandle >> updateCaseColumns , RetryMode retryMode )
572
+ {
573
+ throw new TrinoException (NOT_SUPPORTED , "This connector does not support modifying table rows" );
574
+ }
575
+
506
576
@ Override
507
577
public Optional <TableFunctionApplicationResult <ConnectorTableHandle >> applyTableFunction (ConnectorSession session , ConnectorTableFunctionHandle handle )
508
578
{
0 commit comments