Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Incremental processing implementation #315

Merged
merged 1 commit into from
Feb 2, 2020
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
20 changes: 20 additions & 0 deletions api/src/main/java/org/apache/iceberg/TableScan.java
Original file line number Diff line number Diff line change
Expand Up @@ -131,6 +131,25 @@ default TableScan select(String... columns) {
*/
Expression filter();

/**
* Create a table scan which can read append data from {@param fromSnapshotId}
* exclusive and up to {@param toSnapshotId} inclusive
* @param fromSnapshotId - the last snapshot id read by the user, exclusive
* @param toSnapshotId - read append data up to this snapshot id
* @return a table scan which can read append data from {@param fromSnapshotId}
* exclusive and up to {@param toSnapshotId} inclusive
*/
TableScan appendsBetween(long fromSnapshotId, long toSnapshotId);

/**
* Create a table scan which can read append data from {@param fromSnapshotId}
* exclusive and up to the current snapshot id exclusive
* @param fromSnapshotId - the last snapshot id read by the user, exclusive
* @return a table scan which can read append data from {@param fromSnapshotId}
* exclusive and up to current snapshot inclusive
*/
TableScan appendsAfter(long fromSnapshotId);

/**
* Plan the {@link FileScanTask files} that will be read by this scan.
* <p>
Expand Down Expand Up @@ -180,4 +199,5 @@ default TableScan select(String... columns) {
* @return true if case sensitive, false otherwise.
*/
boolean isCaseSensitive();

rdsr marked this conversation as resolved.
Show resolved Hide resolved
}
32 changes: 31 additions & 1 deletion core/src/main/java/org/apache/iceberg/BaseTableScan.java
Original file line number Diff line number Diff line change
Expand Up @@ -80,6 +80,26 @@ protected BaseTableScan(TableOperations ops, Table table, Long snapshotId, Schem
this.options = options != null ? options : ImmutableMap.of();
}

protected TableOperations tableOps() {
return ops;
}

protected Long snapshotId() {
return snapshotId;
}

protected boolean colStats() {
return colStats;
}

protected Collection<String> selectedColumns() {
return selectedColumns;
}

protected ImmutableMap<String, String> options() {
return options;
}

@SuppressWarnings("checkstyle:HiddenField")
protected abstract long targetSplitSize(TableOperations ops);

Expand All @@ -98,10 +118,20 @@ public Table table() {
return table;
}

@Override
public TableScan appendsBetween(long fromSnapshotId, long toSnapshotId) {
throw new UnsupportedOperationException("Incremental scan is not supported");
}

@Override
public TableScan appendsAfter(long fromSnapshotId) {
throw new UnsupportedOperationException("Incremental scan is not supported");
}

