Skip to content

Commit

Permalink
Fix RewriteManifests metadata after retry (apache#296)
Browse files Browse the repository at this point in the history
SnapshotProducer caches ManifestFile metadata to avoid reading a manifest more than once when passed a ManifestFile without stats. ReplaceManifests reused manifest file names, which caused the new manifests to be stored with incorrect metadata.
  • Loading branch information
bryanck authored and rdblue committed Jul 18, 2019
1 parent 40bb3c0 commit cfd2737
Showing 1 changed file with 2 additions and 3 deletions.
5 changes: 2 additions & 3 deletions core/src/main/java/org/apache/iceberg/ReplaceManifests.java
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ public class ReplaceManifests extends SnapshotProducer<RewriteManifests> impleme
private final Set<ManifestFile> replacedManifests = Collections.synchronizedSet(new HashSet<>());
private final Map<Object, WriterWrapper> writers = Collections.synchronizedMap(new HashMap<>());

private final AtomicInteger manifestCount = new AtomicInteger(0);
private final AtomicInteger manifestSuffix = new AtomicInteger(0);
private final AtomicLong entryCount = new AtomicLong(0);

private final Map<String, String> summaryProps = new HashMap<>();
Expand Down Expand Up @@ -155,7 +155,6 @@ private void addExistingFromNewCommit(List<ManifestFile> currentManifests) {
private void reset() {
cleanAll();
entryCount.set(0);
manifestCount.set(0);
keptManifests.clear();
replacedManifests.clear();
newManifests.clear();
Expand Down Expand Up @@ -240,7 +239,7 @@ synchronized void addEntry(ManifestEntry entry) {
}

private ManifestWriter newWriter() {
return new ManifestWriter(spec, manifestPath(manifestCount.getAndIncrement()), snapshotId());
return new ManifestWriter(spec, manifestPath(manifestSuffix.getAndIncrement()), snapshotId());
}

synchronized void close() {
Expand Down

0 comments on commit cfd2737

Please sign in to comment.