-
Notifications
You must be signed in to change notification settings - Fork 2.1k
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
Change Data Capture(CDC)[Draft] #4539
base: main
Are you sure you want to change the base?
Changes from all commits
c31b81f
28002f6
86d6c8b
2c4daf0
8cf9678
5c28931
12be09d
0dd085b
fb91731
4ca354c
02aef3c
172f9e5
5e0d03a
4756489
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,64 @@ | ||
/* | ||
* 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.actions; | ||
|
||
public interface GenerateChangeSet extends Action<GenerateChangeSet, GenerateChangeSet.Result> { | ||
/** | ||
* Emit changed data set by a snapshot id. | ||
* | ||
* @param snapshotId id of the snapshot to generate changed data | ||
* @return this for method chaining | ||
*/ | ||
GenerateChangeSet forSnapshot(long snapshotId); | ||
|
||
/** | ||
* Emit changed data set for the current snapshot. | ||
* | ||
* @return this for method chaining | ||
*/ | ||
GenerateChangeSet forCurrentSnapshot(); | ||
|
||
/** | ||
* Emit changed data from a particular snapshot(exclusive). | ||
* | ||
* @param fromSnapshotId id of the start snapshot | ||
* @return this for method chaining | ||
*/ | ||
GenerateChangeSet afterSnapshot(long fromSnapshotId); | ||
|
||
/** | ||
* Emit change data set from the start snapshot (exclusive) to the end snapshot (inclusive). | ||
* | ||
* @param fromSnapshotId id of the start snapshot | ||
* @param toSnapshotId id of the end snapshot | ||
* @return this for method chaining | ||
*/ | ||
GenerateChangeSet betweenSnapshots(long fromSnapshotId, long toSnapshotId); | ||
|
||
/** | ||
* The action result that contains a dataset of changed rows. | ||
*/ | ||
interface Result<T> { | ||
/** | ||
* Returns the change set. | ||
*/ | ||
T changeSet(); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -517,5 +517,32 @@ public void setBatchSize(int batchSize) { | |
} | ||
} | ||
|
||
/** | ||
* A Dummy Vector Reader which doesn't actually read files, instead it returns a dummy | ||
* VectorHolder which indicates whether the row is deleted. | ||
*/ | ||
public static class DeletedVectorReader extends VectorizedArrowReader { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. A basic question, do we need to enable vectorization to read CDC? What about the non-vectorized reader? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Vectorized read is significant faster than non-vectorized read, check benchmark in #3287. We should have it. Non-vectorized read is handled by the change I made in class |
||
public DeletedVectorReader() { | ||
} | ||
|
||
@Override | ||
public VectorHolder read(VectorHolder reuse, int numValsToRead) { | ||
return VectorHolder.deleteMetaColumnHolder(numValsToRead); | ||
} | ||
|
||
@Override | ||
public void setRowGroupInfo(PageReadStore source, Map<ColumnPath, ColumnChunkMetaData> metadata, long rowPosition) { | ||
} | ||
|
||
@Override | ||
public String toString() { | ||
return "DeletedVectorReader"; | ||
} | ||
|
||
@Override | ||
public void setBatchSize(int batchSize) { | ||
} | ||
} | ||
|
||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -41,7 +41,7 @@ | |
import org.apache.iceberg.types.Types; | ||
import org.apache.iceberg.util.ParallelIterable; | ||
|
||
class ManifestGroup { | ||
public class ManifestGroup { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If we use scan API, we may not need to expose ManifestGroup as public There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It'd be awesome. We have to design the scan API nicely though. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I am not sure about extending There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @aokolnychyi we are not talking about extending |
||
private static final Types.StructType EMPTY_STRUCT = Types.StructType.of(); | ||
|
||
private final FileIO io; | ||
|
@@ -55,6 +55,8 @@ class ManifestGroup { | |
private Expression partitionFilter; | ||
private boolean ignoreDeleted; | ||
private boolean ignoreExisting; | ||
private boolean ignoreAdded; | ||
private boolean onlyWithRowLevelDeletes; | ||
private boolean ignoreResiduals; | ||
private List<String> columns; | ||
private boolean caseSensitive; | ||
|
@@ -66,7 +68,7 @@ class ManifestGroup { | |
Iterables.filter(manifests, manifest -> manifest.content() == ManifestContent.DELETES)); | ||
} | ||
|
||
ManifestGroup(FileIO io, Iterable<ManifestFile> dataManifests, Iterable<ManifestFile> deleteManifests) { | ||
public ManifestGroup(FileIO io, Iterable<ManifestFile> dataManifests, Iterable<ManifestFile> deleteManifests) { | ||
this.io = io; | ||
this.dataManifests = Sets.newHashSet(dataManifests); | ||
this.deleteIndexBuilder = DeleteFileIndex.builderFor(io, deleteManifests); | ||
|
@@ -75,20 +77,22 @@ class ManifestGroup { | |
this.partitionFilter = Expressions.alwaysTrue(); | ||
this.ignoreDeleted = false; | ||
this.ignoreExisting = false; | ||
this.ignoreAdded = false; | ||
this.onlyWithRowLevelDeletes = false; | ||
this.ignoreResiduals = false; | ||
this.columns = ManifestReader.ALL_COLUMNS; | ||
this.caseSensitive = true; | ||
this.manifestPredicate = m -> true; | ||
this.manifestEntryPredicate = e -> true; | ||
} | ||
|
||
ManifestGroup specsById(Map<Integer, PartitionSpec> newSpecsById) { | ||
public ManifestGroup specsById(Map<Integer, PartitionSpec> newSpecsById) { | ||
this.specsById = newSpecsById; | ||
deleteIndexBuilder.specsById(newSpecsById); | ||
return this; | ||
} | ||
|
||
ManifestGroup filterData(Expression newDataFilter) { | ||
public ManifestGroup filterData(Expression newDataFilter) { | ||
this.dataFilter = Expressions.and(dataFilter, newDataFilter); | ||
deleteIndexBuilder.filterData(newDataFilter); | ||
return this; | ||
|
@@ -125,6 +129,16 @@ ManifestGroup ignoreExisting() { | |
return this; | ||
} | ||
|
||
public ManifestGroup ignoreAdded() { | ||
this.ignoreAdded = true; | ||
return this; | ||
} | ||
|
||
public ManifestGroup onlyWithRowLevelDeletes() { | ||
this.onlyWithRowLevelDeletes = true; | ||
return this; | ||
} | ||
|
||
ManifestGroup ignoreResiduals() { | ||
this.ignoreResiduals = true; | ||
return this; | ||
|
@@ -180,7 +194,7 @@ public CloseableIterable<FileScanTask> planFiles() { | |
return CloseableIterable.transform(entries, e -> new BaseFileScanTask( | ||
e.file().copy(), deleteFiles.forEntry(e), schemaString, specString, residuals)); | ||
} | ||
}); | ||
}, deleteFiles); | ||
|
||
if (executorService != null) { | ||
return new ParallelIterable<>(tasks, executorService); | ||
|
@@ -198,11 +212,13 @@ public CloseableIterable<FileScanTask> planFiles() { | |
* @return a CloseableIterable of manifest entries. | ||
*/ | ||
public CloseableIterable<ManifestEntry<DataFile>> entries() { | ||
return CloseableIterable.concat(entries((manifest, entries) -> entries)); | ||
return CloseableIterable.concat(entries((manifest, entries) -> entries, null)); | ||
} | ||
|
||
@SuppressWarnings({"unchecked", "checkstyle:CyclomaticComplexity"}) | ||
private <T> Iterable<CloseableIterable<T>> entries( | ||
BiFunction<ManifestFile, CloseableIterable<ManifestEntry<DataFile>>, CloseableIterable<T>> entryFn) { | ||
BiFunction<ManifestFile, CloseableIterable<ManifestEntry<DataFile>>, CloseableIterable<T>> entryFn, | ||
DeleteFileIndex deleteFiles) { | ||
LoadingCache<Integer, ManifestEvaluator> evalCache = specsById == null ? | ||
null : Caffeine.newBuilder().build(specId -> { | ||
PartitionSpec spec = specsById.get(specId); | ||
|
@@ -237,6 +253,12 @@ private <T> Iterable<CloseableIterable<T>> entries( | |
manifest -> manifest.hasAddedFiles() || manifest.hasDeletedFiles()); | ||
} | ||
|
||
if (ignoreAdded) { | ||
// only scan manifests that have entries other than added | ||
matchingManifests = Iterables.filter(matchingManifests, | ||
manifest -> manifest.hasExistingFiles() || manifest.hasDeletedFiles()); | ||
} | ||
|
||
matchingManifests = Iterables.filter(matchingManifests, manifestPredicate::test); | ||
|
||
return Iterables.transform( | ||
|
@@ -258,6 +280,14 @@ private <T> Iterable<CloseableIterable<T>> entries( | |
entry -> entry.status() != ManifestEntry.Status.EXISTING); | ||
} | ||
|
||
if (ignoreAdded) { | ||
entries = CloseableIterable.filter(entries, entry -> entry.status() != ManifestEntry.Status.ADDED); | ||
} | ||
|
||
if (onlyWithRowLevelDeletes && deleteFiles != null) { | ||
entries = CloseableIterable.filter(entries, entry -> deleteFiles.forEntry(entry).length > 0); | ||
} | ||
|
||
if (evaluator != null) { | ||
entries = CloseableIterable.filter(entries, | ||
entry -> evaluator.eval((GenericDataFile) entry.file())); | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,33 @@ | ||
/* | ||
* 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.actions; | ||
|
||
public class BaseGenerateChangeSetActionResult<T> implements GenerateChangeSet.Result<T> { | ||
private final T changeSet; | ||
|
||
public BaseGenerateChangeSetActionResult(T changeSet) { | ||
this.changeSet = changeSet; | ||
} | ||
|
||
@Override | ||
public T changeSet() { | ||
return changeSet; | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think "from" here is ambiguous, probably "since" is a bit closer?