Skip to content

Commit

Permalink
Refactor Iceberg connector to support call distributed procedure
Browse files Browse the repository at this point in the history
  • Loading branch information
hantangwangd committed Nov 3, 2024
1 parent 5fb19a6 commit 06fd562
Show file tree
Hide file tree
Showing 16 changed files with 415 additions and 19 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,135 @@
/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.facebook.presto.iceberg;

import com.facebook.presto.iceberg.delete.DeleteFile;
import com.facebook.presto.spi.ConnectorSession;
import com.facebook.presto.spi.ConnectorSplit;
import com.facebook.presto.spi.ConnectorSplitSource;
import com.facebook.presto.spi.SplitWeight;
import com.facebook.presto.spi.connector.ConnectorPartitionHandle;
import com.google.common.collect.ImmutableList;
import com.google.common.io.Closer;
import org.apache.iceberg.FileScanTask;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.PartitionSpecParser;
import org.apache.iceberg.TableScan;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.io.CloseableIterator;

import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.function.Consumer;

import static com.facebook.presto.hive.HiveCommonSessionProperties.getNodeSelectionStrategy;
import static com.facebook.presto.iceberg.FileFormat.fromIcebergFileFormat;
import static com.facebook.presto.iceberg.IcebergUtil.getDataSequenceNumber;
import static com.facebook.presto.iceberg.IcebergUtil.getPartitionKeys;
import static com.facebook.presto.iceberg.IcebergUtil.partitionDataFromStructLike;
import static com.google.common.collect.ImmutableList.toImmutableList;
import static com.google.common.collect.Iterators.limit;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.CompletableFuture.completedFuture;

public class CallDistributedProcedureSplitSource
implements ConnectorSplitSource
{
private CloseableIterator<FileScanTask> fileScanTaskIterator;
private Optional<Consumer<FileScanTask>> fileScanTaskConsumer;

private final TableScan tableScan;
private final Closer closer = Closer.create();
private final double minimumAssignedSplitWeight;
private final ConnectorSession session;

public CallDistributedProcedureSplitSource(
ConnectorSession session,
TableScan tableScan,
CloseableIterable<FileScanTask> fileScanTaskIterable,
Optional<Consumer<FileScanTask>> fileScanTaskConsumer,
double minimumAssignedSplitWeight)
{
this.session = requireNonNull(session, "session is null");
this.tableScan = requireNonNull(tableScan, "tableScan is null");
this.fileScanTaskIterator = fileScanTaskIterable.iterator();
this.fileScanTaskConsumer = requireNonNull(fileScanTaskConsumer, "fileScanTaskConsumer is null");
this.minimumAssignedSplitWeight = minimumAssignedSplitWeight;
closer.register(fileScanTaskIterator);
}

@Override
public CompletableFuture<ConnectorSplitBatch> getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize)
{
// TODO: move this to a background thread
List<ConnectorSplit> splits = new ArrayList<>();
Iterator<FileScanTask> iterator = limit(fileScanTaskIterator, maxSize);
while (iterator.hasNext()) {
FileScanTask task = iterator.next();
fileScanTaskConsumer.ifPresent(consumer -> consumer.accept(task));
splits.add(toIcebergSplit(task));
}
return completedFuture(new ConnectorSplitBatch(splits, isFinished()));
}

@Override
public boolean isFinished()
{
return !fileScanTaskIterator.hasNext();
}

@Override
public void close()
{
try {
closer.close();
// TODO: remove this after org.apache.iceberg.io.CloseableIterator'withClose
// correct release resources holds by iterator.
fileScanTaskIterator = CloseableIterator.empty();
}
catch (IOException e) {
throw new UncheckedIOException(e);
}
}

