Skip to content

Commit 1c47d5b

Browse files
committed
Refactor Iceberg connector to support call distributed procedure
1 parent b73cf46 commit 1c47d5b

16 files changed

+496
-19
lines changed
Lines changed: 135 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,135 @@
1+
/*
2+
* Licensed under the Apache License, Version 2.0 (the "License");
3+
* you may not use this file except in compliance with the License.
4+
* You may obtain a copy of the License at
5+
*
6+
* http://www.apache.org/licenses/LICENSE-2.0
7+
*
8+
* Unless required by applicable law or agreed to in writing, software
9+
* distributed under the License is distributed on an "AS IS" BASIS,
10+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
11+
* See the License for the specific language governing permissions and
12+
* limitations under the License.
13+
*/
14+
package com.facebook.presto.iceberg;
15+
16+
import com.facebook.presto.iceberg.delete.DeleteFile;
17+
import com.facebook.presto.spi.ConnectorSession;
18+
import com.facebook.presto.spi.ConnectorSplit;
19+
import com.facebook.presto.spi.ConnectorSplitSource;
20+
import com.facebook.presto.spi.SplitWeight;
21+
import com.facebook.presto.spi.connector.ConnectorPartitionHandle;
22+
import com.google.common.collect.ImmutableList;
23+
import com.google.common.io.Closer;
24+
import org.apache.iceberg.FileScanTask;
25+
import org.apache.iceberg.PartitionSpec;
26+
import org.apache.iceberg.PartitionSpecParser;
27+
import org.apache.iceberg.TableScan;
28+
import org.apache.iceberg.io.CloseableIterable;
29+
import org.apache.iceberg.io.CloseableIterator;
30+
31+
import java.io.IOException;
32+
import java.io.UncheckedIOException;
33+
import java.util.ArrayList;
34+
import java.util.Iterator;
35+
import java.util.List;
36+
import java.util.Optional;
37+
import java.util.concurrent.CompletableFuture;
38+
import java.util.function.Consumer;
39+
40+
import static com.facebook.presto.hive.HiveCommonSessionProperties.getNodeSelectionStrategy;
41+
import static com.facebook.presto.iceberg.FileFormat.fromIcebergFileFormat;
42+
import static com.facebook.presto.iceberg.IcebergUtil.getDataSequenceNumber;
43+
import static com.facebook.presto.iceberg.IcebergUtil.getPartitionKeys;
44+
import static com.facebook.presto.iceberg.IcebergUtil.partitionDataFromStructLike;
45+
import static com.google.common.collect.ImmutableList.toImmutableList;
46+
import static com.google.common.collect.Iterators.limit;
47+
import static java.util.Objects.requireNonNull;
48+
import static java.util.concurrent.CompletableFuture.completedFuture;
49+
50+
public class CallDistributedProcedureSplitSource
51+
implements ConnectorSplitSource
52+
{
53+
private CloseableIterator<FileScanTask> fileScanTaskIterator;
54+
private Optional<Consumer<FileScanTask>> fileScanTaskConsumer;
55+
56+
private final TableScan tableScan;
57+
private final Closer closer = Closer.create();
58+
private final double minimumAssignedSplitWeight;
59+
private final ConnectorSession session;
60+
61+
public CallDistributedProcedureSplitSource(
62+
ConnectorSession session,
63+
TableScan tableScan,
64+
CloseableIterable<FileScanTask> fileScanTaskIterable,
65+
Optional<Consumer<FileScanTask>> fileScanTaskConsumer,
66+
double minimumAssignedSplitWeight)
67+
{
68+
this.session = requireNonNull(session, "session is null");
69+
this.tableScan = requireNonNull(tableScan, "tableScan is null");
70+
this.fileScanTaskIterator = fileScanTaskIterable.iterator();
71+
this.fileScanTaskConsumer = requireNonNull(fileScanTaskConsumer, "fileScanTaskConsumer is null");
72+
this.minimumAssignedSplitWeight = minimumAssignedSplitWeight;
73+
closer.register(fileScanTaskIterator);
74+
}
75+
76+
@Override
77+
public CompletableFuture<ConnectorSplitBatch> getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize)
78+
{
79+
// TODO: move this to a background thread
80+
List<ConnectorSplit> splits = new ArrayList<>();
81+
Iterator<FileScanTask> iterator = limit(fileScanTaskIterator, maxSize);
82+
while (iterator.hasNext()) {
83+
FileScanTask task = iterator.next();
84+
fileScanTaskConsumer.ifPresent(consumer -> consumer.accept(task));
85+
splits.add(toIcebergSplit(task));
86+
}
87+
return completedFuture(new ConnectorSplitBatch(splits, isFinished()));
88+
}
89+
90+
@Override
91+
public boolean isFinished()
92+
{
93+
return !fileScanTaskIterator.hasNext();
94+
}
95+
96+
@Override
97+
public void close()
98+
{
99+
try {
100+
closer.close();
101+
// TODO: remove this after org.apache.iceberg.io.CloseableIterator'withClose
102+
// correct release resources holds by iterator.
103+
fileScanTaskIterator = CloseableIterator.empty();
104+
}
105+
catch (IOException e) {
106+
throw new UncheckedIOException(e);
107+
}
108+
}
109+
110+
private ConnectorSplit toIcebergSplit(FileScanTask task)
111+
{
112+
PartitionSpec spec = task.spec();
113+
Optional<PartitionData> partitionData = partitionDataFromStructLike(spec, task.file().partition());
114+
115+
// TODO: We should leverage residual expression and convert that to TupleDomain.
116+
// The predicate here is used by readers for predicate push down at reader level,
117+
// so when we do not use residual expression, we are just wasting CPU cycles
118+
// on reader side evaluating a condition that we know will always be true.
119+
120+
return new IcebergSplit(
121+
task.file().path().toString(),
122+
task.start(),
123+
task.length(),
124+
fromIcebergFileFormat(task.file().format()),
125+
ImmutableList.of(),
126+
getPartitionKeys(task),
127+
PartitionSpecParser.toJson(spec),
128+
partitionData.map(PartitionData::toJson),
129+
getNodeSelectionStrategy(session),
130+
SplitWeight.fromProportion(Math.min(Math.max((double) task.length() / tableScan.targetSplitSize(), minimumAssignedSplitWeight), 1.0)),
131+
task.deletes().stream().map(DeleteFile::fromIceberg).collect(toImmutableList()),
132+
Optional.empty(),
133+
getDataSequenceNumber(task.file()));
134+
}
135+
}

presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergAbstractMetadata.java

Lines changed: 69 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,7 @@
1515

1616
import com.facebook.airlift.json.JsonCodec;
1717
import com.facebook.airlift.log.Logger;
18+
import com.facebook.presto.common.QualifiedObjectName;
1819
import com.facebook.presto.common.Subfield;
1920
import com.facebook.presto.common.predicate.TupleDomain;
2021
import com.facebook.presto.common.type.BigintType;
@@ -27,10 +28,13 @@
2728
import com.facebook.presto.iceberg.changelog.ChangelogUtil;
2829
import com.facebook.presto.spi.ColumnHandle;
2930
import com.facebook.presto.spi.ColumnMetadata;
31+
import com.facebook.presto.spi.ConnectorDistributedProcedureHandle;
32+
import com.facebook.presto.spi.ConnectorId;
3033
import com.facebook.presto.spi.ConnectorInsertTableHandle;
3134
import com.facebook.presto.spi.ConnectorNewTableLayout;
3235
import com.facebook.presto.spi.ConnectorOutputTableHandle;
3336
import com.facebook.presto.spi.ConnectorSession;
37+
import com.facebook.presto.spi.ConnectorSplitSource;
3438
import com.facebook.presto.spi.ConnectorTableHandle;
3539
import com.facebook.presto.spi.ConnectorTableLayout;
3640
import com.facebook.presto.spi.ConnectorTableLayoutHandle;
@@ -49,6 +53,9 @@
4953
import com.facebook.presto.spi.connector.ConnectorTableVersion.VersionOperator;
5054
import com.facebook.presto.spi.connector.ConnectorTableVersion.VersionType;
5155
import com.facebook.presto.spi.function.StandardFunctionResolution;
56+
import com.facebook.presto.spi.procedure.DistributedProcedure;
57+
import com.facebook.presto.spi.procedure.IProcedureRegistry;
58+
import com.facebook.presto.spi.procedure.Procedure;
5259
import com.facebook.presto.spi.relation.RowExpression;
5360
import com.facebook.presto.spi.relation.RowExpressionService;
5461
import com.facebook.presto.spi.statistics.ColumnStatisticMetadata;
@@ -169,23 +176,26 @@ public abstract class IcebergAbstractMetadata
169176
implements ConnectorMetadata
170177
{
171178
protected final TypeManager typeManager;
179+
protected final IProcedureRegistry procedureRegistry;
172180
protected final JsonCodec<CommitTaskData> commitTaskCodec;
173181
protected final NodeVersion nodeVersion;
174182
protected final RowExpressionService rowExpressionService;
175-
protected Transaction transaction;
183+
protected IcebergTransactionContext transactionContext;
176184

177185
private final StandardFunctionResolution functionResolution;
178186
private final ConcurrentMap<SchemaTableName, Table> icebergTables = new ConcurrentHashMap<>();
179187
private static final Logger log = Logger.get(IcebergAbstractMetadata.class);
180188

181189
public IcebergAbstractMetadata(
182190
TypeManager typeManager,
191+
IProcedureRegistry procedureRegistry,
183192
StandardFunctionResolution functionResolution,
184193
RowExpressionService rowExpressionService,
185194
JsonCodec<CommitTaskData> commitTaskCodec,
186195
NodeVersion nodeVersion)
187196
{
188197
this.typeManager = requireNonNull(typeManager, "typeManager is null");
198+
this.procedureRegistry = requireNonNull(procedureRegistry, "procedureRegistry is null");
189199
this.commitTaskCodec = requireNonNull(commitTaskCodec, "commitTaskCodec is null");
190200
this.functionResolution = requireNonNull(functionResolution, "functionResolution is null");
191201
this.rowExpressionService = requireNonNull(rowExpressionService, "rowExpressionService is null");
@@ -207,6 +217,11 @@ protected final Table getIcebergTable(ConnectorSession session, SchemaTableName
207217

208218
public abstract void unregisterTable(ConnectorSession clientSession, SchemaTableName schemaTableName);
209219

220+
public Optional<ConnectorSplitSource> getSplitSourceInCurrentCallProcedureTransaction()
221+
{
222+
return transactionContext == null ? Optional.empty() : transactionContext.getConnectorSplitSource();
223+
}
224+
210225
/**
211226
* This class implements the default implementation for getTableLayoutForConstraint which will be used in the case of a Java Worker
212227
*/
@@ -436,7 +451,7 @@ public Optional<ConnectorOutputMetadata> finishCreateTable(ConnectorSession sess
436451

437452
protected ConnectorInsertTableHandle beginIcebergTableInsert(ConnectorSession session, IcebergTableHandle table, Table icebergTable)
438453
{
439-
transaction = icebergTable.newTransaction();
454+
transactionContext = new IcebergTransactionContext(Optional.of(icebergTable), icebergTable.newTransaction());
440455

441456
return new IcebergInsertTableHandle(
442457
table.getSchemaName(),
@@ -459,11 +474,12 @@ public Optional<ConnectorOutputMetadata> finishInsert(ConnectorSession session,
459474
private Optional<ConnectorOutputMetadata> finishWrite(ConnectorSession session, IcebergWritableTableHandle writableTableHandle, Collection<Slice> fragments, Collection<ComputedStatistics> computedStatistics)
460475
{
461476
if (fragments.isEmpty()) {
462-
transaction.commitTransaction();
477+
transactionContext.getTransaction().commitTransaction();
478+
transactionContext.destroy();
463479
return Optional.empty();
464480
}
465481

466-
Table icebergTable = transaction.table();
482+
Table icebergTable = transactionContext.getTransaction().table();
467483

468484
List<CommitTaskData> commitTasks = fragments.stream()
469485
.map(slice -> commitTaskCodec.fromJson(slice.getBytes()))
@@ -474,7 +490,7 @@ private Optional<ConnectorOutputMetadata> finishWrite(ConnectorSession session,
474490
icebergTable.schema().findType(field.sourceId())))
475491
.toArray(Type[]::new);
476492

477-
AppendFiles appendFiles = transaction.newFastAppend();
493+
AppendFiles appendFiles = transactionContext.getTransaction().newFastAppend();
478494
for (CommitTaskData task : commitTasks) {
479495
DataFiles.Builder builder = DataFiles.builder(icebergTable.spec())
480496
.withPath(task.getPath())
@@ -492,7 +508,8 @@ private Optional<ConnectorOutputMetadata> finishWrite(ConnectorSession session,
492508
}
493509

494510
appendFiles.commit();
495-
transaction.commitTransaction();
511+
transactionContext.getTransaction().commitTransaction();
512+
transactionContext.destroy();
496513

497514
return Optional.of(new HiveWrittenPartitions(commitTasks.stream()
498515
.map(CommitTaskData::getPath)
@@ -805,6 +822,44 @@ public void truncateTable(ConnectorSession session, ConnectorTableHandle tableHa
805822
removeScanFiles(icebergTable, TupleDomain.all());
806823
}
807824

825+
@Override
826+
public ConnectorDistributedProcedureHandle beginCallDistributedProcedure(
827+
ConnectorSession session,
828+
QualifiedObjectName procedureName,
829+
ConnectorTableLayoutHandle tableLayoutHandle,
830+
Object[] arguments)
831+
{
832+
IcebergTableHandle handle = ((IcebergTableLayoutHandle) tableLayoutHandle).getTable();
833+
Table icebergTable = getIcebergTable(session, handle.getSchemaTableName());
834+
835+
if (handle.isSnapshotSpecified()) {
836+
throw new PrestoException(NOT_SUPPORTED, "This connector do not allow table execute at specified snapshot");
837+
}
838+
839+
transactionContext = new IcebergTransactionContext(Optional.of(icebergTable), icebergTable.newTransaction());
840+
Procedure procedure = procedureRegistry.resolve(
841+
new ConnectorId(procedureName.getCatalogName()),
842+
new SchemaTableName(
843+
procedureName.getSchemaName(),
844+
procedureName.getObjectName()));
845+
verify(procedure instanceof DistributedProcedure, "procedure must be DistributedProcedure");
846+
return ((DistributedProcedure) procedure).getBeginCallDistributedProcedure().begin(session, transactionContext, tableLayoutHandle, arguments);
847+
}
848+
849+
@Override
850+
public void finishCallDistributedProcedure(ConnectorSession session, ConnectorDistributedProcedureHandle procedureHandle, QualifiedObjectName procedureName, Collection<Slice> fragments)
851+
{
852+
Procedure procedure = procedureRegistry.resolve(
853+
new ConnectorId(procedureName.getCatalogName()),
854+
new SchemaTableName(
855+
procedureName.getSchemaName(),
856+
procedureName.getObjectName()));
857+
verify(procedure instanceof DistributedProcedure, "procedure must be DistributedProcedure");
858+
((DistributedProcedure) procedure).getFinishCallDistributedProcedure().finish(transactionContext, procedureHandle, fragments);
859+
transactionContext.getTransaction().commitTransaction();
860+
transactionContext.destroy();
861+
}
862+
808863
@Override
809864
public ConnectorTableHandle beginDelete(ConnectorSession session, ConnectorTableHandle tableHandle)
810865
{
@@ -825,7 +880,7 @@ public ConnectorTableHandle beginDelete(ConnectorSession session, ConnectorTable
825880
}
826881

827882
validateTableMode(session, icebergTable);
828-
transaction = icebergTable.newTransaction();
883+
transactionContext = new IcebergTransactionContext(Optional.of(icebergTable), icebergTable.newTransaction());
829884

830885
return handle;
831886
}
@@ -836,7 +891,7 @@ public void finishDelete(ConnectorSession session, ConnectorTableHandle tableHan
836891
IcebergTableHandle handle = (IcebergTableHandle) tableHandle;
837892
Table icebergTable = getIcebergTable(session, handle.getSchemaTableName());
838893

839-
RowDelta rowDelta = transaction.newRowDelta();
894+
RowDelta rowDelta = transactionContext.getTransaction().newRowDelta();
840895

841896
List<CommitTaskData> commitTasks = fragments.stream()
842897
.map(slice -> commitTaskCodec.fromJson(slice.getBytes()))
@@ -873,7 +928,8 @@ public void finishDelete(ConnectorSession session, ConnectorTableHandle tableHan
873928
}
874929

875930
rowDelta.commit();
876-
transaction.commitTransaction();
931+
transactionContext.getTransaction().commitTransaction();
932+
transactionContext.destroy();
877933
}
878934

879935
@Override
@@ -951,11 +1007,12 @@ public OptionalLong metadataDelete(ConnectorSession session, ConnectorTableHandl
9511007
*/
9521008
private OptionalLong removeScanFiles(Table icebergTable, TupleDomain<IcebergColumnHandle> predicate)
9531009
{
954-
transaction = icebergTable.newTransaction();
955-
DeleteFiles deleteFiles = transaction.newDelete()
1010+
transactionContext = new IcebergTransactionContext(Optional.of(icebergTable), icebergTable.newTransaction());
1011+
DeleteFiles deleteFiles = transactionContext.getTransaction().newDelete()
9561012
.deleteFromRowFilter(toIcebergExpression(predicate));
9571013
deleteFiles.commit();
958-
transaction.commitTransaction();
1014+
transactionContext.getTransaction().commitTransaction();
1015+
transactionContext.destroy();
9591016

9601017
Map<String, String> summary = icebergTable.currentSnapshot().summary();
9611018
long deletedRecords = Long.parseLong(summary.getOrDefault(DELETED_RECORDS_PROP, "0"));

0 commit comments

Comments
 (0)