Skip to content

Commit

Permalink
[core] Refactor codes for compact with deletion vector (apache#4013)
Browse files Browse the repository at this point in the history
  • Loading branch information
JingsongLi authored Aug 23, 2024
1 parent 65f400e commit 257f21a
Show file tree
Hide file tree
Showing 16 changed files with 127 additions and 272 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -94,11 +94,6 @@ public AppendOnlyFileStoreWrite newWrite(String commitUser) {
@Override
public AppendOnlyFileStoreWrite newWrite(
String commitUser, ManifestCacheFilter manifestFilter) {
DeletionVectorsMaintainer.Factory deletionVectorsMaintainerFactory = null;
if (options.deletionVectorsEnabled()) {
deletionVectorsMaintainerFactory =
new DeletionVectorsMaintainer.Factory(newIndexFileHandler());
}
return new AppendOnlyFileStoreWrite(
fileIO,
newRead(),
Expand All @@ -110,7 +105,9 @@ public AppendOnlyFileStoreWrite newWrite(
newScan(true).withManifestCacheFilter(manifestFilter),
options,
bucketMode(),
deletionVectorsMaintainerFactory,
options.deletionVectorsEnabled()
? DeletionVectorsMaintainer.factory(newIndexFileHandler())
: null,
tableName);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.paimon.append;

import org.apache.paimon.annotation.VisibleForTesting;
import org.apache.paimon.compact.CompactDeletionFile;
import org.apache.paimon.compact.CompactManager;
import org.apache.paimon.data.InternalRow;
import org.apache.paimon.data.serializer.InternalRowSerializer;
Expand All @@ -27,12 +28,10 @@
import org.apache.paimon.fileindex.FileIndexOptions;
import org.apache.paimon.format.FileFormat;
import org.apache.paimon.fs.FileIO;
import org.apache.paimon.index.IndexFileMeta;
import org.apache.paimon.io.CompactIncrement;
import org.apache.paimon.io.DataFileMeta;
import org.apache.paimon.io.DataFilePathFactory;
import org.apache.paimon.io.DataIncrement;
import org.apache.paimon.io.IndexIncrement;
import org.apache.paimon.io.RowDataRollingFileWriter;
import org.apache.paimon.manifest.FileSource;
import org.apache.paimon.memory.MemoryOwner;
Expand Down Expand Up @@ -77,8 +76,7 @@ public class AppendOnlyWriter implements RecordWriter<InternalRow>, MemoryOwner
private final List<DataFileMeta> deletedFiles;
private final List<DataFileMeta> compactBefore;
private final List<DataFileMeta> compactAfter;
private final List<IndexFileMeta> indexFilesBefore;
private final List<IndexFileMeta> indexFilesAfter;
@Nullable private CompactDeletionFile compactDeletionFile;
private final LongCounter seqNumCounter;
private final String fileCompression;
private final String spillCompression;
Expand Down Expand Up @@ -125,8 +123,6 @@ public AppendOnlyWriter(
this.deletedFiles = new ArrayList<>();
this.compactBefore = new ArrayList<>();
this.compactAfter = new ArrayList<>();
this.indexFilesBefore = new ArrayList<>();
this.indexFilesAfter = new ArrayList<>();
this.seqNumCounter = new LongCounter(maxSequenceNumber + 1);
this.fileCompression = fileCompression;
this.spillCompression = spillCompression;
Expand All @@ -145,10 +141,7 @@ public AppendOnlyWriter(
deletedFiles.addAll(increment.newFilesIncrement().deletedFiles());
compactBefore.addAll(increment.compactIncrement().compactBefore());
compactAfter.addAll(increment.compactIncrement().compactAfter());
if (increment.indexIncrement() != null) {
indexFilesBefore.addAll(increment.indexIncrement().deletedIndexFiles());
indexFilesAfter.addAll(increment.indexIncrement().newIndexFiles());
}
updateCompactDeletionFile(increment.compactDeletionFile());
}
}

Expand Down Expand Up @@ -237,6 +230,10 @@ public void close() throws Exception {
}

sinkWriter.close();

if (compactDeletionFile != null) {
compactDeletionFile.clean();
}
}

public void toBufferedWriter() throws Exception {
Expand Down Expand Up @@ -286,14 +283,19 @@ private void trySyncLatestCompaction(boolean blocking)
result -> {
compactBefore.addAll(result.before());
compactAfter.addAll(result.after());
if (result.indexIncrement() != null) {
indexFilesBefore.addAll(
result.indexIncrement().deletedIndexFiles());
indexFilesAfter.addAll(result.indexIncrement().newIndexFiles());
}
updateCompactDeletionFile(result.deletionFile());
});
}

private void updateCompactDeletionFile(@Nullable CompactDeletionFile newDeletionFile) {
if (newDeletionFile != null) {
compactDeletionFile =
compactDeletionFile == null
? newDeletionFile
: newDeletionFile.mergeOldFile(compactDeletionFile);
}
}

private CommitIncrement drainIncrement() {
DataIncrement dataIncrement =
new DataIncrement(
Expand All @@ -305,22 +307,15 @@ private CommitIncrement drainIncrement() {
new ArrayList<>(compactBefore),
new ArrayList<>(compactAfter),
Collections.emptyList());

IndexIncrement indexIncrement = null;
if (!indexFilesBefore.isEmpty() || !indexFilesAfter.isEmpty()) {
indexIncrement =
new IndexIncrement(
new ArrayList<>(indexFilesAfter), new ArrayList<>(indexFilesBefore));
}
CompactDeletionFile drainDeletionFile = compactDeletionFile;

newFiles.clear();
deletedFiles.clear();
compactBefore.clear();
compactAfter.clear();
indexFilesBefore.clear();
indexFilesAfter.clear();
compactDeletionFile = null;

return new CommitIncrement(dataIncrement, compactIncrement, indexIncrement, null);
return new CommitIncrement(dataIncrement, compactIncrement, drainDeletionFile);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,18 +20,14 @@

import org.apache.paimon.AppendOnlyFileStore;
import org.apache.paimon.annotation.VisibleForTesting;
import org.apache.paimon.compact.CompactDeletionFile;
import org.apache.paimon.compact.CompactFutureManager;
import org.apache.paimon.compact.CompactResult;
import org.apache.paimon.compact.CompactTask;
import org.apache.paimon.deletionvectors.append.AppendDeletionFileMaintainer;
import org.apache.paimon.index.IndexFileMeta;
import org.apache.paimon.deletionvectors.DeletionVectorsMaintainer;
import org.apache.paimon.io.DataFileMeta;
import org.apache.paimon.io.IndexIncrement;
import org.apache.paimon.manifest.FileKind;
import org.apache.paimon.manifest.IndexManifestEntry;
import org.apache.paimon.operation.metrics.CompactionMetrics;
import org.apache.paimon.operation.metrics.MetricUtils;
import org.apache.paimon.table.source.DeletionFile;
import org.apache.paimon.utils.Preconditions;

import org.slf4j.Logger;
Expand All @@ -42,7 +38,6 @@
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.LinkedList;
import java.util.List;
Expand All @@ -51,6 +46,8 @@
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;

import static java.util.Collections.emptyList;

/** Compact manager for {@link AppendOnlyFileStore}. */
public class BucketedAppendCompactManager extends CompactFutureManager {

Expand All @@ -59,7 +56,7 @@ public class BucketedAppendCompactManager extends CompactFutureManager {
private static final int FULL_COMPACT_MIN_FILE = 3;

private final ExecutorService executor;
private final AppendDeletionFileMaintainer dvIndexFileMaintainer;
private final DeletionVectorsMaintainer dvMaintainer;
private final TreeSet<DataFileMeta> toCompact;
private final int minFileNum;
private final int maxFileNum;
Expand All @@ -73,14 +70,14 @@ public class BucketedAppendCompactManager extends CompactFutureManager {
public BucketedAppendCompactManager(
ExecutorService executor,
List<DataFileMeta> restored,
@Nullable AppendDeletionFileMaintainer dvIndexFileMaintainer,
@Nullable DeletionVectorsMaintainer dvMaintainer,
int minFileNum,
int maxFileNum,
long targetFileSize,
CompactRewriter rewriter,
@Nullable CompactionMetrics.Reporter metricsReporter) {
this.executor = executor;
this.dvIndexFileMaintainer = dvIndexFileMaintainer;
this.dvMaintainer = dvMaintainer;
this.toCompact = new TreeSet<>(fileComparator(false));
this.toCompact.addAll(restored);
this.minFileNum = minFileNum;
Expand All @@ -106,14 +103,14 @@ private void triggerFullCompaction() {
+ "forces a new compaction. This is unexpected.");
// if deletion vector enables, always trigger compaction.
if (toCompact.isEmpty()
|| (dvIndexFileMaintainer == null && toCompact.size() < FULL_COMPACT_MIN_FILE)) {
|| (dvMaintainer == null && toCompact.size() < FULL_COMPACT_MIN_FILE)) {
return;
}

taskFuture =
executor.submit(
new FullCompactTask(
dvIndexFileMaintainer,
dvMaintainer,
toCompact,
targetFileSize,
rewriter,
Expand All @@ -132,7 +129,7 @@ private void triggerCompactionWithBestEffort() {
taskFuture =
executor.submit(
new AutoCompactTask(
dvIndexFileMaintainer, compacting, rewriter, metricsReporter));
dvMaintainer, compacting, rewriter, metricsReporter));
}
}

Expand Down Expand Up @@ -226,19 +223,19 @@ public void close() throws IOException {
/** A {@link CompactTask} impl for full compaction of append-only table. */
public static class FullCompactTask extends CompactTask {

private final AppendDeletionFileMaintainer dvIndexFileMaintainer;
private final DeletionVectorsMaintainer dvMaintainer;
private final LinkedList<DataFileMeta> toCompact;
private final long targetFileSize;
private final CompactRewriter rewriter;

public FullCompactTask(
AppendDeletionFileMaintainer dvIndexFileMaintainer,
DeletionVectorsMaintainer dvMaintainer,
Collection<DataFileMeta> inputs,
long targetFileSize,
CompactRewriter rewriter,
@Nullable CompactionMetrics.Reporter metricsReporter) {
super(metricsReporter);
this.dvIndexFileMaintainer = dvIndexFileMaintainer;
this.dvMaintainer = dvMaintainer;
this.toCompact = new LinkedList<>(inputs);
this.targetFileSize = targetFileSize;
this.rewriter = rewriter;
Expand All @@ -258,9 +255,9 @@ protected CompactResult doCompact() throws Exception {
}

// do compaction
if (dvIndexFileMaintainer != null) {
if (dvMaintainer != null) {
// if deletion vector enables, always trigger compaction.
return compact(dvIndexFileMaintainer, toCompact, rewriter);
return compact(dvMaintainer, toCompact, rewriter);
} else {
// compute small files
int big = 0;
Expand All @@ -273,16 +270,16 @@ protected CompactResult doCompact() throws Exception {
}
}
if (small > big && toCompact.size() >= FULL_COMPACT_MIN_FILE) {
return compact(dvIndexFileMaintainer, toCompact, rewriter);
return compact(null, toCompact, rewriter);
} else {
return result(Collections.emptyList(), Collections.emptyList());
return result(emptyList(), emptyList());
}
}
}

private boolean hasDeletionFile(DataFileMeta file) {
return dvIndexFileMaintainer != null
&& dvIndexFileMaintainer.getDeletionFile(file.fileName()) == null;
return dvMaintainer != null
&& dvMaintainer.deletionVectorOf(file.fileName()).isPresent();
}
}

Expand All @@ -295,77 +292,45 @@ private boolean hasDeletionFile(DataFileMeta file) {
*/
public static class AutoCompactTask extends CompactTask {

private final AppendDeletionFileMaintainer dvIndexFileMaintainer;
private final DeletionVectorsMaintainer dvMaintainer;
private final List<DataFileMeta> toCompact;
private final CompactRewriter rewriter;

public AutoCompactTask(
AppendDeletionFileMaintainer dvIndexFileMaintainer,
DeletionVectorsMaintainer dvMaintainer,
List<DataFileMeta> toCompact,
CompactRewriter rewriter,
@Nullable CompactionMetrics.Reporter metricsReporter) {
super(metricsReporter);
this.dvIndexFileMaintainer = dvIndexFileMaintainer;
this.dvMaintainer = dvMaintainer;
this.toCompact = toCompact;
this.rewriter = rewriter;
}

@Override
protected CompactResult doCompact() throws Exception {
return compact(dvIndexFileMaintainer, toCompact, rewriter);
return compact(dvMaintainer, toCompact, rewriter);
}
}

private static CompactResult compact(
AppendDeletionFileMaintainer dvIndexFileMaintainer,
@Nullable DeletionVectorsMaintainer dvMaintainer,
List<DataFileMeta> toCompact,
CompactRewriter rewriter)
throws Exception {
if (dvIndexFileMaintainer == null) {
return result(toCompact, rewriter.rewrite(toCompact));
} else {
List<DeletionFile> deletionFiles = new ArrayList<>();
for (DataFileMeta dataFile : toCompact) {
deletionFiles.add(dvIndexFileMaintainer.getDeletionFile(dataFile.fileName()));
}
List<DataFileMeta> compactAfter = rewriter.rewrite(toCompact);
toCompact.forEach(f -> dvIndexFileMaintainer.notifyRemovedDeletionVector(f.fileName()));

List<IndexManifestEntry> indexManifestEntries = dvIndexFileMaintainer.persist();
if (indexManifestEntries.isEmpty()) {
return result(toCompact, compactAfter);
} else {
List<IndexFileMeta> indexFilesBefore = new ArrayList<>();
List<IndexFileMeta> indexFilesAfter = new ArrayList<>();
for (IndexManifestEntry entry : indexManifestEntries) {
if (entry.kind() == FileKind.ADD) {
indexFilesAfter.add(entry.indexFile());
} else {
indexFilesBefore.add(entry.indexFile());
}
}
return result(toCompact, indexFilesBefore, compactAfter, indexFilesAfter);
}
List<DataFileMeta> rewrite = rewriter.rewrite(toCompact);
CompactResult result = result(toCompact, rewrite);
if (dvMaintainer != null) {
toCompact.forEach(f -> dvMaintainer.removeDeletionVectorOf(f.fileName()));
result.setDeletionFile(CompactDeletionFile.generateFiles(dvMaintainer));
}
return result;
}

private static CompactResult result(List<DataFileMeta> before, List<DataFileMeta> after) {
return new CompactResult(before, after);
}

private static CompactResult result(
List<DataFileMeta> before,
@Nullable List<IndexFileMeta> indexFilesBefore,
List<DataFileMeta> after,
@Nullable List<IndexFileMeta> indexFilesAfter) {
CompactResult result = new CompactResult(before, after);
if (indexFilesBefore != null || indexFilesAfter != null) {
IndexIncrement indexIncrement = new IndexIncrement(indexFilesAfter, indexFilesBefore);
result.setIndexIncrement(indexIncrement);
}
return result;
}

/** Compact rewriter for append-only table. */
public interface CompactRewriter {
List<DataFileMeta> rewrite(List<DataFileMeta> compactBefore) throws Exception;
Expand Down
Loading

0 comments on commit 257f21a

Please sign in to comment.