-
Notifications
You must be signed in to change notification settings - Fork 5.4k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Refactor Iceberg connector to support call distributed procedure
- Loading branch information
1 parent
5fb19a6
commit 06fd562
Showing
16 changed files
with
415 additions
and
19 deletions.
There are no files selected for viewing
135 changes: 135 additions & 0 deletions
135
...ceberg/src/main/java/com/facebook/presto/iceberg/CallDistributedProcedureSplitSource.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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())); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
51 changes: 51 additions & 0 deletions
51
...-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergDistributedProcedureHandle.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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); | ||
} | ||
} |
Oops, something went wrong.