Skip to content

Commit

Permalink
Core: Avoid generating huge manifests during commits (#6335)
Browse files Browse the repository at this point in the history
Co-authored-by: xianyangliu <[email protected]>
  • Loading branch information
ConeyLiu and ConeyLiu authored Jul 28, 2023
1 parent e209f18 commit a7a09d4
Show file tree
Hide file tree
Showing 5 changed files with 419 additions and 38 deletions.
47 changes: 28 additions & 19 deletions core/src/main/java/org/apache/iceberg/FastAppend.java
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@ class FastAppend extends SnapshotProducer<AppendFiles> implements AppendFiles {
private final List<DataFile> newFiles = Lists.newArrayList();
private final List<ManifestFile> appendManifests = Lists.newArrayList();
private final List<ManifestFile> rewrittenAppendManifests = Lists.newArrayList();
private ManifestFile newManifest = null;
private List<ManifestFile> newManifests = null;
private boolean hasNewFiles = false;

FastAppend(String tableName, TableOperations ops) {
Expand Down Expand Up @@ -143,12 +143,12 @@ private ManifestFile copyManifest(ManifestFile manifest) {

@Override
public List<ManifestFile> apply(TableMetadata base, Snapshot snapshot) {
List<ManifestFile> newManifests = Lists.newArrayList();
List<ManifestFile> manifests = Lists.newArrayList();

try {
ManifestFile manifest = writeManifest();
if (manifest != null) {
newManifests.add(manifest);
List<ManifestFile> newWrittenManifests = writeNewManifests();
if (newWrittenManifests != null) {
manifests.addAll(newWrittenManifests);
}
} catch (IOException e) {
throw new RuntimeIOException(e, "Failed to write manifest");
Expand All @@ -158,13 +158,13 @@ public List<ManifestFile> apply(TableMetadata base, Snapshot snapshot) {
Iterables.transform(
Iterables.concat(appendManifests, rewrittenAppendManifests),
manifest -> GenericManifestFile.copyOf(manifest).withSnapshotId(snapshotId()).build());
Iterables.addAll(newManifests, appendManifestsWithMetadata);
Iterables.addAll(manifests, appendManifestsWithMetadata);

if (snapshot != null) {
newManifests.addAll(snapshot.allManifests(ops.io()));
manifests.addAll(snapshot.allManifests(ops.io()));
}

return newManifests;
return manifests;
}

@Override
Expand All @@ -178,8 +178,17 @@ public Object updateEvent() {

@Override
protected void cleanUncommitted(Set<ManifestFile> committed) {
if (newManifest != null && !committed.contains(newManifest)) {
deleteFile(newManifest.path());
if (newManifests != null) {
List<ManifestFile> committedNewManifests = Lists.newArrayList();
for (ManifestFile manifest : newManifests) {
if (committed.contains(manifest)) {
committedNewManifests.add(manifest);
} else {
deleteFile(manifest.path());
}
}

this.newManifests = committedNewManifests;
}

// clean up only rewrittenAppendManifests as they are always owned by the table
Expand All @@ -191,24 +200,24 @@ protected void cleanUncommitted(Set<ManifestFile> committed) {
}
}

private ManifestFile writeManifest() throws IOException {
if (hasNewFiles && newManifest != null) {
deleteFile(newManifest.path());
newManifest = null;
private List<ManifestFile> writeNewManifests() throws IOException {
if (hasNewFiles && newManifests != null) {
newManifests.forEach(file -> deleteFile(file.path()));
newManifests = null;
}

if (newManifest == null && newFiles.size() > 0) {
ManifestWriter<DataFile> writer = newManifestWriter(spec);
if (newManifests == null && newFiles.size() > 0) {
RollingManifestWriter<DataFile> writer = newRollingManifestWriter(spec);
try {
writer.addAll(newFiles);
newFiles.forEach(writer::add);
} finally {
writer.close();
}

this.newManifest = writer.toManifestFile();
this.newManifests = writer.toManifestFiles();
hasNewFiles = false;
}

return newManifest;
return newManifests;
}
}
44 changes: 25 additions & 19 deletions core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,7 @@ abstract class MergingSnapshotProducer<ThisT> extends SnapshotProducer<ThisT> {
private PartitionSpec dataSpec;

// cache new data manifests after writing
private ManifestFile cachedNewDataManifest = null;
private List<ManifestFile> cachedNewDataManifests = null;
private boolean hasNewDataFiles = false;

// cache new manifests for delete files
Expand Down Expand Up @@ -907,9 +907,17 @@ public Object updateEvent() {
}

private void cleanUncommittedAppends(Set<ManifestFile> committed) {
if (cachedNewDataManifest != null && !committed.contains(cachedNewDataManifest)) {
deleteFile(cachedNewDataManifest.path());
this.cachedNewDataManifest = null;
if (cachedNewDataManifests != null) {
List<ManifestFile> committedNewDataManifests = Lists.newArrayList();
for (ManifestFile manifest : cachedNewDataManifests) {
if (committed.contains(manifest)) {
committedNewDataManifests.add(manifest);
} else {
deleteFile(manifest.path());
}
}

this.cachedNewDataManifests = committedNewDataManifests;
}

ListIterator<ManifestFile> deleteManifestsIterator = cachedNewDeleteManifests.listIterator();
Expand Down Expand Up @@ -952,10 +960,8 @@ protected void cleanUncommitted(Set<ManifestFile> committed) {
private Iterable<ManifestFile> prepareNewDataManifests() {
Iterable<ManifestFile> newManifests;
if (newDataFiles.size() > 0) {
ManifestFile newManifest = newDataFilesAsManifest();
newManifests =
Iterables.concat(
ImmutableList.of(newManifest), appendManifests, rewrittenAppendManifests);
List<ManifestFile> dataFileManifests = newDataFilesAsManifests();
newManifests = Iterables.concat(dataFileManifests, appendManifests, rewrittenAppendManifests);
} else {
newManifests = Iterables.concat(appendManifests, rewrittenAppendManifests);
}
Expand All @@ -965,33 +971,33 @@ private Iterable<ManifestFile> prepareNewDataManifests() {
manifest -> GenericManifestFile.copyOf(manifest).withSnapshotId(snapshotId()).build());
}

private ManifestFile newDataFilesAsManifest() {
if (hasNewDataFiles && cachedNewDataManifest != null) {
deleteFile(cachedNewDataManifest.path());
cachedNewDataManifest = null;
private List<ManifestFile> newDataFilesAsManifests() {
if (hasNewDataFiles && cachedNewDataManifests != null) {
cachedNewDataManifests.forEach(file -> deleteFile(file.path()));
cachedNewDataManifests = null;
}

if (cachedNewDataManifest == null) {
if (cachedNewDataManifests == null) {
try {
ManifestWriter<DataFile> writer = newManifestWriter(dataSpec());
RollingManifestWriter<DataFile> writer = newRollingManifestWriter(dataSpec());
try {
if (newDataFilesDataSequenceNumber == null) {
writer.addAll(newDataFiles);
newDataFiles.forEach(writer::add);
} else {
newDataFiles.forEach(f -> writer.add(f, newDataFilesDataSequenceNumber));
}
} finally {
writer.close();
}

this.cachedNewDataManifest = writer.toManifestFile();
this.cachedNewDataManifests = writer.toManifestFiles();
this.hasNewDataFiles = false;
} catch (IOException e) {
throw new RuntimeIOException(e, "Failed to close manifest writer");
}
}

return cachedNewDataManifest;
return cachedNewDataManifests;
}

private Iterable<ManifestFile> prepareDeleteManifests() {
Expand All @@ -1017,7 +1023,7 @@ private List<ManifestFile> newDeleteFilesAsManifests() {
(specId, deleteFiles) -> {
PartitionSpec spec = ops.current().spec(specId);
try {
ManifestWriter<DeleteFile> writer = newDeleteManifestWriter(spec);
RollingManifestWriter<DeleteFile> writer = newRollingDeleteManifestWriter(spec);
try {
deleteFiles.forEach(
df -> {
Expand All @@ -1030,7 +1036,7 @@ private List<ManifestFile> newDeleteFilesAsManifests() {
} finally {
writer.close();
}
cachedNewDeleteManifests.add(writer.toManifestFile());
cachedNewDeleteManifests.addAll(writer.toManifestFiles());
} catch (IOException e) {
throw new RuntimeIOException(e, "Failed to close manifest writer");
}
Expand Down
150 changes: 150 additions & 0 deletions core/src/main/java/org/apache/iceberg/RollingManifestWriter.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,150 @@
/*
* 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 java.io.Closeable;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.List;
import java.util.function.Supplier;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;

/** As opposed to {@link ManifestWriter}, a rolling writer could produce multiple manifest files. */
public class RollingManifestWriter<F extends ContentFile<F>> implements Closeable {
private static final int ROWS_DIVISOR = 250;

private final Supplier<ManifestWriter<F>> manifestWriterSupplier;
private final long targetFileSizeInBytes;
private final List<ManifestFile> manifestFiles;

private long currentFileRows = 0;
private ManifestWriter<F> currentWriter = null;

private boolean closed = false;

public RollingManifestWriter(
Supplier<ManifestWriter<F>> manifestWriterSupplier, long targetFileSizeInBytes) {
this.manifestWriterSupplier = manifestWriterSupplier;
this.targetFileSizeInBytes = targetFileSizeInBytes;
this.manifestFiles = Lists.newArrayList();
}

/**
* Add an added entry for a file.
*
* <p>The entry's snapshot ID will be this manifest's snapshot ID. The data and file sequence
* numbers will be assigned at commit.
*
* @param addedFile a data file
*/
public void add(F addedFile) {
currentWriter().add(addedFile);
currentFileRows++;
}

/**
* Add an added entry for a file with a specific sequence number.
*
* <p>The entry's snapshot ID will be this manifest's snapshot ID. The entry's data sequence
* number will be the provided data sequence number. The entry's file sequence number will be
* assigned at commit.
*
* @param addedFile a data file
* @param dataSequenceNumber a data sequence number for the file
*/
public void add(F addedFile, long dataSequenceNumber) {
currentWriter().add(addedFile, dataSequenceNumber);
currentFileRows++;
}

/**
* Add an existing entry for a file.
*
* <p>The original data and file sequence numbers, snapshot ID, which were assigned at commit,
* must be preserved when adding an existing entry.
*
* @param existingFile a file
* @param fileSnapshotId snapshot ID when the data file was added to the table
* @param dataSequenceNumber a data sequence number of the file (assigned when the file was added)
* @param fileSequenceNumber a file sequence number (assigned when the file was added)
*/
public void existing(
F existingFile, long fileSnapshotId, long dataSequenceNumber, Long fileSequenceNumber) {
currentWriter().existing(existingFile, fileSnapshotId, dataSequenceNumber, fileSequenceNumber);
currentFileRows++;
}

/**
* Add a delete entry for a file.
*
* <p>The entry's snapshot ID will be this manifest's snapshot ID. However, the original data and
* file sequence numbers of the file must be preserved when the file is marked as deleted.
*
* @param deletedFile a file
* @param dataSequenceNumber a data sequence number of the file (assigned when the file was added)
* @param fileSequenceNumber a file sequence number (assigned when the file was added)
*/
public void delete(F deletedFile, long dataSequenceNumber, Long fileSequenceNumber) {
currentWriter().delete(deletedFile, dataSequenceNumber, fileSequenceNumber);
currentFileRows++;
}

private ManifestWriter<F> currentWriter() {
if (currentWriter == null) {
this.currentWriter = manifestWriterSupplier.get();
} else if (shouldRollToNewFile()) {
closeCurrentWriter();
this.currentWriter = manifestWriterSupplier.get();
}

return currentWriter;
}

private boolean shouldRollToNewFile() {
return currentFileRows % ROWS_DIVISOR == 0 && currentWriter.length() >= targetFileSizeInBytes;
}

private void closeCurrentWriter() {
if (currentWriter != null) {
try {
currentWriter.close();
ManifestFile currentFile = currentWriter.toManifestFile();
manifestFiles.add(currentFile);
this.currentWriter = null;
this.currentFileRows = 0;
} catch (IOException e) {
throw new UncheckedIOException("Failed to close current writer", e);
}
}
}

@Override
public void close() throws IOException {
if (!closed) {
closeCurrentWriter();
this.closed = true;
}
}

public List<ManifestFile> toManifestFiles() {
Preconditions.checkState(closed, "Cannot get ManifestFile list from unclosed writer");
return manifestFiles;
}
}
15 changes: 15 additions & 0 deletions core/src/main/java/org/apache/iceberg/SnapshotProducer.java
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,8 @@
import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES_DEFAULT;
import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS;
import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT;
import static org.apache.iceberg.TableProperties.MANIFEST_TARGET_SIZE_BYTES;
import static org.apache.iceberg.TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT;

import com.github.benmanes.caffeine.cache.Caffeine;
import com.github.benmanes.caffeine.cache.LoadingCache;
Expand Down Expand Up @@ -85,6 +87,7 @@ public void accept(String file) {
private final AtomicInteger manifestCount = new AtomicInteger(0);
private final AtomicInteger attempt = new AtomicInteger(0);
private final List<String> manifestLists = Lists.newArrayList();
private final long targetManifestSizeBytes;
private MetricsReporter reporter = LoggingMetricsReporter.instance();
private volatile Long snapshotId = null;
private TableMetadata base;
Expand All @@ -107,6 +110,9 @@ protected SnapshotProducer(TableOperations ops) {
}
return addMetadata(ops, file);
});
this.targetManifestSizeBytes =
ops.current()
.propertyAsLong(MANIFEST_TARGET_SIZE_BYTES, MANIFEST_TARGET_SIZE_BYTES_DEFAULT);
}

protected abstract ThisT self();
Expand Down Expand Up @@ -494,6 +500,15 @@ protected ManifestWriter<DeleteFile> newDeleteManifestWriter(PartitionSpec spec)
ops.current().formatVersion(), spec, newManifestOutput(), snapshotId());
}

protected RollingManifestWriter<DataFile> newRollingManifestWriter(PartitionSpec spec) {
return new RollingManifestWriter<>(() -> newManifestWriter(spec), targetManifestSizeBytes);
}

protected RollingManifestWriter<DeleteFile> newRollingDeleteManifestWriter(PartitionSpec spec) {
return new RollingManifestWriter<>(
() -> newDeleteManifestWriter(spec), targetManifestSizeBytes);
}

protected ManifestReader<DataFile> newManifestReader(ManifestFile manifest) {
return ManifestFiles.read(manifest, ops.io(), ops.current().specsById());
}
Expand Down
Loading

0 comments on commit a7a09d4

Please sign in to comment.