Skip to content

Commit

Permalink
[flink] support computing the changelog generated by compact during r…
Browse files Browse the repository at this point in the history
…ead time.

This is used when changelog producer is none, but CoreOptions#needLookup is true and the table is used as a dim table.
  • Loading branch information
liming30 committed Aug 29, 2024
1 parent ed48b6b commit 9119c6a
Show file tree
Hide file tree
Showing 17 changed files with 419 additions and 14 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -2338,7 +2338,8 @@ public enum StreamScanMode implements DescribedEnum {
COMPACT_BUCKET_TABLE("compact-bucket-table", "Compaction for traditional bucket table."),
COMPACT_APPEND_NO_BUCKET(
"compact-append-no-bucket", "Compaction for append table with bucket unaware."),
FILE_MONITOR("file-monitor", "Monitor data file changes.");
FILE_MONITOR("file-monitor", "Monitor data file changes."),
COMPACT_DELTA_MONITOR("compact-delta-monitor", "Monitor delta changes for compaction.");

private final String value;
private final String description;
Expand Down
32 changes: 28 additions & 4 deletions paimon-core/src/main/java/org/apache/paimon/Snapshot.java
Original file line number Diff line number Diff line change
Expand Up @@ -32,8 +32,10 @@
import javax.annotation.Nullable;

import java.io.IOException;
import java.util.Arrays;
import java.util.Map;
import java.util.Objects;
import java.util.stream.Collectors;

/**
* This file is the entrance to all data committed at some specific time point.
Expand Down Expand Up @@ -411,15 +413,37 @@ public boolean equals(Object o) {
public enum CommitKind {

/** Changes flushed from the mem table. */
APPEND,
APPEND((byte) 1),

/** Changes by compacting existing data files. */
COMPACT,
COMPACT((byte) 2),

/** Changes that clear up the whole partition and then add new records. */
OVERWRITE,
OVERWRITE((byte) 3),

/** Collect statistics. */
ANALYZE
ANALYZE((byte) 4);

private static final Map<Byte, CommitKind> BYTE_TO_COMMIT_KIND =
Arrays.stream(CommitKind.values()).collect(Collectors.toMap(e -> e.code, e -> e));

private final byte code;

CommitKind(byte code) {
this.code = code;
}

public byte getCode() {
return code;
}

public static CommitKind fromByte(byte kind) {
final CommitKind commitKind = BYTE_TO_COMMIT_KIND.get(kind);
if (commitKind == null) {
throw new IllegalArgumentException(
String.format("The given kind %s is not supported.", kind));
}
return commitKind;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -249,6 +249,11 @@ public ScanMode scanMode() {
public List<ManifestEntry> files() {
return files;
}

@Override
public Snapshot.CommitKind commitKind() {
return readSnapshot == null ? null : readSnapshot.commitKind();
}
};
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -127,6 +127,9 @@ default List<ManifestEntry> files(FileKind kind) {
return files().stream().filter(e -> e.kind() == kind).collect(Collectors.toList());
}

/** {@link org.apache.paimon.Snapshot.CommitKind} of the snapshot. */
Snapshot.CommitKind commitKind();

/** Return a map group by partition and bucket. */
static Map<BinaryRow, Map<Integer, List<DataFileMeta>>> groupByPartFiles(
List<ManifestEntry> files) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -118,6 +118,7 @@ protected StartingScanner createStartingScanner(boolean isStreaming) {
return new ContinuousCompactorStartingScanner(snapshotManager);
case COMPACT_APPEND_NO_BUCKET:
case FILE_MONITOR:
case COMPACT_DELTA_MONITOR:
return new FullStartingScanner(snapshotManager);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.paimon.table.source;

import org.apache.paimon.Snapshot;
import org.apache.paimon.data.BinaryRow;
import org.apache.paimon.io.DataFileMeta;
import org.apache.paimon.io.DataFileMeta08Serializer;
Expand Down Expand Up @@ -49,7 +50,7 @@ public class DataSplit implements Split {

private static final long serialVersionUID = 7L;
private static final long MAGIC = -2394839472490812314L;
private static final int VERSION = 2;
private static final int VERSION = 3;

private long snapshotId = 0;
private BinaryRow partition;
Expand All @@ -64,6 +65,7 @@ public class DataSplit implements Split {

private boolean isStreaming = false;
private boolean rawConvertible;
@Nullable private Snapshot.CommitKind commitKind;

public DataSplit() {}

Expand Down Expand Up @@ -108,6 +110,11 @@ public boolean rawConvertible() {
return rawConvertible;
}

@Nullable
public Snapshot.CommitKind commitKind() {
return commitKind;
}

public OptionalLong latestFileCreationEpochMillis() {
return this.dataFiles.stream().mapToLong(DataFileMeta::creationTimeEpochMillis).max();
}
Expand Down Expand Up @@ -193,7 +200,8 @@ public boolean equals(Object o) {
&& Objects.equals(beforeFiles, dataSplit.beforeFiles)
&& Objects.equals(beforeDeletionFiles, dataSplit.beforeDeletionFiles)
&& Objects.equals(dataFiles, dataSplit.dataFiles)
&& Objects.equals(dataDeletionFiles, dataSplit.dataDeletionFiles);
&& Objects.equals(dataDeletionFiles, dataSplit.dataDeletionFiles)
&& Objects.equals(commitKind, dataSplit.commitKind);
}

@Override
Expand All @@ -208,7 +216,8 @@ public int hashCode() {
dataFiles,
dataDeletionFiles,
isStreaming,
rawConvertible);
rawConvertible,
commitKind);
}

private void writeObject(ObjectOutputStream out) throws IOException {
Expand All @@ -230,6 +239,7 @@ private void assign(DataSplit other) {
this.dataDeletionFiles = other.dataDeletionFiles;
this.isStreaming = other.isStreaming;
this.rawConvertible = other.rawConvertible;
this.commitKind = other.commitKind;
}

public void serialize(DataOutputView out) throws IOException {
Expand Down Expand Up @@ -258,6 +268,9 @@ public void serialize(DataOutputView out) throws IOException {
out.writeBoolean(isStreaming);

out.writeBoolean(rawConvertible);

byte code = commitKind == null ? (byte) -1 : commitKind.getCode();
out.writeByte(code);
}

public static DataSplit deserialize(DataInputView in) throws IOException {
Expand Down Expand Up @@ -289,6 +302,7 @@ public static DataSplit deserialize(DataInputView in) throws IOException {

boolean isStreaming = in.readBoolean();
boolean rawConvertible = in.readBoolean();
Snapshot.CommitKind commitKind = deserializeForCommitKind(version, in);

DataSplit.Builder builder =
builder()
Expand All @@ -299,7 +313,8 @@ public static DataSplit deserialize(DataInputView in) throws IOException {
.withBeforeFiles(beforeFiles)
.withDataFiles(dataFiles)
.isStreaming(isStreaming)
.rawConvertible(rawConvertible);
.rawConvertible(rawConvertible)
.withCommitKind(commitKind);

if (beforeDeletionFiles != null) {
builder.withBeforeDeletionFiles(beforeDeletionFiles);
Expand All @@ -310,12 +325,18 @@ public static DataSplit deserialize(DataInputView in) throws IOException {
return builder.build();
}

private static Snapshot.CommitKind deserializeForCommitKind(int version, DataInputView in)
throws IOException {
byte code = version >= 3 ? in.readByte() : -1;
return code == -1 ? null : Snapshot.CommitKind.fromByte(code);
}

private static FunctionWithIOException<DataInputView, DataFileMeta> getFileMetaSerde(
int version) {
if (version == 1) {
DataFileMeta08Serializer serializer = new DataFileMeta08Serializer();
return serializer::deserialize;
} else if (version == 2) {
} else if (version >= 2) {
DataFileMetaSerializer serializer = new DataFileMetaSerializer();
return serializer::deserialize;
} else {
Expand Down Expand Up @@ -387,6 +408,11 @@ public Builder rawConvertible(boolean rawConvertible) {
return this;
}

public Builder withCommitKind(Snapshot.CommitKind commitKind) {
this.split.commitKind = commitKind;
return this;
}

public DataSplit build() {
checkArgument(split.partition != null);
checkArgument(split.bucket != -1);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.paimon.table.source.snapshot.AllDeltaFollowUpScanner;
import org.apache.paimon.table.source.snapshot.BoundedChecker;
import org.apache.paimon.table.source.snapshot.CompactionChangelogFollowUpScanner;
import org.apache.paimon.table.source.snapshot.CompactionFollowUpScanner;
import org.apache.paimon.table.source.snapshot.ContinuousAppendAndCompactFollowUpScanner;
import org.apache.paimon.table.source.snapshot.DeltaFollowUpScanner;
import org.apache.paimon.table.source.snapshot.FollowUpScanner;
Expand Down Expand Up @@ -208,6 +209,8 @@ private FollowUpScanner createFollowUpScanner() {
return new ContinuousAppendAndCompactFollowUpScanner();
case FILE_MONITOR:
return new AllDeltaFollowUpScanner();
case COMPACT_DELTA_MONITOR:
return new CompactionFollowUpScanner();
}

CoreOptions.ChangelogProducer changelogProducer = options.changelogProducer();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import org.apache.paimon.reader.RecordReader;
import org.apache.paimon.schema.TableSchema;
import org.apache.paimon.table.source.splitread.IncrementalChangelogReadProvider;
import org.apache.paimon.table.source.splitread.IncrementalCompactDiffReadProvider;
import org.apache.paimon.table.source.splitread.IncrementalDiffReadProvider;
import org.apache.paimon.table.source.splitread.MergeFileSplitReadProvider;
import org.apache.paimon.table.source.splitread.RawFileSplitReadProvider;
Expand Down Expand Up @@ -62,6 +63,8 @@ public KeyValueTableRead(
Arrays.asList(
new RawFileSplitReadProvider(batchRawReadSupplier, this::assignValues),
new MergeFileSplitReadProvider(mergeReadSupplier, this::assignValues),
new IncrementalCompactDiffReadProvider(
mergeReadSupplier, this::assignValues),
new IncrementalChangelogReadProvider(mergeReadSupplier, this::assignValues),
new IncrementalDiffReadProvider(mergeReadSupplier, this::assignValues));
}
Expand Down
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.paimon.table.source.snapshot;

import org.apache.paimon.Snapshot;
import org.apache.paimon.table.source.ScanMode;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/** {@link FollowUpScanner} for read all changed files after compact. */
public class CompactionFollowUpScanner implements FollowUpScanner {

private static final Logger LOG = LoggerFactory.getLogger(CompactionFollowUpScanner.class);

@Override
public boolean shouldScanSnapshot(Snapshot snapshot) {
if (snapshot.commitKind() == Snapshot.CommitKind.COMPACT) {
return true;
}

LOG.debug(
"Next snapshot id {} is not COMPACT, but is {}, check next one.",
snapshot.id(),
snapshot.commitKind());
return false;
}

@Override
public SnapshotReader.Plan scan(Snapshot snapshot, SnapshotReader snapshotReader) {
return snapshotReader.withMode(ScanMode.DELTA).withSnapshot(snapshot).readChanges();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -397,7 +397,8 @@ private Plan toChangesPlan(
.withBeforeFiles(before)
.withDataFiles(data)
.isStreaming(isStreaming)
.withBucketPath(pathFactory.bucketPath(part, bucket).toString());
.withBucketPath(pathFactory.bucketPath(part, bucket).toString())
.withCommitKind(plan.commitKind());
if (deletionVectors) {
builder.withBeforeDeletionFiles(
getDeletionFiles(
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
/*
* 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.paimon.table.source.splitread;

import org.apache.paimon.Snapshot;
import org.apache.paimon.data.InternalRow;
import org.apache.paimon.operation.MergeFileSplitRead;
import org.apache.paimon.operation.SplitRead;
import org.apache.paimon.table.source.DataSplit;
import org.apache.paimon.utils.LazyField;

import java.util.function.Consumer;
import java.util.function.Supplier;

/** A {@link SplitReadProvider} to streaming incremental diff read after compaction. */
public class IncrementalCompactDiffReadProvider implements SplitReadProvider {

private final LazyField<SplitRead<InternalRow>> splitRead;

public IncrementalCompactDiffReadProvider(
Supplier<MergeFileSplitRead> supplier,
Consumer<SplitRead<InternalRow>> valuesAssigner) {
this.splitRead =
new LazyField<>(
() -> {
SplitRead<InternalRow> read = create(supplier);
valuesAssigner.accept(read);
return read;
});
}

private SplitRead<InternalRow> create(Supplier<MergeFileSplitRead> supplier) {
return new IncrementalCompactDiffSplitRead(supplier.get());
}

@Override
public boolean match(DataSplit split, boolean forceKeepDelete) {
return split.commitKind() == Snapshot.CommitKind.COMPACT
&& !split.beforeFiles().isEmpty()
&& split.isStreaming();
}

@Override
public boolean initialized() {
return splitRead.initialized();
}

@Override
public SplitRead<InternalRow> getOrCreate() {
return splitRead.get();
}
}
Loading

0 comments on commit 9119c6a

Please sign in to comment.