1515
1616import com .facebook .airlift .json .JsonCodec ;
1717import com .facebook .airlift .log .Logger ;
18+ import com .facebook .presto .common .QualifiedObjectName ;
1819import com .facebook .presto .common .Subfield ;
1920import com .facebook .presto .common .predicate .TupleDomain ;
2021import com .facebook .presto .common .type .BigintType ;
2728import com .facebook .presto .iceberg .changelog .ChangelogUtil ;
2829import com .facebook .presto .spi .ColumnHandle ;
2930import com .facebook .presto .spi .ColumnMetadata ;
31+ import com .facebook .presto .spi .ConnectorDistributedProcedureHandle ;
32+ import com .facebook .presto .spi .ConnectorId ;
3033import com .facebook .presto .spi .ConnectorInsertTableHandle ;
3134import com .facebook .presto .spi .ConnectorNewTableLayout ;
3235import com .facebook .presto .spi .ConnectorOutputTableHandle ;
3336import com .facebook .presto .spi .ConnectorSession ;
37+ import com .facebook .presto .spi .ConnectorSplitSource ;
3438import com .facebook .presto .spi .ConnectorTableHandle ;
3539import com .facebook .presto .spi .ConnectorTableLayout ;
3640import com .facebook .presto .spi .ConnectorTableLayoutHandle ;
4751import com .facebook .presto .spi .connector .ConnectorOutputMetadata ;
4852import com .facebook .presto .spi .connector .ConnectorTableVersion ;
4953import com .facebook .presto .spi .function .StandardFunctionResolution ;
54+ import com .facebook .presto .spi .procedure .DistributedProcedure ;
55+ import com .facebook .presto .spi .procedure .IProcedureRegistry ;
56+ import com .facebook .presto .spi .procedure .Procedure ;
5057import com .facebook .presto .spi .relation .RowExpression ;
5158import com .facebook .presto .spi .relation .RowExpressionService ;
5259import com .facebook .presto .spi .statistics .ColumnStatisticMetadata ;
7077import org .apache .iceberg .FileFormat ;
7178import org .apache .iceberg .FileMetadata ;
7279import org .apache .iceberg .FileScanTask ;
80+ import org .apache .iceberg .ManifestFile ;
7381import org .apache .iceberg .PartitionField ;
7482import org .apache .iceberg .PartitionSpec ;
7583import org .apache .iceberg .PartitionSpecParser ;
@@ -161,23 +169,26 @@ public abstract class IcebergAbstractMetadata
161169 implements ConnectorMetadata
162170{
163171 protected final TypeManager typeManager ;
172+ protected final IProcedureRegistry procedureRegistry ;
164173 protected final JsonCodec <CommitTaskData > commitTaskCodec ;
165174 protected final NodeVersion nodeVersion ;
166175 protected final RowExpressionService rowExpressionService ;
167- protected Transaction transaction ;
176+ protected IcebergTransactionContext transactionContext ;
168177
169178 private final StandardFunctionResolution functionResolution ;
170179 private final ConcurrentMap <SchemaTableName , Table > icebergTables = new ConcurrentHashMap <>();
171180 private static final Logger log = Logger .get (IcebergAbstractMetadata .class );
172181
173182 public IcebergAbstractMetadata (
174183 TypeManager typeManager ,
184+ IProcedureRegistry procedureRegistry ,
175185 StandardFunctionResolution functionResolution ,
176186 RowExpressionService rowExpressionService ,
177187 JsonCodec <CommitTaskData > commitTaskCodec ,
178188 NodeVersion nodeVersion )
179189 {
180190 this .typeManager = requireNonNull (typeManager , "typeManager is null" );
191+ this .procedureRegistry = requireNonNull (procedureRegistry , "procedureRegistry is null" );
181192 this .commitTaskCodec = requireNonNull (commitTaskCodec , "commitTaskCodec is null" );
182193 this .functionResolution = requireNonNull (functionResolution , "functionResolution is null" );
183194 this .rowExpressionService = requireNonNull (rowExpressionService , "rowExpressionService is null" );
@@ -199,6 +210,11 @@ protected final Table getIcebergTable(ConnectorSession session, SchemaTableName
199210
200211 protected abstract void unregisterTable (ConnectorSession clientSession , SchemaTableName schemaTableName );
201212
213+ public Optional <ConnectorSplitSource > getSplitSourceInCurrentCallProcedureTransaction ()
214+ {
215+ return transactionContext == null ? Optional .empty () : transactionContext .getConnectorSplitSource ();
216+ }
217+
202218 /**
203219 * This class implements the default implementation for getTableLayoutForConstraint which will be used in the case of a Java Worker
204220 */
@@ -420,7 +436,7 @@ public Optional<ConnectorOutputMetadata> finishCreateTable(ConnectorSession sess
420436
421437 protected ConnectorInsertTableHandle beginIcebergTableInsert (IcebergTableHandle table , Table icebergTable )
422438 {
423- transaction = icebergTable .newTransaction ();
439+ transactionContext = new IcebergTransactionContext ( Optional . of ( icebergTable ), icebergTable .newTransaction () );
424440
425441 return new IcebergWritableTableHandle (
426442 table .getSchemaName (),
@@ -437,12 +453,13 @@ protected ConnectorInsertTableHandle beginIcebergTableInsert(IcebergTableHandle
437453 public Optional <ConnectorOutputMetadata > finishInsert (ConnectorSession session , ConnectorInsertTableHandle insertHandle , Collection <Slice > fragments , Collection <ComputedStatistics > computedStatistics )
438454 {
439455 if (fragments .isEmpty ()) {
440- transaction .commitTransaction ();
456+ transactionContext .getTransaction ().commitTransaction ();
457+ transactionContext .destroy ();
441458 return Optional .empty ();
442459 }
443460
444461 IcebergWritableTableHandle table = (IcebergWritableTableHandle ) insertHandle ;
445- Table icebergTable = transaction .table ();
462+ Table icebergTable = transactionContext . getTransaction () .table ();
446463
447464 List <CommitTaskData > commitTasks = fragments .stream ()
448465 .map (slice -> commitTaskCodec .fromJson (slice .getBytes ()))
@@ -453,7 +470,7 @@ public Optional<ConnectorOutputMetadata> finishInsert(ConnectorSession session,
453470 icebergTable .schema ().findType (field .sourceId ())))
454471 .toArray (Type []::new );
455472
456- AppendFiles appendFiles = transaction .newFastAppend ();
473+ AppendFiles appendFiles = transactionContext . getTransaction () .newFastAppend ();
457474 for (CommitTaskData task : commitTasks ) {
458475 DataFiles .Builder builder = DataFiles .builder (icebergTable .spec ())
459476 .withPath (task .getPath ())
@@ -471,7 +488,8 @@ public Optional<ConnectorOutputMetadata> finishInsert(ConnectorSession session,
471488 }
472489
473490 appendFiles .commit ();
474- transaction .commitTransaction ();
491+ transactionContext .getTransaction ().commitTransaction ();
492+ transactionContext .destroy ();
475493
476494 return Optional .of (new HiveWrittenPartitions (commitTasks .stream ()
477495 .map (CommitTaskData ::getPath )
@@ -762,6 +780,44 @@ public void truncateTable(ConnectorSession session, ConnectorTableHandle tableHa
762780 }
763781 }
764782
783+ @ Override
784+ public ConnectorDistributedProcedureHandle beginCallDistributedProcedure (
785+ ConnectorSession session ,
786+ QualifiedObjectName procedureName ,
787+ ConnectorTableLayoutHandle tableLayoutHandle ,
788+ Object [] arguments )
789+ {
790+ IcebergTableHandle handle = ((IcebergTableLayoutHandle ) tableLayoutHandle ).getTable ();
791+ Table icebergTable = getIcebergTable (session , handle .getSchemaTableName ());
792+
793+ if (handle .isSnapshotSpecified ()) {
794+ throw new PrestoException (NOT_SUPPORTED , "This connector do not allow table execute at specified snapshot" );
795+ }
796+
797+ transactionContext = new IcebergTransactionContext (Optional .of (icebergTable ), icebergTable .newTransaction ());
798+ Procedure procedure = procedureRegistry .resolve (
799+ new ConnectorId (procedureName .getCatalogName ()),
800+ new SchemaTableName (
801+ procedureName .getSchemaName (),
802+ procedureName .getObjectName ()));
803+ verify (procedure instanceof DistributedProcedure , "procedure must be DistributedProcedure" );
804+ return ((DistributedProcedure ) procedure ).getBeginCallDistributedProcedure ().begin (session , transactionContext , tableLayoutHandle , arguments );
805+ }
806+
807+ @ Override
808+ public void finishCallDistributedProcedure (ConnectorSession session , ConnectorDistributedProcedureHandle tableHandle , QualifiedObjectName procedureName , Collection <Slice > fragments )
809+ {
810+ Procedure procedure = procedureRegistry .resolve (
811+ new ConnectorId (procedureName .getCatalogName ()),
812+ new SchemaTableName (
813+ procedureName .getSchemaName (),
814+ procedureName .getObjectName ()));
815+ verify (procedure instanceof DistributedProcedure , "procedure must be DistributedProcedure" );
816+ ((DistributedProcedure ) procedure ).getFinishCallDistributedProcedure ().finish (transactionContext , tableHandle , fragments );
817+ transactionContext .getTransaction ().commitTransaction ();
818+ transactionContext .destroy ();
819+ }
820+
765821 @ Override
766822 public ConnectorTableHandle beginDelete (ConnectorSession session , ConnectorTableHandle tableHandle )
767823 {
@@ -782,7 +838,7 @@ public ConnectorTableHandle beginDelete(ConnectorSession session, ConnectorTable
782838 }
783839
784840 validateTableMode (session , icebergTable );
785- transaction = icebergTable .newTransaction ();
841+ transactionContext = new IcebergTransactionContext ( Optional . of ( icebergTable ), icebergTable .newTransaction () );
786842
787843 return handle ;
788844 }
@@ -793,7 +849,7 @@ public void finishDelete(ConnectorSession session, ConnectorTableHandle tableHan
793849 IcebergTableHandle handle = (IcebergTableHandle ) tableHandle ;
794850 Table icebergTable = getIcebergTable (session , handle .getSchemaTableName ());
795851
796- RowDelta rowDelta = transaction .newRowDelta ();
852+ RowDelta rowDelta = transactionContext . getTransaction () .newRowDelta ();
797853
798854 List <CommitTaskData > commitTasks = fragments .stream ()
799855 .map (slice -> commitTaskCodec .fromJson (slice .getBytes ()))
@@ -830,7 +886,8 @@ public void finishDelete(ConnectorSession session, ConnectorTableHandle tableHan
830886 }
831887
832888 rowDelta .commit ();
833- transaction .commitTransaction ();
889+ transactionContext .getTransaction ().commitTransaction ();
890+ transactionContext .destroy ();
834891 }
835892
836893 @ Override
@@ -864,7 +921,15 @@ public boolean supportsMetadataDelete(ConnectorSession session, ConnectorTableHa
864921 Table icebergTable = getIcebergTable (session , handle .getSchemaTableName ());
865922
866923 boolean supportsMetadataDelete = true ;
867- for (PartitionSpec spec : icebergTable .specs ().values ()) {
924+ // Get partition specs that really need to be checked
925+ Set <PartitionSpec > partitionSpecs = handle .getIcebergTableName ().getSnapshotId ().map (
926+ snapshot -> icebergTable .snapshot (snapshot ).allManifests (icebergTable .io ()).stream ()
927+ .map (ManifestFile ::partitionSpecId )
928+ .map (icebergTable .specs ()::get )
929+ .collect (toImmutableSet ()))
930+ .orElseGet (() -> ImmutableSet .copyOf (icebergTable .specs ().values ())); // No snapshot, so no data. This case doesn't matter.
931+
932+ for (PartitionSpec spec : partitionSpecs ) {
868933 // Currently we do not support delete when any partition columns in predicate is not transform by identity()
869934 Set <Integer > partitionColumnSourceIds = spec .fields ().stream ()
870935 .filter (field -> field .transform ().isIdentity ())
@@ -920,15 +985,16 @@ public OptionalLong metadataDelete(ConnectorSession session, ConnectorTableHandl
920985 */
921986 private long removeScanFiles (Table icebergTable , Iterable <FileScanTask > scan )
922987 {
923- transaction = icebergTable .newTransaction ();
924- DeleteFiles deletes = transaction .newDelete ();
988+ transactionContext = new IcebergTransactionContext ( Optional . of ( icebergTable ), icebergTable .newTransaction () );
989+ DeleteFiles deletes = transactionContext . getTransaction () .newDelete ();
925990 AtomicLong rowsDeleted = new AtomicLong (0L );
926991 scan .forEach (t -> {
927992 deletes .deleteFile (t .file ());
928993 rowsDeleted .addAndGet (t .estimatedRowsCount ());
929994 });
930995 deletes .commit ();
931- transaction .commitTransaction ();
996+ transactionContext .getTransaction ().commitTransaction ();
997+ transactionContext .destroy ();
932998 return rowsDeleted .get ();
933999 }
9341000
0 commit comments