You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iceberg.apache.org by dw...@apache.org on 2020/05/21 17:31:15 UTC

[incubator-iceberg] branch master updated: Add assertions for sequence numbers to TestFastAppend (#1038)

This is an automated email from the ASF dual-hosted git repository.

dweeks 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 8db4a61  Add assertions for sequence numbers to TestFastAppend (#1038)
8db4a61 is described below

commit 8db4a61a091b9e713153ba69ce1a85015b21538f
Author: Ryan Blue <bl...@apache.org>
AuthorDate: Thu May 21 10:31:07 2020 -0700

    Add assertions for sequence numbers to TestFastAppend (#1038)
    
    * Add assertions for sequence numbers to TestFastAppend.
    
    * Fix checkstyle.
---
 .../java/org/apache/iceberg/TableTestBase.java     | 69 +++++++++++++++++++++-
 .../java/org/apache/iceberg/TestFastAppend.java    | 54 ++++++++++++-----
 2 files changed, 106 insertions(+), 17 deletions(-)

diff --git a/core/src/test/java/org/apache/iceberg/TableTestBase.java b/core/src/test/java/org/apache/iceberg/TableTestBase.java
index 7e4b895..5796f2f 100644
--- a/core/src/test/java/org/apache/iceberg/TableTestBase.java
+++ b/core/src/test/java/org/apache/iceberg/TableTestBase.java
@@ -29,6 +29,7 @@ import java.io.IOException;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
+import java.util.stream.LongStream;
 import org.apache.iceberg.io.FileIO;
 import org.apache.iceberg.io.OutputFile;
 import org.apache.iceberg.types.Types;
@@ -87,9 +88,15 @@ public class TableTestBase {
   public TestTables.TestTable table = null;
 
   protected final int formatVersion;
+  @SuppressWarnings("checkstyle:MemberName")
+  protected final Assertions V1Assert;
+  @SuppressWarnings("checkstyle:MemberName")
+  protected final Assertions V2Assert;
 
   public TableTestBase(int formatVersion) {
     this.formatVersion = formatVersion;
+    this.V1Assert = new Assertions(1, formatVersion);
+    this.V2Assert = new Assertions(2, formatVersion);
   }
 
   @Before
@@ -209,6 +216,14 @@ public class TableTestBase {
   }
 
   void validateSnapshot(Snapshot old, Snapshot snap, DataFile... newFiles) {
+    validateSnapshot(old, snap, null, newFiles);
+  }
+
+  void validateSnapshot(Snapshot old, Snapshot snap, long sequenceNumber, DataFile... newFiles) {
+    validateSnapshot(old, snap, (Long) sequenceNumber, newFiles);
+  }
+
+  void validateSnapshot(Snapshot old, Snapshot snap, Long sequenceNumber, DataFile... newFiles) {
     List<ManifestFile> oldManifests = old != null ? old.manifests() : ImmutableList.of();
 
     // copy the manifests to a modifiable list and remove the existing manifests
@@ -227,6 +242,10 @@ public class TableTestBase {
 
     for (ManifestEntry entry : ManifestFiles.read(manifest, FILE_IO).entries()) {
       DataFile file = entry.file();
+      if (sequenceNumber != null) {
+        V1Assert.assertEquals("Sequence number should default to 0", 0, entry.sequenceNumber().longValue());
+        V2Assert.assertEquals("Sequence number should match expected", sequenceNumber, entry.sequenceNumber());
+      }
       Assert.assertEquals("Path should match expected", newPaths.next(), file.path().toString());
       Assert.assertEquals("File's snapshot ID should match", id, (long) entry.snapshotId());
     }
@@ -254,12 +273,23 @@ public class TableTestBase {
     return paths;
   }
 
-  static void validateManifest(ManifestFile manifest,
-                               Iterator<Long> ids,
-                               Iterator<DataFile> expectedFiles) {
+  void validateManifest(ManifestFile manifest,
+                        Iterator<Long> ids,
+                        Iterator<DataFile> expectedFiles) {
+    validateManifest(manifest, null, ids, expectedFiles);
+  }
+
+  void validateManifest(ManifestFile manifest,
+                        Iterator<Long> seqs,
+                        Iterator<Long> ids,
+                        Iterator<DataFile> expectedFiles) {
     for (ManifestEntry entry : ManifestFiles.read(manifest, FILE_IO).entries()) {
       DataFile file = entry.file();
       DataFile expected = expectedFiles.next();
+      if (seqs != null) {
+        V1Assert.assertEquals("Sequence number should default to 0", 0, entry.sequenceNumber().longValue());
+        V2Assert.assertEquals("Sequence number should match expected", seqs.next(), entry.sequenceNumber());
+      }
       Assert.assertEquals("Path should match expected",
           expected.path().toString(), file.path().toString());
       Assert.assertEquals("Snapshot ID should match expected ID",
@@ -292,6 +322,10 @@ public class TableTestBase {
     return Iterators.forArray(statuses);
   }
 
+  static Iterator<Long> seqs(long... seqs) {
+    return LongStream.of(seqs).iterator();
+  }
+
   static Iterator<Long> ids(Long... ids) {
     return Iterators.forArray(ids);
   }
@@ -303,4 +337,33 @@ public class TableTestBase {
   static Iterator<DataFile> files(ManifestFile manifest) {
     return ManifestFiles.read(manifest, FILE_IO).iterator();
   }
+
+  /**
+   * Used for assertions that only apply if the table version is v2.
+   */
+  protected static class Assertions {
+    private final boolean enabled;
+
+    private Assertions(int validForVersion, int formatVersion) {
+      this.enabled = validForVersion == formatVersion;
+    }
+
+    void assertEquals(String context, int expected, int actual) {
+      if (enabled) {
+        Assert.assertEquals(context, expected, actual);
+      }
+    }
+
+    void assertEquals(String context, long expected, long actual) {
+      if (enabled) {
+        Assert.assertEquals(context, expected, actual);
+      }
+    }
+
+    void assertEquals(String context, Object expected, Object actual) {
+      if (enabled) {
+        Assert.assertEquals(context, expected, actual);
+      }
+    }
+  }
 }
diff --git a/core/src/test/java/org/apache/iceberg/TestFastAppend.java b/core/src/test/java/org/apache/iceberg/TestFastAppend.java
index 4f071e1..4822d92 100644
--- a/core/src/test/java/org/apache/iceberg/TestFastAppend.java
+++ b/core/src/test/java/org/apache/iceberg/TestFastAppend.java
@@ -52,13 +52,21 @@ public class TestFastAppend extends TableTestBase {
 
     TableMetadata base = readMetadata();
     Assert.assertNull("Should not have a current snapshot", base.currentSnapshot());
+    Assert.assertEquals("Table should start with last-sequence-number 0", 0, base.lastSequenceNumber());
 
-    Snapshot pending = table.newFastAppend()
+    table.newFastAppend()
         .appendFile(FILE_A)
         .appendFile(FILE_B)
-        .apply();
+        .commit();
+
+    Snapshot snap = table.currentSnapshot();
+
+    validateSnapshot(base.currentSnapshot(), snap, 1, FILE_A, FILE_B);
 
-    validateSnapshot(base.currentSnapshot(), pending, FILE_A, FILE_B);
+    V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap.sequenceNumber());
+    V2Assert.assertEquals("Last sequence number should be 1", 1, readMetadata().lastSequenceNumber());
+
+    V1Assert.assertEquals("Table should end with last-sequence-number 0", 0, base.lastSequenceNumber());
   }
 
   @Test
@@ -67,17 +75,25 @@ public class TestFastAppend extends TableTestBase {
 
     TableMetadata base = readMetadata();
     Assert.assertNull("Should not have a current snapshot", base.currentSnapshot());
+    Assert.assertEquals("Table should start with last-sequence-number 0", 0, base.lastSequenceNumber());
 
     ManifestFile manifest = writeManifest(FILE_A, FILE_B);
-    Snapshot pending = table.newFastAppend()
+    table.newFastAppend()
         .appendManifest(manifest)
-        .apply();
+        .commit();
 
-    validateSnapshot(base.currentSnapshot(), pending, FILE_A, FILE_B);
+    Snapshot snap = table.currentSnapshot();
+
+    validateSnapshot(base.currentSnapshot(), snap, 1, FILE_A, FILE_B);
 
     // validate that the metadata summary is correct when using appendManifest
     Assert.assertEquals("Summary metadata should include 2 added files",
-        "2", pending.summary().get("added-data-files"));
+        "2", snap.summary().get("added-data-files"));
+
+    V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap.sequenceNumber());
+    V2Assert.assertEquals("Last sequence number should be 1", 1, readMetadata().lastSequenceNumber());
+
+    V1Assert.assertEquals("Table should end with last-sequence-number 0", 0, base.lastSequenceNumber());
   }
 
   @Test
@@ -86,22 +102,32 @@ public class TestFastAppend extends TableTestBase {
 
     TableMetadata base = readMetadata();
     Assert.assertNull("Should not have a current snapshot", base.currentSnapshot());
+    Assert.assertEquals("Table should start with last-sequence-number 0", 0, base.lastSequenceNumber());
 
     ManifestFile manifest = writeManifest(FILE_A, FILE_B);
-    Snapshot pending = table.newFastAppend()
+    table.newFastAppend()
         .appendFile(FILE_C)
         .appendFile(FILE_D)
         .appendManifest(manifest)
-        .apply();
+        .commit();
+
+    Snapshot snap = table.currentSnapshot();
 
-    long pendingId = pending.snapshotId();
+    long commitId = snap.snapshotId();
 
-    validateManifest(pending.manifests().get(0),
-        ids(pendingId, pendingId),
+    validateManifest(snap.manifests().get(0),
+        seqs(1, 1),
+        ids(commitId, commitId),
         files(FILE_C, FILE_D));
-    validateManifest(pending.manifests().get(1),
-        ids(pendingId, pendingId),
+    validateManifest(snap.manifests().get(1),
+        seqs(1, 1),
+        ids(commitId, commitId),
         files(FILE_A, FILE_B));
+
+    V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap.sequenceNumber());
+    V2Assert.assertEquals("Last sequence number should be 1", 1, readMetadata().lastSequenceNumber());
+
+    V1Assert.assertEquals("Table should end with last-sequence-number 0", 0, base.lastSequenceNumber());
   }
 
   @Test