You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iceberg.apache.org by bl...@apache.org on 2019/07/18 15:57:00 UTC
[incubator-iceberg] branch master updated: Fix RewriteManifests
metadata after retry (#296)
This is an automated email from the ASF dual-hosted git repository.
blue pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-iceberg.git
The following commit(s) were added to refs/heads/master by this push:
new cfd2737 Fix RewriteManifests metadata after retry (#296)
cfd2737 is described below
commit cfd2737d30f31df4d4b664a54efa5b7ee0b93aca
Author: bryanck <br...@gmail.com>
AuthorDate: Thu Jul 18 08:56:56 2019 -0700
Fix RewriteManifests metadata after retry (#296)
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.
---
core/src/main/java/org/apache/iceberg/ReplaceManifests.java | 5 ++---
1 file changed, 2 insertions(+), 3 deletions(-)
diff --git a/core/src/main/java/org/apache/iceberg/ReplaceManifests.java b/core/src/main/java/org/apache/iceberg/ReplaceManifests.java
index 771bd0d..a5ad5b4 100644
--- a/core/src/main/java/org/apache/iceberg/ReplaceManifests.java
+++ b/core/src/main/java/org/apache/iceberg/ReplaceManifests.java
@@ -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<>();
@@ -155,7 +155,6 @@ public class ReplaceManifests extends SnapshotProducer<RewriteManifests> impleme
private void reset() {
cleanAll();
entryCount.set(0);
- manifestCount.set(0);
keptManifests.clear();
replacedManifests.clear();
newManifests.clear();
@@ -240,7 +239,7 @@ public class ReplaceManifests extends SnapshotProducer<RewriteManifests> impleme
}
private ManifestWriter newWriter() {
- return new ManifestWriter(spec, manifestPath(manifestCount.getAndIncrement()), snapshotId());
+ return new ManifestWriter(spec, manifestPath(manifestSuffix.getAndIncrement()), snapshotId());
}
synchronized void close() {