private ConnectorSplit toIcebergSplit(FileScanTask task)
{
PartitionSpec spec = task.spec();
Optional<PartitionData> partitionData = partitionDataFromStructLike(spec, task.file().partition());

// TODO: We should leverage residual expression and convert that to TupleDomain.
// The predicate here is used by readers for predicate push down at reader level,
// so when we do not use residual expression, we are just wasting CPU cycles
// on reader side evaluating a condition that we know will always be true.

return new IcebergSplit(
task.file().path().toString(),
task.start(),
task.length(),
fromIcebergFileFormat(task.file().format()),
ImmutableList.of(),
getPartitionKeys(task),
PartitionSpecParser.toJson(spec),
partitionData.map(PartitionData::toJson),
getNodeSelectionStrategy(session),
SplitWeight.fromProportion(Math.min(Math.max((double) task.length() / tableScan.targetSplitSize(), minimumAssignedSplitWeight), 1.0)),
task.deletes().stream().map(DeleteFile::fromIceberg).collect(toImmutableList()),
Optional.empty(),
getDataSequenceNumber(task.file()));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@

import com.facebook.airlift.json.JsonCodec;
import com.facebook.airlift.log.Logger;
import com.facebook.presto.common.QualifiedObjectName;
import com.facebook.presto.common.Subfield;
import com.facebook.presto.common.predicate.TupleDomain;
import com.facebook.presto.common.type.BigintType;
Expand All @@ -30,10 +31,13 @@
import com.facebook.presto.iceberg.statistics.StatisticsFileCache;
import com.facebook.presto.spi.ColumnHandle;
import com.facebook.presto.spi.ColumnMetadata;
import com.facebook.presto.spi.ConnectorDistributedProcedureHandle;
import com.facebook.presto.spi.ConnectorId;
import com.facebook.presto.spi.ConnectorInsertTableHandle;
import com.facebook.presto.spi.ConnectorNewTableLayout;
import com.facebook.presto.spi.ConnectorOutputTableHandle;
import com.facebook.presto.spi.ConnectorSession;
import com.facebook.presto.spi.ConnectorSplitSource;
import com.facebook.presto.spi.ConnectorTableHandle;
import com.facebook.presto.spi.ConnectorTableLayout;
import com.facebook.presto.spi.ConnectorTableLayoutHandle;
Expand All @@ -53,6 +57,9 @@
import com.facebook.presto.spi.connector.ConnectorTableVersion.VersionType;
import com.facebook.presto.spi.function.StandardFunctionResolution;
import com.facebook.presto.spi.plan.FilterStatsCalculatorService;
import com.facebook.presto.spi.procedure.DistributedProcedure;
import com.facebook.presto.spi.procedure.IProcedureRegistry;
import com.facebook.presto.spi.procedure.Procedure;
import com.facebook.presto.spi.relation.RowExpression;
import com.facebook.presto.spi.relation.RowExpressionService;
import com.facebook.presto.spi.statistics.ColumnStatisticMetadata;
Expand Down Expand Up @@ -180,18 +187,20 @@ public abstract class IcebergAbstractMetadata
private static final Logger log = Logger.get(IcebergAbstractMetadata.class);

protected final TypeManager typeManager;
protected final IProcedureRegistry procedureRegistry;
protected final JsonCodec<CommitTaskData> commitTaskCodec;
protected final NodeVersion nodeVersion;
protected final RowExpressionService rowExpressionService;
protected final FilterStatsCalculatorService filterStatsCalculatorService;
protected Transaction transaction;
protected IcebergTransactionContext transactionContext;
protected final StatisticsFileCache statisticsFileCache;

private final StandardFunctionResolution functionResolution;
private final ConcurrentMap<SchemaTableName, Table> icebergTables = new ConcurrentHashMap<>();

public IcebergAbstractMetadata(
TypeManager typeManager,
IProcedureRegistry procedureRegistry,
StandardFunctionResolution functionResolution,
RowExpressionService rowExpressionService,
JsonCodec<CommitTaskData> commitTaskCodec,
Expand All @@ -200,6 +209,7 @@ public IcebergAbstractMetadata(
StatisticsFileCache statisticsFileCache)
{
this.typeManager = requireNonNull(typeManager, "typeManager is null");
this.procedureRegistry = requireNonNull(procedureRegistry, "procedureRegistry is null");
this.commitTaskCodec = requireNonNull(commitTaskCodec, "commitTaskCodec is null");
this.functionResolution = requireNonNull(functionResolution, "functionResolution is null");
this.rowExpressionService = requireNonNull(rowExpressionService, "rowExpressionService is null");
Expand All @@ -223,6 +233,11 @@ protected final Table getIcebergTable(ConnectorSession session, SchemaTableName

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

public Optional<ConnectorSplitSource> getSplitSourceInCurrentCallProcedureTransaction()
{
return transactionContext == null ? Optional.empty() : transactionContext.getConnectorSplitSource();
}

/**
* This class implements the default implementation for getTableLayoutForConstraint which will be used in the case of a Java Worker
*/
Expand Down Expand Up @@ -454,7 +469,7 @@ public Optional<ConnectorOutputMetadata> finishCreateTable(ConnectorSession sess

protected ConnectorInsertTableHandle beginIcebergTableInsert(ConnectorSession session, IcebergTableHandle table, Table icebergTable)
{
transaction = icebergTable.newTransaction();
transactionContext = new IcebergTransactionContext(Optional.of(icebergTable), icebergTable.newTransaction());

return new IcebergInsertTableHandle(
table.getSchemaName(),
Expand All @@ -477,11 +492,12 @@ public Optional<ConnectorOutputMetadata> finishInsert(ConnectorSession session,
private Optional<ConnectorOutputMetadata> finishWrite(ConnectorSession session, IcebergWritableTableHandle writableTableHandle, Collection<Slice> fragments, Collection<ComputedStatistics> computedStatistics)
{
if (fragments.isEmpty()) {
transaction.commitTransaction();
transactionContext.getTransaction().commitTransaction();
transactionContext.destroy();
return Optional.empty();
}

Table icebergTable = transaction.table();
Table icebergTable = transactionContext.getTransaction().table();

List<CommitTaskData> commitTasks = fragments.stream()
.map(slice -> commitTaskCodec.fromJson(slice.getBytes()))
Expand All @@ -492,7 +508,7 @@ private Optional<ConnectorOutputMetadata> finishWrite(ConnectorSession session,
icebergTable.schema().findType(field.sourceId())))
.toArray(Type[]::new);

AppendFiles appendFiles = transaction.newFastAppend();
AppendFiles appendFiles = transactionContext.getTransaction().newFastAppend();
for (CommitTaskData task : commitTasks) {
DataFiles.Builder builder = DataFiles.builder(icebergTable.spec())
.withPath(task.getPath())
Expand All @@ -510,7 +526,8 @@ private Optional<ConnectorOutputMetadata> finishWrite(ConnectorSession session,
}

appendFiles.commit();
transaction.commitTransaction();
transactionContext.getTransaction().commitTransaction();
transactionContext.destroy();

return Optional.of(new HiveWrittenPartitions(commitTasks.stream()
.map(CommitTaskData::getPath)
Expand Down Expand Up @@ -815,6 +832,44 @@ public void truncateTable(ConnectorSession session, ConnectorTableHandle tableHa
removeScanFiles(icebergTable, TupleDomain.all());
}

@Override
public ConnectorDistributedProcedureHandle beginCallDistributedProcedure(
ConnectorSession session,
QualifiedObjectName procedureName,
ConnectorTableLayoutHandle tableLayoutHandle,
Object[] arguments)
{
IcebergTableHandle handle = ((IcebergTableLayoutHandle) tableLayoutHandle).getTable();
Table icebergTable = getIcebergTable(session, handle.getSchemaTableName());

if (handle.isSnapshotSpecified()) {
throw new PrestoException(NOT_SUPPORTED, "This connector do not allow table execute at specified snapshot");
}

transactionContext = new IcebergTransactionContext(Optional.of(icebergTable), icebergTable.newTransaction());
Procedure procedure = procedureRegistry.resolve(
new ConnectorId(procedureName.getCatalogName()),
new SchemaTableName(
procedureName.getSchemaName(),
procedureName.getObjectName()));
verify(procedure instanceof DistributedProcedure, "procedure must be DistributedProcedure");
return ((DistributedProcedure) procedure).getBeginCallDistributedProcedure().begin(session, transactionContext, tableLayoutHandle, arguments);
}

@Override
public void finishCallDistributedProcedure(ConnectorSession session, ConnectorDistributedProcedureHandle procedureHandle, QualifiedObjectName procedureName, Collection<Slice> fragments)
{
Procedure procedure = procedureRegistry.resolve(
new ConnectorId(procedureName.getCatalogName()),
new SchemaTableName(
procedureName.getSchemaName(),
procedureName.getObjectName()));
verify(procedure instanceof DistributedProcedure, "procedure must be DistributedProcedure");
((DistributedProcedure) procedure).getFinishCallDistributedProcedure().finish(transactionContext, procedureHandle, fragments);
transactionContext.getTransaction().commitTransaction();
transactionContext.destroy();
}

@Override
public ConnectorTableHandle beginDelete(ConnectorSession session, ConnectorTableHandle tableHandle)
{
Expand All @@ -835,7 +890,7 @@ public ConnectorTableHandle beginDelete(ConnectorSession session, ConnectorTable
}

validateTableMode(session, icebergTable);
transaction = icebergTable.newTransaction();
transactionContext = new IcebergTransactionContext(Optional.of(icebergTable), icebergTable.newTransaction());

return handle;
}
Expand All @@ -846,7 +901,7 @@ public void finishDelete(ConnectorSession session, ConnectorTableHandle tableHan
IcebergTableHandle handle = (IcebergTableHandle) tableHandle;
Table icebergTable = getIcebergTable(session, handle.getSchemaTableName());

RowDelta rowDelta = transaction.newRowDelta();
RowDelta rowDelta = transactionContext.getTransaction().newRowDelta();

List<CommitTaskData> commitTasks = fragments.stream()
.map(slice -> commitTaskCodec.fromJson(slice.getBytes()))
Expand Down Expand Up @@ -883,7 +938,8 @@ public void finishDelete(ConnectorSession session, ConnectorTableHandle tableHan
}

rowDelta.commit();
transaction.commitTransaction();
transactionContext.getTransaction().commitTransaction();
transactionContext.destroy();
}

@Override
Expand Down Expand Up @@ -961,11 +1017,12 @@ public OptionalLong metadataDelete(ConnectorSession session, ConnectorTableHandl
*/
private OptionalLong removeScanFiles(Table icebergTable, TupleDomain<IcebergColumnHandle> predicate)
{
transaction = icebergTable.newTransaction();
DeleteFiles deleteFiles = transaction.newDelete()
transactionContext = new IcebergTransactionContext(Optional.of(icebergTable), icebergTable.newTransaction());
DeleteFiles deleteFiles = transactionContext.getTransaction().newDelete()
.deleteFromRowFilter(toIcebergExpression(predicate));
deleteFiles.commit();
transaction.commitTransaction();
transactionContext.getTransaction().commitTransaction();
transactionContext.destroy();

Map<String, String> summary = icebergTable.currentSnapshot().summary();
long deletedRecords = Long.parseLong(summary.getOrDefault(DELETED_RECORDS_PROP, "0"));
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.facebook.presto.iceberg;

import com.facebook.presto.hive.HiveCompressionCodec;
import com.facebook.presto.spi.ConnectorDistributedProcedureHandle;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;

import java.util.List;
import java.util.Map;

public class IcebergDistributedProcedureHandle
extends IcebergWritableTableHandle
implements ConnectorDistributedProcedureHandle
{
@JsonCreator
public IcebergDistributedProcedureHandle(
@JsonProperty("schemaName") String schemaName,
@JsonProperty("tableName") IcebergTableName tableName,
@JsonProperty("schema") PrestoIcebergSchema schema,
@JsonProperty("partitionSpec") PrestoIcebergPartitionSpec partitionSpec,
@JsonProperty("inputColumns") List<IcebergColumnHandle> inputColumns,
@JsonProperty("outputPath") String outputPath,
@JsonProperty("fileFormat") FileFormat fileFormat,
@JsonProperty("compressionCodec") HiveCompressionCodec compressionCodec,
@JsonProperty("storageProperties") Map<String, String> storageProperties)
{
super(
schemaName,
tableName,
schema,
partitionSpec,
inputColumns,
outputPath,
fileFormat,
compressionCodec,
storageProperties);
}
}
Loading

0 comments on commit 06fd562

Please sign in to comment.