-
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
Changes from 7 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,49 @@ | ||
/* | ||
* 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 Cdc extends Action<Cdc, Cdc.Result> { | ||
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. Is this an action? Action typically modify the table like rewrite, expire snapshots. 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 is. Action usually modifies table, but is not necessarily limited by that. This is the first PR, we will also explore a way to use scan for CDC. 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. All the existing actions in Iceberg seem to modify the table. Since this is a public API, I like to double check. Can we keep it in SparkActions only for now if we are in experimental phase? Conceptually, this is a scan/read (not a maintenance action). 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. Agreed. This is the start point. As I said in another comment, we need a well designed scan interface first. 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 wouldn't mind having an action like this. We have I would match the naming style we have in other actions, though. I think it can be 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.
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. The primary goal of adding the That being said, I'd be happy to discuss alternative ways to expose this. |
||
/** | ||
* Emit changed data set by a snapshot id. | ||
* | ||
* @param snapshotId id of the snapshot to generate changed data | ||
* @return this for method chaining | ||
*/ | ||
Cdc useSnapshot(long snapshotId); | ||
|
||
/** | ||
* Emit changed data set by a range of snapshots | ||
* | ||
* @param fromSnapshotId id of the first snapshot | ||
* @param toSnapshotId id of the last snapshot | ||
* @return this for method chaining | ||
*/ | ||
Cdc between(long fromSnapshotId, long toSnapshotId); | ||
|
||
/** | ||
* The action result that contains a dataset of changed rows. | ||
*/ | ||
interface Result { | ||
/** | ||
* Returns CDC records. | ||
*/ | ||
Object cdcRecords(); | ||
} | ||
} |
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 onlyWithDeletes; | ||
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.onlyWithDeletes = 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 onlyWithDeletes() { | ||
flyrain marked this conversation as resolved.
Show resolved
Hide resolved
|
||
this.onlyWithDeletes = 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 (onlyWithDeletes && 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 |
---|---|---|
|
@@ -22,6 +22,7 @@ | |
import java.io.IOException; | ||
import java.io.UncheckedIOException; | ||
import java.util.List; | ||
import java.util.function.Consumer; | ||
import java.util.function.Function; | ||
import org.apache.iceberg.Accessor; | ||
import org.apache.iceberg.MetadataColumns; | ||
|
@@ -73,6 +74,16 @@ public static <T> CloseableIterable<T> filter(CloseableIterable<T> rows, Functio | |
return filter.filter(rows); | ||
} | ||
|
||
public static <T> CloseableIterable<T> marker(CloseableIterable<T> rows, Function<T, Long> rowToPosition, | ||
PositionDeleteIndex deleteSet, Consumer<T> markDeleted) { | ||
if (deleteSet.isEmpty()) { | ||
return rows; | ||
} | ||
|
||
PositionSetDeleteMarker<T> deleteMarker = new PositionSetDeleteMarker<>(rowToPosition, deleteSet, markDeleted); | ||
return deleteMarker.filter(rows); | ||
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. since 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. Maybe just use this method from
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 think it traverses the rows and adds 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. Or maybe we can add a new interface called |
||
} | ||
|
||
public static StructLikeSet toEqualitySet(CloseableIterable<StructLike> eqDeletes, Types.StructType eqType) { | ||
try (CloseableIterable<StructLike> deletes = eqDeletes) { | ||
StructLikeSet deleteSet = StructLikeSet.create(eqType); | ||
|
@@ -107,6 +118,14 @@ public static <T> CloseableIterable<T> streamingFilter(CloseableIterable<T> rows | |
return new PositionStreamDeleteFilter<>(rows, rowToPosition, posDeletes); | ||
} | ||
|
||
|
||
public static <T> CloseableIterable<T> streamingMarker(CloseableIterable<T> rows, | ||
Function<T, Long> rowToPosition, | ||
CloseableIterable<Long> posDeletes, | ||
Consumer<T> markDeleted) { | ||
return new PositionStreamDeleteMarker<>(rows, rowToPosition, posDeletes, markDeleted); | ||
} | ||
|
||
public static CloseableIterable<Long> deletePositions(CharSequence dataLocation, | ||
CloseableIterable<StructLike> deleteFile) { | ||
return deletePositions(dataLocation, ImmutableList.of(deleteFile)); | ||
|
@@ -152,6 +171,29 @@ protected boolean shouldKeep(T row) { | |
} | ||
} | ||
|
||
private static class PositionSetDeleteMarker<T> extends Filter<T> { | ||
private final Function<T, Long> rowToPosition; | ||
private final PositionDeleteIndex deleteSet; | ||
private final Consumer<T> markDeleted; | ||
|
||
private PositionSetDeleteMarker(Function<T, Long> rowToPosition, PositionDeleteIndex deleteSet, | ||
Consumer<T> markDeleted) { | ||
this.rowToPosition = rowToPosition; | ||
this.deleteSet = deleteSet; | ||
this.markDeleted = markDeleted; | ||
} | ||
|
||
@Override | ||
protected boolean shouldKeep(T row) { | ||
if (deleteSet.isDeleted(rowToPosition.apply(row))) { | ||
markDeleted.accept(row); | ||
} | ||
|
||
// always return true, since we don't want to remove the row | ||
return true; | ||
} | ||
} | ||
|
||
private static class PositionStreamDeleteFilter<T> extends CloseableGroup implements CloseableIterable<T> { | ||
private final CloseableIterable<T> rows; | ||
private final Function<T, Long> extractPos; | ||
|
@@ -170,7 +212,7 @@ public CloseableIterator<T> iterator() { | |
|
||
CloseableIterator<T> iter; | ||
if (deletePosIterator.hasNext()) { | ||
iter = new PositionFilterIterator(rows.iterator(), deletePosIterator); | ||
iter = createPosDeleteIterator(rows.iterator(), deletePosIterator); | ||
} else { | ||
iter = rows.iterator(); | ||
try { | ||
|
@@ -185,7 +227,12 @@ public CloseableIterator<T> iterator() { | |
return iter; | ||
} | ||
|
||
private class PositionFilterIterator extends FilterIterator<T> { | ||
protected PositionFilterIterator createPosDeleteIterator(CloseableIterator<T> items, | ||
CloseableIterator<Long> deletePosIterator) { | ||
return new PositionFilterIterator(items, deletePosIterator); | ||
} | ||
|
||
protected class PositionFilterIterator extends FilterIterator<T> { | ||
private final CloseableIterator<Long> deletePosIterator; | ||
private long nextDeletePos; | ||
|
||
|
@@ -227,6 +274,37 @@ public void close() { | |
} | ||
} | ||
|
||
private static class PositionStreamDeleteMarker<T> extends PositionStreamDeleteFilter<T> { | ||
private final Consumer<T> markDeleted; | ||
|
||
private PositionStreamDeleteMarker(CloseableIterable<T> rows, Function<T, Long> extractPos, | ||
CloseableIterable<Long> deletePositions, Consumer<T> markDeleted) { | ||
super(rows, extractPos, deletePositions); | ||
this.markDeleted = markDeleted; | ||
} | ||
|
||
@Override | ||
protected PositionFilterIterator createPosDeleteIterator(CloseableIterator<T> items, | ||
CloseableIterator<Long> deletePosIterator) { | ||
return new PositionDeleteMarkerIterator(items, deletePosIterator); | ||
} | ||
|
||
private class PositionDeleteMarkerIterator extends PositionFilterIterator { | ||
private PositionDeleteMarkerIterator(CloseableIterator<T> items, CloseableIterator<Long> deletePositions) { | ||
super(items, deletePositions); | ||
} | ||
|
||
@Override | ||
protected boolean shouldKeep(T row) { | ||
boolean isDeleted = !super.shouldKeep(row); | ||
if (isDeleted) { | ||
markDeleted.accept(row); | ||
} | ||
return true; | ||
} | ||
} | ||
} | ||
|
||
private static class DataFileFilter<T extends StructLike> extends Filter<T> { | ||
private final CharSequence dataLocation; | ||
|
||
|
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.
Nit: Consider using
CDC
as all caps instead, like it is in the javadoc comments. For me, it looks a lot cleaner.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'm also open to other names, e.g.,
ChangeDataSet
,ChangeDataCapture
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.
In Flink it's referred to as
Changelog
.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 feel we should not use acronyms.
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.
Yeah I agree that using a full word would be better.
In comments and even method names it's fine in my opinion but as the main class name it probably would be best to use the full name.
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.
+1 for
Changelog
. Here, it meansgenerateChangelog
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 am OK with
generateChangelog
.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.
or
generateChangeSet
, since action is a batch execution. if it is a long-running streaming execution,changelog
would be more accurate as it implies a stream.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 liked
generateChangelog
but if it can confuse people,generateChangeSet
sounds good too.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.
Combining feedbacks, changed it to
GetChangeSet
. The nameGenerateChangeSet
is good, but it is way too long. Think about the class nameBaseGenerateChangeSetSparkActionResult
. I admit the verbget
is plain comparing togenerate
. But I think it is fine, a plain name is suitable for a tool.