@Override
public TableScan useSnapshot(long scanSnapshotId) {
Preconditions.checkArgument(this.snapshotId == null,
"Cannot override snapshot, already set to id=%s", scanSnapshotId);
"Cannot override snapshot, already set to id=%s", snapshotId);
Preconditions.checkArgument(ops.current().snapshot(scanSnapshotId) != null,
"Cannot find snapshot with ID %s", scanSnapshotId);
return newRefinedScan(
Expand Down
70 changes: 31 additions & 39 deletions core/src/main/java/org/apache/iceberg/DataTableScan.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,33 +19,28 @@

package org.apache.iceberg;

import com.github.benmanes.caffeine.cache.Caffeine;
import com.github.benmanes.caffeine.cache.LoadingCache;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import java.util.Collection;
import org.apache.iceberg.expressions.Expression;
import org.apache.iceberg.expressions.ManifestEvaluator;
import org.apache.iceberg.expressions.ResidualEvaluator;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.util.ParallelIterable;
import org.apache.iceberg.util.ThreadPools;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class DataTableScan extends BaseTableScan {
private static final Logger LOG = LoggerFactory.getLogger(DataTableScan.class);

private static final ImmutableList<String> SCAN_COLUMNS = ImmutableList.of(
static final ImmutableList<String> SCAN_COLUMNS = ImmutableList.of(
"snapshot_id", "file_path", "file_ordinal", "file_format", "block_size_in_bytes",
"file_size_in_bytes", "record_count", "partition", "key_metadata"
);
private static final ImmutableList<String> SCAN_WITH_STATS_COLUMNS = ImmutableList.<String>builder()
static final ImmutableList<String> SCAN_WITH_STATS_COLUMNS = ImmutableList.<String>builder()
.addAll(SCAN_COLUMNS)
.add("value_counts", "null_value_counts", "lower_bounds", "upper_bounds", "column_sizes")
.build();
private static final boolean PLAN_SCANS_WITH_WORKER_POOL =
static final boolean PLAN_SCANS_WITH_WORKER_POOL =
SystemProperties.getBoolean(SystemProperties.SCAN_THREAD_POOL_ENABLED, true);

public DataTableScan(TableOperations ops, Table table) {
Expand All @@ -58,6 +53,24 @@ protected DataTableScan(TableOperations ops, Table table, Long snapshotId, Schem
super(ops, table, snapshotId, schema, rowFilter, caseSensitive, colStats, selectedColumns, options);
}

@Override
public TableScan appendsBetween(long fromSnapshotId, long toSnapshotId) {
Long scanSnapshotId = snapshotId();
Preconditions.checkState(scanSnapshotId == null,
"Cannot enable incremental scan, scan-snapshot set to id=%s", scanSnapshotId);
rdblue marked this conversation as resolved.
Show resolved Hide resolved
return new IncrementalDataTableScan(
tableOps(), table(), schema(), filter(), isCaseSensitive(), colStats(), selectedColumns(), options(),
fromSnapshotId, toSnapshotId);
}

@Override
public TableScan appendsAfter(long fromSnapshotId) {
Snapshot currentSnapshot = table().currentSnapshot();
Preconditions.checkState(currentSnapshot != null, "Cannot scan appends after %s, there is no current snapshot",
fromSnapshotId);
return appendsBetween(fromSnapshotId, currentSnapshot.snapshotId());
}

@Override
protected TableScan newRefinedScan(
TableOperations ops, Table table, Long snapshotId, Schema schema, Expression rowFilter,
Expand All @@ -70,39 +83,18 @@ protected TableScan newRefinedScan(
@Override
public CloseableIterable<FileScanTask> planFiles(TableOperations ops, Snapshot snapshot,
Expression rowFilter, boolean caseSensitive, boolean colStats) {
LoadingCache<Integer, ManifestEvaluator> evalCache = Caffeine.newBuilder().build(specId -> {
PartitionSpec spec = ops.current().spec(specId);
return ManifestEvaluator.forRowFilter(rowFilter, spec, caseSensitive);
});

Iterable<ManifestFile> nonEmptyManifests = Iterables.filter(snapshot.manifests(),
manifest -> manifest.hasAddedFiles() || manifest.hasExistingFiles());
Iterable<ManifestFile> matchingManifests = Iterables.filter(nonEmptyManifests,
manifest -> evalCache.get(manifest.partitionSpecId()).eval(manifest));

Iterable<CloseableIterable<FileScanTask>> readers = Iterables.transform(
matchingManifests,
manifest -> {
ManifestReader reader = ManifestReader.read(
ops.io().newInputFile(manifest.path()),
ops.current().specsById());
PartitionSpec spec = ops.current().spec(manifest.partitionSpecId());
String schemaString = SchemaParser.toJson(spec.schema());
String specString = PartitionSpecParser.toJson(spec);
ResidualEvaluator residuals = ResidualEvaluator.of(spec, rowFilter, caseSensitive);
return CloseableIterable.transform(
reader.filterRows(rowFilter)
.caseSensitive(caseSensitive)
.select(colStats ? SCAN_WITH_STATS_COLUMNS : SCAN_COLUMNS),
file -> new BaseFileScanTask(file, schemaString, specString, residuals)
);
});
ManifestGroup manifestGroup = new ManifestGroup(ops.io(), snapshot.manifests())
.caseSensitive(caseSensitive)
.select(colStats ? SCAN_WITH_STATS_COLUMNS : SCAN_COLUMNS)
.filterData(rowFilter)
.specsById(ops.current().specsById())
.ignoreDeleted();

if (PLAN_SCANS_WITH_WORKER_POOL && snapshot.manifests().size() > 1) {
return new ParallelIterable<>(readers, ThreadPools.getWorkerPool());
} else {
return CloseableIterable.concat(readers);
manifestGroup = manifestGroup.planWith(ThreadPools.getWorkerPool());
}

return manifestGroup.planFiles();
}

@Override
Expand Down
140 changes: 140 additions & 0 deletions core/src/main/java/org/apache/iceberg/IncrementalDataTableScan.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,140 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 org.apache.iceberg;

import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import java.util.Collection;
import java.util.List;
import org.apache.iceberg.expressions.Expression;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.util.SnapshotUtil;
import org.apache.iceberg.util.ThreadPools;

class IncrementalDataTableScan extends DataTableScan {
private long fromSnapshotId;
private long toSnapshotId;

IncrementalDataTableScan(TableOperations ops, Table table, Schema schema,
Expression rowFilter, boolean caseSensitive, boolean colStats,
Collection<String> selectedColumns, ImmutableMap<String, String> options,
long fromSnapshotId, long toSnapshotId) {
super(ops, table, null, schema, rowFilter, caseSensitive, colStats, selectedColumns, options);
Preconditions.checkArgument(fromSnapshotId != toSnapshotId, "fromSnapshotId and toSnapshotId cannot be the same");
this.fromSnapshotId = fromSnapshotId;
this.toSnapshotId = toSnapshotId;
}

@Override
public TableScan asOfTime(long timestampMillis) {
throw new UnsupportedOperationException(String.format(
"Cannot scan table as of time %s: configured for incremental data in snapshots (%s, %s]",
timestampMillis, fromSnapshotId, toSnapshotId));
}

@Override
public TableScan useSnapshot(long scanSnapshotId) {
throw new UnsupportedOperationException(String.format(
"Cannot scan table using scan snapshot id %s: configured for incremental data in snapshots (%s, %s]",
scanSnapshotId, fromSnapshotId, toSnapshotId));
}

@Override
public TableScan appendsBetween(long newFromSnapshotId, long newToSnapshotId) {
Preconditions.checkArgument(
table().snapshot(newFromSnapshotId) != null, "fromSnapshotId: %s does not exist", newFromSnapshotId);
Preconditions.checkArgument(
table().snapshot(newToSnapshotId) != null, "toSnapshotId: %s does not exist", newToSnapshotId);
return new IncrementalDataTableScan(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should this ensure that newFromSnapshot is an ancestor of newToSnapshot?

This probably doesn't need to be done in this commit, but it would be a good follow-up to ensure that the range exists.

Since this is a refinement, it may also be a good idea to make this a subset of the existing selected range. That is, both newFromSnapshotId and newToSnapshotId must be in the existing range of fromSnapshotId to toSnapshotId.

Putting it another way, when I create a scan using appendsBetween(A, C).appendsBetween(B, D), what should the behavior be? I'd say that is concerning because D is outside the original range. Probably a good idea to fail instead.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thankyou, I'll file ticket for followup!

tableOps(), table(), schema(), filter(), isCaseSensitive(), colStats(), selectedColumns(), options(),
newFromSnapshotId, newToSnapshotId);
}

@Override
public TableScan appendsAfter(long newFromSnapshotId) {
final Snapshot currentSnapshot = table().currentSnapshot();
Preconditions.checkState(currentSnapshot != null,
"Cannot scan appends after %s, there is no current snapshot", newFromSnapshotId);
return appendsBetween(newFromSnapshotId, currentSnapshot.snapshotId());
}

@Override
public CloseableIterable<FileScanTask> planFiles() {
//TODO publish an incremental appends scan event
List<Snapshot> snapshots = snapshotsWithin(table(), fromSnapshotId, toSnapshotId);
Iterable<CloseableIterable<FileScanTask>> files = Iterables.transform(snapshots, this::planFiles);
return CloseableIterable.concat(files);
}

private CloseableIterable<FileScanTask> planFiles(Snapshot snapshot) {
Predicate<ManifestFile> matchingManifests = manifest -> manifest.snapshotId() == snapshot.snapshotId();

Predicate<ManifestEntry> matchingManifestEntries =
manifestEntry ->
manifestEntry.snapshotId() == snapshot.snapshotId() &&
manifestEntry.status() == ManifestEntry.Status.ADDED;

ManifestGroup manifestGroup = new ManifestGroup(tableOps().io(), snapshot.manifests())
.caseSensitive(isCaseSensitive())
.select(colStats() ? SCAN_WITH_STATS_COLUMNS : SCAN_COLUMNS)
.filterData(filter())
.filterManifests(matchingManifests)
.filterManifestEntries(matchingManifestEntries)
.specsById(tableOps().current().specsById())
.ignoreDeleted();

if (PLAN_SCANS_WITH_WORKER_POOL && snapshot.manifests().size() > 1) {
manifestGroup = manifestGroup.planWith(ThreadPools.getWorkerPool());
}

return manifestGroup.planFiles();
}

@Override
@SuppressWarnings("checkstyle:HiddenField")
protected TableScan newRefinedScan(
TableOperations ops, Table table, Long snapshotId, Schema schema, Expression rowFilter,
boolean caseSensitive, boolean colStats, Collection<String> selectedColumns,
ImmutableMap<String, String> options) {
return new IncrementalDataTableScan(
ops, table, schema, rowFilter, caseSensitive, colStats, selectedColumns, options,
fromSnapshotId, toSnapshotId);
}

private static List<Snapshot> snapshotsWithin(Table table, long fromSnapshotId, long toSnapshotId) {
List<Long> snapshotIds = SnapshotUtil.snapshotIdsBetween(table, fromSnapshotId, toSnapshotId);
List<Snapshot> snapshots = Lists.newArrayList();
for (Long snapshotId : snapshotIds) {
Snapshot snapshot = table.snapshot(snapshotId);
// for now, incremental scan supports only appends
if (snapshot.operation().equals(DataOperations.APPEND)) {
snapshots.add(snapshot);
} else if (snapshot.operation().equals(DataOperations.OVERWRITE)) {
throw new UnsupportedOperationException(
String.format("Found %s operation, cannot support incremental data in snapshots (%s, %s]",
DataOperations.OVERWRITE, fromSnapshotId, toSnapshotId));
}
rdsr marked this conversation as resolved.
Show resolved Hide resolved
}
return snapshots;
}
}