You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iceberg.apache.org by ao...@apache.org on 2020/04/21 19:50:39 UTC

[incubator-iceberg] branch master updated: Update TableTestBase tests to run with formats v1 and v2 (#936)

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

aokolnychyi 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 a81d11e  Update TableTestBase tests to run with formats v1 and v2 (#936)
a81d11e is described below

commit a81d11e78fb447f68b2a3e8f98cf02b0c57c7f58
Author: Ryan Blue <bl...@apache.org>
AuthorDate: Tue Apr 21 12:50:32 2020 -0700

    Update TableTestBase tests to run with formats v1 and v2 (#936)
---
 .../java/org/apache/iceberg/ManifestFiles.java     |  4 +--
 .../java/org/apache/iceberg/TableMetadata.java     | 10 +++++-
 .../main/java/org/apache/iceberg/V2Metadata.java   |  7 ++--
 .../java/org/apache/iceberg/TableTestBase.java     | 10 ++++--
 .../org/apache/iceberg/TestCreateTransaction.java  | 17 +++++++++-
 .../java/org/apache/iceberg/TestDataTableScan.java | 39 +++++++---------------
 .../java/org/apache/iceberg/TestDeleteFiles.java   | 15 +++++++++
 .../apache/iceberg/TestEntriesMetadataTable.java   | 14 ++++++++
 .../java/org/apache/iceberg/TestFastAppend.java    | 14 ++++++++
 .../java/org/apache/iceberg/TestFilterFiles.java   | 24 ++++++++++---
 .../java/org/apache/iceberg/TestFindFiles.java     | 15 +++++++++
 .../org/apache/iceberg/TestFormatVersions.java     |  4 +++
 .../iceberg/TestIncrementalDataTableScan.java      | 14 ++++++++
 .../org/apache/iceberg/TestManifestCleanup.java    | 15 +++++++++
 .../org/apache/iceberg/TestManifestReader.java     | 14 ++++++++
 .../org/apache/iceberg/TestManifestWriter.java     | 32 +++++++++++++-----
 .../java/org/apache/iceberg/TestMergeAppend.java   | 15 +++++++++
 .../java/org/apache/iceberg/TestOverwrite.java     | 17 +++++++++-
 .../iceberg/TestOverwriteWithValidation.java       | 17 +++++++++-
 .../org/apache/iceberg/TestPartitionSpecInfo.java  | 23 +++++++++++--
 .../apache/iceberg/TestPartitionSpecParser.java    |  4 +++
 .../org/apache/iceberg/TestRemoveSnapshots.java    | 14 ++++++++
 .../org/apache/iceberg/TestReplacePartitions.java  | 19 +++++++++--
 .../org/apache/iceberg/TestReplaceTransaction.java | 15 +++++++++
 .../java/org/apache/iceberg/TestRewriteFiles.java  | 14 ++++++++
 .../org/apache/iceberg/TestRewriteManifests.java   | 37 +++++++++++++++-----
 .../java/org/apache/iceberg/TestScanSummary.java   | 14 ++++++++
 .../iceberg/TestScansAndSchemaEvolution.java       | 19 ++++++++++-
 .../apache/iceberg/TestSchemaAndMappingUpdate.java | 15 +++++++++
 .../test/java/org/apache/iceberg/TestSnapshot.java | 14 ++++++++
 .../org/apache/iceberg/TestSnapshotSelection.java  | 14 ++++++++
 .../java/org/apache/iceberg/TestSplitPlanning.java | 15 +++++++++
 .../test/java/org/apache/iceberg/TestTables.java   | 15 +++++++--
 .../apache/iceberg/TestTimestampPartitions.java    | 16 ++++++++-
 .../java/org/apache/iceberg/TestTransaction.java   | 15 +++++++++
 .../java/org/apache/iceberg/TestWapWorkflow.java   | 14 ++++++++
 .../apache/iceberg/mapping/TestMappingUpdates.java | 15 +++++++++
 37 files changed, 519 insertions(+), 70 deletions(-)

diff --git a/core/src/main/java/org/apache/iceberg/ManifestFiles.java b/core/src/main/java/org/apache/iceberg/ManifestFiles.java
index 2b13acc..9cbd438 100644
--- a/core/src/main/java/org/apache/iceberg/ManifestFiles.java
+++ b/core/src/main/java/org/apache/iceberg/ManifestFiles.java
@@ -73,8 +73,8 @@ public class ManifestFiles {
    * @return a manifest writer
    */
   public static ManifestWriter write(PartitionSpec spec, OutputFile outputFile) {
-    // always use a v1 writer for appended manifests because sequence number must be inherited
-    return write(1, spec, outputFile, null);
+    // always use a v2 writer to preserve sequence numbers, but use null for sequence number so appends inherit
+    return write(2, spec, outputFile, null);
   }
 
   /**
diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java
index 9b6c404..1b1021f 100644
--- a/core/src/main/java/org/apache/iceberg/TableMetadata.java
+++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java
@@ -52,6 +52,14 @@ public class TableMetadata {
                                                PartitionSpec spec,
                                                String location,
                                                Map<String, String> properties) {
+    return newTableMetadata(schema, spec, location, properties, DEFAULT_TABLE_FORMAT_VERSION);
+  }
+
+  static TableMetadata newTableMetadata(Schema schema,
+                                        PartitionSpec spec,
+                                        String location,
+                                        Map<String, String> properties,
+                                        int formatVersion) {
     // reassign all column ids to ensure consistency
     AtomicInteger lastColumnId = new AtomicInteger(0);
     Schema freshSchema = TypeUtil.assignFreshIds(schema, lastColumnId::incrementAndGet);
@@ -70,7 +78,7 @@ public class TableMetadata {
     }
     PartitionSpec freshSpec = specBuilder.build();
 
-    return new TableMetadata(null, DEFAULT_TABLE_FORMAT_VERSION, UUID.randomUUID().toString(), location,
+    return new TableMetadata(null, formatVersion, UUID.randomUUID().toString(), location,
         INITIAL_SEQUENCE_NUMBER, System.currentTimeMillis(),
         lastColumnId.get(), freshSchema, INITIAL_SPEC_ID, ImmutableList.of(freshSpec),
         ImmutableMap.copyOf(properties), -1, ImmutableList.of(),
diff --git a/core/src/main/java/org/apache/iceberg/V2Metadata.java b/core/src/main/java/org/apache/iceberg/V2Metadata.java
index 17f693d..6ee9d3f 100644
--- a/core/src/main/java/org/apache/iceberg/V2Metadata.java
+++ b/core/src/main/java/org/apache/iceberg/V2Metadata.java
@@ -243,11 +243,11 @@ class V2Metadata {
 
   static class IndexedManifestEntry implements ManifestEntry, IndexedRecord {
     private final org.apache.avro.Schema avroSchema;
-    private final long commitSnapshotId;
+    private final Long commitSnapshotId;
     private final V1Metadata.IndexedDataFile fileWrapper;
     private ManifestEntry wrapped = null;
 
-    IndexedManifestEntry(long commitSnapshotId, Types.StructType partitionType) {
+    IndexedManifestEntry(Long commitSnapshotId, Types.StructType partitionType) {
       this.avroSchema = AvroSchemaUtil.convert(entrySchema(partitionType), "manifest_entry");
       this.commitSnapshotId = commitSnapshotId;
       // TODO: when v2 data files differ from v1, this should use a v2 wrapper
@@ -281,7 +281,8 @@ class V2Metadata {
             // if the entry's sequence number is null, then it will inherit the sequence number of the current commit.
             // to validate that this is correct, check that the snapshot id is either null (will also be inherited) or
             // that it matches the id of the current commit.
-            Preconditions.checkState(wrapped.snapshotId() == null || commitSnapshotId == wrapped.snapshotId(),
+            Preconditions.checkState(
+                wrapped.snapshotId() == null || wrapped.snapshotId().equals(commitSnapshotId),
                 "Found unassigned sequence number for an entry from snapshot: %s", wrapped.snapshotId());
             return null;
           }
diff --git a/core/src/test/java/org/apache/iceberg/TableTestBase.java b/core/src/test/java/org/apache/iceberg/TableTestBase.java
index 2ff751d..c66e8b1 100644
--- a/core/src/test/java/org/apache/iceberg/TableTestBase.java
+++ b/core/src/test/java/org/apache/iceberg/TableTestBase.java
@@ -86,6 +86,12 @@ public class TableTestBase {
   File metadataDir = null;
   public TestTables.TestTable table = null;
 
+  protected final int formatVersion;
+
+  public TableTestBase(int formatVersion) {
+    this.formatVersion = formatVersion;
+  }
+
   @Before
   public void setupTable() throws Exception {
     this.tableDir = temp.newFolder();
@@ -109,8 +115,8 @@ public class TableTestBase {
         !name.startsWith("snap") && Files.getFileExtension(name).equalsIgnoreCase("avro")));
   }
 
-  private TestTables.TestTable create(Schema schema, PartitionSpec spec) {
-    return TestTables.create(tableDir, "test", schema, spec);
+  TestTables.TestTable create(Schema schema, PartitionSpec spec) {
+    return TestTables.create(tableDir, "test", schema, spec, formatVersion);
   }
 
   TestTables.TestTable load() {
diff --git a/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java b/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java
index 90b4644..342e913 100644
--- a/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java
+++ b/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java
@@ -26,10 +26,25 @@ import org.apache.iceberg.exceptions.CommitFailedException;
 import org.apache.iceberg.types.TypeUtil;
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import static org.apache.iceberg.PartitionSpec.unpartitioned;
 
+@RunWith(Parameterized.class)
 public class TestCreateTransaction extends TableTestBase {
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestCreateTransaction(int formatVersion) {
+    super(formatVersion);
+  }
+
   @Test
   public void testCreateTransaction() throws IOException {
     File tableDir = temp.newFolder();
@@ -273,7 +288,7 @@ public class TestCreateTransaction extends TableTestBase {
     Assert.assertNull("Should have no metadata version",
         TestTables.metadataVersion("test_conflict"));
 
-    Table conflict = TestTables.create(tableDir, "test_conflict", SCHEMA, unpartitioned());
+    Table conflict = TestTables.create(tableDir, "test_conflict", SCHEMA, unpartitioned(), formatVersion);
 
     Assert.assertEquals("Table schema should match with reassigned IDs",
         TypeUtil.assignIncreasingFreshIds(SCHEMA).asStruct(), conflict.schema().asStruct());
diff --git a/core/src/test/java/org/apache/iceberg/TestDataTableScan.java b/core/src/test/java/org/apache/iceberg/TestDataTableScan.java
index 5c3dc90..ff75d52 100644
--- a/core/src/test/java/org/apache/iceberg/TestDataTableScan.java
+++ b/core/src/test/java/org/apache/iceberg/TestDataTableScan.java
@@ -19,42 +19,30 @@
 
 package org.apache.iceberg;
 
-import java.io.File;
-import java.io.IOException;
 import org.apache.iceberg.types.Types;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import static org.apache.iceberg.types.Types.NestedField.required;
 import static org.junit.Assert.assertEquals;
 
-public class TestDataTableScan {
-
-  @Rule
-  public TemporaryFolder temp = new TemporaryFolder();
-  private final Schema schema = new Schema(
-      required(1, "id", Types.IntegerType.get()),
-      required(2, "data", Types.StringType.get()));
-  private File tableDir = null;
-
-  @Before
-  public void setupTableDir() throws IOException {
-    this.tableDir = temp.newFolder();
+@RunWith(Parameterized.class)
+public class TestDataTableScan extends TableTestBase {
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
   }
 
-  @After
-  public void cleanupTables() {
-    TestTables.clearTables();
+  public TestDataTableScan(int formatVersion) {
+    super(formatVersion);
   }
 
   @Test
   public void testTableScanHonorsSelect() {
-    PartitionSpec spec = PartitionSpec.unpartitioned();
-    Table table = TestTables.create(tableDir, "test", schema, spec);
-
     TableScan scan = table.newScan().select("id");
 
     Schema expectedSchema = new Schema(required(1, "id", Types.IntegerType.get()));
@@ -66,9 +54,6 @@ public class TestDataTableScan {
 
   @Test
   public void testTableScanHonorsSelectWithoutCaseSensitivity() {
-    PartitionSpec spec = PartitionSpec.unpartitioned();
-    Table table = TestTables.create(tableDir, "test", schema, spec);
-
     TableScan scan1 = table.newScan().caseSensitive(false).select("ID");
     // order of refinements shouldn't matter
     TableScan scan2 = table.newScan().select("ID").caseSensitive(false);
diff --git a/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java b/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java
index cb95d02..bdc5537 100644
--- a/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java
+++ b/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java
@@ -22,8 +22,23 @@ package org.apache.iceberg;
 import org.apache.iceberg.ManifestEntry.Status;
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+@RunWith(Parameterized.class)
 public class TestDeleteFiles extends TableTestBase {
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestDeleteFiles(int formatVersion) {
+    super(formatVersion);
+  }
+
   @Test
   public void testMultipleDeletes() {
     table.newAppend()
diff --git a/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java b/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java
index 5e0ff86..176d3b1 100644
--- a/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java
+++ b/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java
@@ -22,10 +22,24 @@ package org.apache.iceberg;
 import com.google.common.collect.Iterables;
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import static org.junit.Assert.assertEquals;
 
+@RunWith(Parameterized.class)
 public class TestEntriesMetadataTable extends TableTestBase {
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestEntriesMetadataTable(int formatVersion) {
+    super(formatVersion);
+  }
 
   @Test
   public void testEntriesTable() {
diff --git a/core/src/test/java/org/apache/iceberg/TestFastAppend.java b/core/src/test/java/org/apache/iceberg/TestFastAppend.java
index f8cb6d2..4f071e1 100644
--- a/core/src/test/java/org/apache/iceberg/TestFastAppend.java
+++ b/core/src/test/java/org/apache/iceberg/TestFastAppend.java
@@ -29,8 +29,22 @@ import org.apache.iceberg.ManifestEntry.Status;
 import org.apache.iceberg.exceptions.CommitFailedException;
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+@RunWith(Parameterized.class)
 public class TestFastAppend extends TableTestBase {
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestFastAppend(int formatVersion) {
+    super(formatVersion);
+  }
 
   @Test
   public void testEmptyTableAppend() {
diff --git a/core/src/test/java/org/apache/iceberg/TestFilterFiles.java b/core/src/test/java/org/apache/iceberg/TestFilterFiles.java
index 3a1c886..d023048 100644
--- a/core/src/test/java/org/apache/iceberg/TestFilterFiles.java
+++ b/core/src/test/java/org/apache/iceberg/TestFilterFiles.java
@@ -34,11 +34,27 @@ import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import static org.apache.iceberg.types.Types.NestedField.required;
 import static org.junit.Assert.assertEquals;
 
+@RunWith(Parameterized.class)
 public class TestFilterFiles {
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public final int formatVersion;
+
+  public TestFilterFiles(int formatVersion) {
+    this.formatVersion = formatVersion;
+  }
 
   @Rule
   public TemporaryFolder temp = new TemporaryFolder();
@@ -60,28 +76,28 @@ public class TestFilterFiles {
   @Test
   public void testFilterFilesUnpartitionedTable() {
     PartitionSpec spec = PartitionSpec.unpartitioned();
-    Table table = TestTables.create(tableDir, "test", schema, spec);
+    Table table = TestTables.create(tableDir, "test", schema, spec, formatVersion);
     testFilterFiles(table);
   }
 
   @Test
   public void testCaseInsensitiveFilterFilesUnpartitionedTable() {
     PartitionSpec spec = PartitionSpec.unpartitioned();
-    Table table = TestTables.create(tableDir, "test", schema, spec);
+    Table table = TestTables.create(tableDir, "test", schema, spec, formatVersion);
     testCaseInsensitiveFilterFiles(table);
   }
 
   @Test
   public void testFilterFilesPartitionedTable() {
     PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("data", 16).build();
-    Table table = TestTables.create(tableDir, "test", schema, spec);
+    Table table = TestTables.create(tableDir, "test", schema, spec, formatVersion);
     testFilterFiles(table);
   }
 
   @Test
   public void testCaseInsensitiveFilterFilesPartitionedTable() {
     PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("data", 16).build();
-    Table table = TestTables.create(tableDir, "test", schema, spec);
+    Table table = TestTables.create(tableDir, "test", schema, spec, formatVersion);
     testCaseInsensitiveFilterFiles(table);
   }
 
diff --git a/core/src/test/java/org/apache/iceberg/TestFindFiles.java b/core/src/test/java/org/apache/iceberg/TestFindFiles.java
index 4cb5bdd..68121d3 100644
--- a/core/src/test/java/org/apache/iceberg/TestFindFiles.java
+++ b/core/src/test/java/org/apache/iceberg/TestFindFiles.java
@@ -29,8 +29,23 @@ import org.apache.iceberg.types.Conversions;
 import org.apache.iceberg.types.Types;
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+@RunWith(Parameterized.class)
 public class TestFindFiles extends TableTestBase {
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestFindFiles(int formatVersion) {
+    super(formatVersion);
+  }
+
   @Test
   public void testBasicBehavior() {
     table.newAppend()
diff --git a/core/src/test/java/org/apache/iceberg/TestFormatVersions.java b/core/src/test/java/org/apache/iceberg/TestFormatVersions.java
index f58d70b..1d04834 100644
--- a/core/src/test/java/org/apache/iceberg/TestFormatVersions.java
+++ b/core/src/test/java/org/apache/iceberg/TestFormatVersions.java
@@ -23,6 +23,10 @@ import org.junit.Assert;
 import org.junit.Test;
 
 public class TestFormatVersions extends TableTestBase {
+  public TestFormatVersions() {
+    super(1);
+  }
+
   @Test
   public void testDefaultFormatVersion() {
     Assert.assertEquals("Should default to v1", 1, table.ops().current().formatVersion());
diff --git a/core/src/test/java/org/apache/iceberg/TestIncrementalDataTableScan.java b/core/src/test/java/org/apache/iceberg/TestIncrementalDataTableScan.java
index 22b5848..485d4c2 100644
--- a/core/src/test/java/org/apache/iceberg/TestIncrementalDataTableScan.java
+++ b/core/src/test/java/org/apache/iceberg/TestIncrementalDataTableScan.java
@@ -27,8 +27,22 @@ import java.util.List;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+@RunWith(Parameterized.class)
 public class TestIncrementalDataTableScan extends TableTestBase {
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestIncrementalDataTableScan(int formatVersion) {
+    super(formatVersion);
+  }
 
   @Before
   public void setupTableProperties() {
diff --git a/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java b/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java
index 917862d..acb08bb 100644
--- a/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java
+++ b/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java
@@ -22,8 +22,23 @@ package org.apache.iceberg;
 import org.apache.iceberg.expressions.Expressions;
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+@RunWith(Parameterized.class)
 public class TestManifestCleanup extends TableTestBase {
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestManifestCleanup(int formatVersion) {
+    super(formatVersion);
+  }
+
   @Test
   public void testDelete() {
     Assert.assertEquals("Table should start with no manifests",
diff --git a/core/src/test/java/org/apache/iceberg/TestManifestReader.java b/core/src/test/java/org/apache/iceberg/TestManifestReader.java
index 76ede62..058a845 100644
--- a/core/src/test/java/org/apache/iceberg/TestManifestReader.java
+++ b/core/src/test/java/org/apache/iceberg/TestManifestReader.java
@@ -26,8 +26,22 @@ import org.apache.iceberg.ManifestEntry.Status;
 import org.apache.iceberg.types.Types;
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+@RunWith(Parameterized.class)
 public class TestManifestReader extends TableTestBase {
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestManifestReader(int formatVersion) {
+    super(formatVersion);
+  }
 
   @Test
   @SuppressWarnings("deprecation")
diff --git a/core/src/test/java/org/apache/iceberg/TestManifestWriter.java b/core/src/test/java/org/apache/iceberg/TestManifestWriter.java
index 9a67ac7..a344e74 100644
--- a/core/src/test/java/org/apache/iceberg/TestManifestWriter.java
+++ b/core/src/test/java/org/apache/iceberg/TestManifestWriter.java
@@ -24,22 +24,36 @@ import java.util.UUID;
 import org.apache.iceberg.ManifestEntry.Status;
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+@RunWith(Parameterized.class)
 public class TestManifestWriter extends TableTestBase {
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestManifestWriter(int formatVersion) {
+    super(formatVersion);
+  }
 
   @Test
   public void testManifestStats() throws IOException {
     ManifestFile manifest = writeManifest(
         "manifest.avro",
-        manifestEntry(Status.ADDED, 100L, newFile(10)),
-        manifestEntry(Status.ADDED, 100L, newFile(20)),
-        manifestEntry(Status.ADDED, 100L, newFile(5)),
-        manifestEntry(Status.ADDED, 100L, newFile(5)),
-        manifestEntry(Status.EXISTING, 100L, newFile(15)),
-        manifestEntry(Status.EXISTING, 100L, newFile(10)),
-        manifestEntry(Status.EXISTING, 100L, newFile(1)),
-        manifestEntry(Status.DELETED, 100L, newFile(5)),
-        manifestEntry(Status.DELETED, 100L, newFile(2)));
+        manifestEntry(Status.ADDED, null, newFile(10)),
+        manifestEntry(Status.ADDED, null, newFile(20)),
+        manifestEntry(Status.ADDED, null, newFile(5)),
+        manifestEntry(Status.ADDED, null, newFile(5)),
+        manifestEntry(Status.EXISTING, null, newFile(15)),
+        manifestEntry(Status.EXISTING, null, newFile(10)),
+        manifestEntry(Status.EXISTING, null, newFile(1)),
+        manifestEntry(Status.DELETED, null, newFile(5)),
+        manifestEntry(Status.DELETED, null, newFile(2)));
 
     Assert.assertTrue("Added files should be present", manifest.hasAddedFiles());
     Assert.assertEquals("Added files count should match", 4, (int) manifest.addedFilesCount());
diff --git a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java
index d4dcff9..88dc997 100644
--- a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java
+++ b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java
@@ -30,10 +30,25 @@ import org.apache.iceberg.exceptions.CommitFailedException;
 import org.apache.iceberg.types.Types;
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import static com.google.common.collect.Iterators.concat;
 
+@RunWith(Parameterized.class)
 public class TestMergeAppend extends TableTestBase {
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestMergeAppend(int formatVersion) {
+    super(formatVersion);
+  }
+
   @Test
   public void testEmptyTableAppend() {
     Assert.assertEquals("Table should start empty", 0, listManifestFiles().size());
diff --git a/core/src/test/java/org/apache/iceberg/TestOverwrite.java b/core/src/test/java/org/apache/iceberg/TestOverwrite.java
index 81f1a99..dca9875 100644
--- a/core/src/test/java/org/apache/iceberg/TestOverwrite.java
+++ b/core/src/test/java/org/apache/iceberg/TestOverwrite.java
@@ -30,6 +30,8 @@ import org.apache.iceberg.types.Types;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import static org.apache.iceberg.expressions.Expressions.and;
 import static org.apache.iceberg.expressions.Expressions.equal;
@@ -37,6 +39,7 @@ import static org.apache.iceberg.expressions.Expressions.lessThan;
 import static org.apache.iceberg.types.Types.NestedField.optional;
 import static org.apache.iceberg.types.Types.NestedField.required;
 
+@RunWith(Parameterized.class)
 public class TestOverwrite extends TableTestBase {
   private static final Schema DATE_SCHEMA = new Schema(
       required(1, "id", Types.LongType.get()),
@@ -89,6 +92,18 @@ public class TestOverwrite extends TableTestBase {
       ))
       .build();
 
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestOverwrite(int formatVersion) {
+    super(formatVersion);
+  }
+
   private static ByteBuffer longToBuffer(long value) {
     return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, value);
   }
@@ -100,7 +115,7 @@ public class TestOverwrite extends TableTestBase {
     File tableDir = temp.newFolder();
     Assert.assertTrue(tableDir.delete());
 
-    this.table = TestTables.create(tableDir, TABLE_NAME, DATE_SCHEMA, PARTITION_BY_DATE);
+    this.table = TestTables.create(tableDir, TABLE_NAME, DATE_SCHEMA, PARTITION_BY_DATE, formatVersion);
 
     table.newAppend()
         .appendFile(FILE_0_TO_4)
diff --git a/core/src/test/java/org/apache/iceberg/TestOverwriteWithValidation.java b/core/src/test/java/org/apache/iceberg/TestOverwriteWithValidation.java
index 1997067..9eea57c 100644
--- a/core/src/test/java/org/apache/iceberg/TestOverwriteWithValidation.java
+++ b/core/src/test/java/org/apache/iceberg/TestOverwriteWithValidation.java
@@ -31,6 +31,8 @@ import org.apache.iceberg.types.Types;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import static org.apache.iceberg.expressions.Expressions.alwaysTrue;
 import static org.apache.iceberg.expressions.Expressions.and;
@@ -40,6 +42,7 @@ import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
 import static org.apache.iceberg.types.Types.NestedField.optional;
 import static org.apache.iceberg.types.Types.NestedField.required;
 
+@RunWith(Parameterized.class)
 public class TestOverwriteWithValidation extends TableTestBase {
 
   private static final String TABLE_NAME = "overwrite_table";
@@ -116,6 +119,18 @@ public class TestOverwriteWithValidation extends TableTestBase {
       greaterThanOrEqual("id", 5L),
       lessThanOrEqual("id", 9L));
 
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestOverwriteWithValidation(int formatVersion) {
+    super(formatVersion);
+  }
+
   private static ByteBuffer longToBuffer(long value) {
     return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, value);
   }
@@ -126,7 +141,7 @@ public class TestOverwriteWithValidation extends TableTestBase {
   public void before() throws IOException {
     File tableDir = temp.newFolder();
     Assert.assertTrue(tableDir.delete());
-    this.table = TestTables.create(tableDir, TABLE_NAME, DATE_SCHEMA, PARTITION_SPEC);
+    this.table = TestTables.create(tableDir, TABLE_NAME, DATE_SCHEMA, PARTITION_SPEC, formatVersion);
   }
 
   @Test
diff --git a/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java b/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java
index 4c7d7c4..2d70bbe 100644
--- a/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java
+++ b/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java
@@ -29,9 +29,12 @@ import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import static org.apache.iceberg.types.Types.NestedField.required;
 
+@RunWith(Parameterized.class)
 public class TestPartitionSpecInfo {
 
   @Rule
@@ -41,6 +44,20 @@ public class TestPartitionSpecInfo {
       required(2, "data", Types.StringType.get()));
   private File tableDir = null;
 
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  private final int formatVersion;
+
+  public TestPartitionSpecInfo(int formatVersion) {
+    this.formatVersion = formatVersion;
+  }
+
   @Before
   public void setupTableDir() throws IOException {
     this.tableDir = temp.newFolder();
@@ -54,7 +71,7 @@ public class TestPartitionSpecInfo {
   @Test
   public void testSpecInfoUnpartitionedTable() {
     PartitionSpec spec = PartitionSpec.unpartitioned();
-    TestTables.TestTable table = TestTables.create(tableDir, "test", schema, spec);
+    TestTables.TestTable table = TestTables.create(tableDir, "test", schema, spec, formatVersion);
 
     Assert.assertEquals(spec, table.spec());
     Assert.assertEquals(spec.lastAssignedFieldId(), table.spec().lastAssignedFieldId());
@@ -65,7 +82,7 @@ public class TestPartitionSpecInfo {
   @Test
   public void testSpecInfoPartitionedTable() {
     PartitionSpec spec = PartitionSpec.builderFor(schema).identity("data").build();
-    TestTables.TestTable table = TestTables.create(tableDir, "test", schema, spec);
+    TestTables.TestTable table = TestTables.create(tableDir, "test", schema, spec, formatVersion);
 
     Assert.assertEquals(spec, table.spec());
     Assert.assertEquals(spec.lastAssignedFieldId(), table.spec().lastAssignedFieldId());
@@ -78,7 +95,7 @@ public class TestPartitionSpecInfo {
     PartitionSpec spec = PartitionSpec.builderFor(schema)
         .bucket("data", 4)
         .build();
-    TestTables.TestTable table = TestTables.create(tableDir, "test", schema, spec);
+    TestTables.TestTable table = TestTables.create(tableDir, "test", schema, spec, formatVersion);
 
     Assert.assertEquals(spec, table.spec());
 
diff --git a/core/src/test/java/org/apache/iceberg/TestPartitionSpecParser.java b/core/src/test/java/org/apache/iceberg/TestPartitionSpecParser.java
index c31f894..847ff42 100644
--- a/core/src/test/java/org/apache/iceberg/TestPartitionSpecParser.java
+++ b/core/src/test/java/org/apache/iceberg/TestPartitionSpecParser.java
@@ -23,6 +23,10 @@ import org.junit.Assert;
 import org.junit.Test;
 
 public class TestPartitionSpecParser extends TableTestBase {
+  public TestPartitionSpecParser() {
+    super(1);
+  }
+
   @Test
   public void testToJsonForV1Table() {
     String expected = "{\n" +
diff --git a/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java b/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java
index ed6b250..ae3b45f 100644
--- a/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java
+++ b/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java
@@ -30,8 +30,22 @@ import java.util.Set;
 import org.apache.iceberg.ManifestEntry.Status;
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+@RunWith(Parameterized.class)
 public class TestRemoveSnapshots extends TableTestBase {
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestRemoveSnapshots(int formatVersion) {
+    super(formatVersion);
+  }
 
   @Test
   public void testRetainLastWithExpireOlderThan() {
diff --git a/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java b/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java
index e677d38..811cc7e 100644
--- a/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java
+++ b/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java
@@ -25,7 +25,10 @@ import org.apache.iceberg.ManifestEntry.Status;
 import org.apache.iceberg.exceptions.ValidationException;
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+@RunWith(Parameterized.class)
 public class TestReplacePartitions extends TableTestBase {
 
   static final DataFile FILE_E = DataFiles.builder(SPEC)
@@ -49,6 +52,18 @@ public class TestReplacePartitions extends TableTestBase {
       .withRecordCount(0)
       .build();
 
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestReplacePartitions(int formatVersion) {
+    super(formatVersion);
+  }
+
   @Test
   public void testReplaceOnePartition() {
     table.newFastAppend()
@@ -114,7 +129,7 @@ public class TestReplacePartitions extends TableTestBase {
     Assert.assertTrue(tableDir.delete());
 
     Table unpartitioned = TestTables.create(
-        tableDir, "unpartitioned", SCHEMA, PartitionSpec.unpartitioned());
+        tableDir, "unpartitioned", SCHEMA, PartitionSpec.unpartitioned(), formatVersion);
 
     Assert.assertEquals("Table version should be 0",
         0, (long) TestTables.metadataVersion("unpartitioned"));
@@ -153,7 +168,7 @@ public class TestReplacePartitions extends TableTestBase {
     Assert.assertTrue(tableDir.delete());
 
     Table unpartitioned = TestTables.create(
-        tableDir, "unpartitioned", SCHEMA, PartitionSpec.unpartitioned());
+        tableDir, "unpartitioned", SCHEMA, PartitionSpec.unpartitioned(), formatVersion);
 
     // ensure the overwrite results in a merge
     unpartitioned.updateProperties().set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "1").commit();
diff --git a/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java b/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java
index 68ef9b9..9a35dfb 100644
--- a/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java
+++ b/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java
@@ -29,11 +29,26 @@ import org.apache.iceberg.types.TypeUtil;
 import org.apache.iceberg.types.Types;
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import static org.apache.iceberg.PartitionSpec.unpartitioned;
 import static org.apache.iceberg.types.Types.NestedField.required;
 
+@RunWith(Parameterized.class)
 public class TestReplaceTransaction extends TableTestBase {
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestReplaceTransaction(int formatVersion) {
+    super(formatVersion);
+  }
+
   @Test
   public void testReplaceTransaction() {
     Schema newSchema = new Schema(
diff --git a/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java b/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java
index 9c4258b..b4b64e7 100644
--- a/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java
+++ b/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java
@@ -25,13 +25,27 @@ import org.apache.iceberg.exceptions.CommitFailedException;
 import org.apache.iceberg.exceptions.ValidationException;
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 import org.mockito.internal.util.collections.Sets;
 
 import static org.apache.iceberg.ManifestEntry.Status.ADDED;
 import static org.apache.iceberg.ManifestEntry.Status.DELETED;
 import static org.apache.iceberg.ManifestEntry.Status.EXISTING;
 
+@RunWith(Parameterized.class)
 public class TestRewriteFiles extends TableTestBase {
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestRewriteFiles(int formatVersion) {
+    super(formatVersion);
+  }
 
   @Test
   public void testEmptyTable() {
diff --git a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java
index 1d85a2e..c664bf5 100644
--- a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java
+++ b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java
@@ -32,13 +32,27 @@ import org.apache.iceberg.exceptions.ValidationException;
 import org.apache.iceberg.expressions.Expressions;
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import static org.apache.iceberg.TableProperties.MANIFEST_MERGE_ENABLED;
 import static org.apache.iceberg.TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
 
+@RunWith(Parameterized.class)
 public class TestRewriteManifests extends TableTestBase {
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestRewriteManifests(int formatVersion) {
+    super(formatVersion);
+  }
 
   @Test
   public void testRewriteManifestsAppendedDirectly() throws IOException {
@@ -941,9 +955,10 @@ public class TestRewriteManifests extends TableTestBase {
 
     Assert.assertEquals(3, Iterables.size(table.snapshots()));
 
-    ManifestFile newManifest = writeManifest(
-        "manifest-file-1.avro",
-        manifestEntry(ManifestEntry.Status.EXISTING, firstSnapshot.snapshotId(), FILE_A));
+    ManifestEntry entry = manifestEntry(ManifestEntry.Status.EXISTING, firstSnapshot.snapshotId(), FILE_A);
+    // update the entry's sequence number or else it will be rejected by the writer
+    entry.setSequenceNumber(firstSnapshot.sequenceNumber());
+    ManifestFile newManifest = writeManifest("manifest-file-1.avro", entry);
 
     RewriteManifests rewriteManifests = table.rewriteManifests()
         .deleteManifest(firstSnapshotManifest)
@@ -990,9 +1005,11 @@ public class TestRewriteManifests extends TableTestBase {
     Assert.assertEquals(1, manifests.size());
     ManifestFile manifest = manifests.get(0);
 
-    ManifestFile invalidAddedFileManifest = writeManifest(
-        "manifest-file-2.avro",
-        manifestEntry(ManifestEntry.Status.ADDED, snapshot.snapshotId(), FILE_A));
+    ManifestEntry appendEntry = manifestEntry(ManifestEntry.Status.ADDED, snapshot.snapshotId(), FILE_A);
+    // update the entry's sequence number or else it will be rejected by the writer
+    appendEntry.setSequenceNumber(snapshot.sequenceNumber());
+
+    ManifestFile invalidAddedFileManifest = writeManifest("manifest-file-2.avro", appendEntry);
 
     AssertHelpers.assertThrows("Should reject commit",
         IllegalArgumentException.class, "Cannot add manifest with added files",
@@ -1001,9 +1018,11 @@ public class TestRewriteManifests extends TableTestBase {
             .addManifest(invalidAddedFileManifest)
             .commit());
 
-    ManifestFile invalidDeletedFileManifest = writeManifest(
-        "manifest-file-3.avro",
-        manifestEntry(ManifestEntry.Status.DELETED, snapshot.snapshotId(), FILE_A));
+    ManifestEntry deleteEntry = manifestEntry(ManifestEntry.Status.DELETED, snapshot.snapshotId(), FILE_A);
+    // update the entry's sequence number or else it will be rejected by the writer
+    deleteEntry.setSequenceNumber(snapshot.sequenceNumber());
+
+    ManifestFile invalidDeletedFileManifest = writeManifest("manifest-file-3.avro", deleteEntry);
 
     AssertHelpers.assertThrows("Should reject commit",
         IllegalArgumentException.class, "Cannot add manifest with deleted files",
diff --git a/core/src/test/java/org/apache/iceberg/TestScanSummary.java b/core/src/test/java/org/apache/iceberg/TestScanSummary.java
index 2110bef..77068f9 100644
--- a/core/src/test/java/org/apache/iceberg/TestScanSummary.java
+++ b/core/src/test/java/org/apache/iceberg/TestScanSummary.java
@@ -24,6 +24,8 @@ import com.google.common.collect.Lists;
 import org.apache.iceberg.util.Pair;
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import static org.apache.iceberg.ScanSummary.timestampRange;
 import static org.apache.iceberg.ScanSummary.toMillis;
@@ -33,7 +35,19 @@ import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
 import static org.apache.iceberg.expressions.Expressions.lessThan;
 import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
 
+@RunWith(Parameterized.class)
 public class TestScanSummary extends TableTestBase {
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestScanSummary(int formatVersion) {
+    super(formatVersion);
+  }
 
   @Test
   public void testSnapshotTimeRangeValidation() {
diff --git a/core/src/test/java/org/apache/iceberg/TestScansAndSchemaEvolution.java b/core/src/test/java/org/apache/iceberg/TestScansAndSchemaEvolution.java
index 6add3a8..274ee80 100644
--- a/core/src/test/java/org/apache/iceberg/TestScansAndSchemaEvolution.java
+++ b/core/src/test/java/org/apache/iceberg/TestScansAndSchemaEvolution.java
@@ -35,9 +35,12 @@ import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import static org.apache.iceberg.types.Types.NestedField.required;
 
+@RunWith(Parameterized.class)
 public class TestScansAndSchemaEvolution {
   private static final Schema SCHEMA = new Schema(
       required(1, "id", Types.LongType.get()),
@@ -48,6 +51,20 @@ public class TestScansAndSchemaEvolution {
       .identity("part")
       .build();
 
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public final int formatVersion;
+
+  public TestScansAndSchemaEvolution(int formatVersion) {
+    this.formatVersion = formatVersion;
+  }
+
   @Rule
   public TemporaryFolder temp = new TemporaryFolder();
 
@@ -85,7 +102,7 @@ public class TestScansAndSchemaEvolution {
     File dataLocation = new File(location, "data");
     Assert.assertTrue(location.delete()); // should be created by table create
 
-    Table table = TestTables.create(location, "test", SCHEMA, SPEC);
+    Table table = TestTables.create(location, "test", SCHEMA, SPEC, formatVersion);
 
     DataFile fileOne = createDataFile(dataLocation, "one");
     DataFile fileTwo = createDataFile(dataLocation, "two");
diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java b/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java
index 349a797..07e5a26 100644
--- a/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java
+++ b/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java
@@ -32,8 +32,23 @@ import org.apache.iceberg.mapping.NameMappingParser;
 import org.apache.iceberg.types.Types;
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+@RunWith(Parameterized.class)
 public class TestSchemaAndMappingUpdate extends TableTestBase {
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestSchemaAndMappingUpdate(int formatVersion) {
+    super(formatVersion);
+  }
+
   @Test
   public void testAddPrimitiveColumn() {
     NameMapping mapping = MappingUtil.create(table.schema());
diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshot.java b/core/src/test/java/org/apache/iceberg/TestSnapshot.java
index 1374fbe..91b8716 100644
--- a/core/src/test/java/org/apache/iceberg/TestSnapshot.java
+++ b/core/src/test/java/org/apache/iceberg/TestSnapshot.java
@@ -20,8 +20,22 @@
 package org.apache.iceberg;
 
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+@RunWith(Parameterized.class)
 public class TestSnapshot extends TableTestBase {
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestSnapshot(int formatVersion) {
+    super(formatVersion);
+  }
 
   @Test
   public void testAppendFilesFromTable() {
diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotSelection.java b/core/src/test/java/org/apache/iceberg/TestSnapshotSelection.java
index 9da4c4b..17d54c2 100644
--- a/core/src/test/java/org/apache/iceberg/TestSnapshotSelection.java
+++ b/core/src/test/java/org/apache/iceberg/TestSnapshotSelection.java
@@ -25,8 +25,22 @@ import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+@RunWith(Parameterized.class)
 public class TestSnapshotSelection extends TableTestBase {
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestSnapshotSelection(int formatVersion) {
+    super(formatVersion);
+  }
 
   @Test
   public void testSnapshotSelectionById() {
diff --git a/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java b/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java
index 0d78870..e827def 100644
--- a/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java
+++ b/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java
@@ -34,9 +34,12 @@ import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import static org.apache.iceberg.types.Types.NestedField.optional;
 
+@RunWith(Parameterized.class)
 public class TestSplitPlanning extends TableTestBase {
 
   private static final Configuration CONF = new Configuration();
@@ -50,6 +53,18 @@ public class TestSplitPlanning extends TableTestBase {
   public TemporaryFolder temp = new TemporaryFolder();
   private Table table = null;
 
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestSplitPlanning(int formatVersion) {
+    super(formatVersion);
+  }
+
   @Before
   public void setupTable() throws IOException {
     File tableDir = temp.newFolder();
diff --git a/core/src/test/java/org/apache/iceberg/TestTables.java b/core/src/test/java/org/apache/iceberg/TestTables.java
index 41974e3..e7897ce 100644
--- a/core/src/test/java/org/apache/iceberg/TestTables.java
+++ b/core/src/test/java/org/apache/iceberg/TestTables.java
@@ -38,12 +38,20 @@ public class TestTables {
 
   private TestTables() {}
 
-  public static TestTable create(File temp, String name, Schema schema, PartitionSpec spec) {
+  private static TestTable upgrade(File temp, String name, int newFormatVersion) {
+    TestTable table = load(temp, name);
+    TableOperations ops = table.ops();
+    TableMetadata base = ops.current();
+    ops.commit(base, ops.current().upgradeToFormatVersion(newFormatVersion));
+    return table;
+  }
+
+  public static TestTable create(File temp, String name, Schema schema, PartitionSpec spec, int formatVersion) {
     TestTableOperations ops = new TestTableOperations(name, temp);
     if (ops.current() != null) {
       throw new AlreadyExistsException("Table %s already exists at location: %s", name, temp);
     }
-    ops.commit(null, TableMetadata.newTableMetadata(schema, spec, temp.toString(), ImmutableMap.of()));
+    ops.commit(null, TableMetadata.newTableMetadata(schema, spec, temp.toString(), ImmutableMap.of(), formatVersion));
     return new TestTable(ops, name);
   }
 
@@ -53,7 +61,8 @@ public class TestTables {
       throw new AlreadyExistsException("Table %s already exists at location: %s", name, temp);
     }
 
-    TableMetadata metadata = TableMetadata.newTableMetadata(schema, spec, temp.toString(), ImmutableMap.of());
+    TableMetadata metadata = TableMetadata.newTableMetadata(
+        schema, spec, temp.toString(), ImmutableMap.of(), 1);
 
     return Transactions.createTableTransaction(ops, metadata);
   }
diff --git a/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java b/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java
index fa6c98f..2343b7c 100644
--- a/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java
+++ b/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java
@@ -24,11 +24,25 @@ import java.io.IOException;
 import org.apache.iceberg.types.Types;
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import static org.apache.iceberg.types.Types.NestedField.optional;
 import static org.apache.iceberg.types.Types.NestedField.required;
 
+@RunWith(Parameterized.class)
 public class TestTimestampPartitions extends TableTestBase {
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestTimestampPartitions(int formatVersion) {
+    super(formatVersion);
+  }
 
   @Test
   public void testPartitionAppend() throws IOException {
@@ -52,7 +66,7 @@ public class TestTimestampPartitions extends TableTestBase {
     File tableDir = temp.newFolder();
     Assert.assertTrue(tableDir.delete());
 
-    this.table = TestTables.create(tableDir, "test_date_partition", dateSchema, partitionSpec);
+    this.table = TestTables.create(tableDir, "test_date_partition", dateSchema, partitionSpec, formatVersion);
 
     table.newAppend()
         .appendFile(dataFile)
diff --git a/core/src/test/java/org/apache/iceberg/TestTransaction.java b/core/src/test/java/org/apache/iceberg/TestTransaction.java
index 8f0fedb..d855e16 100644
--- a/core/src/test/java/org/apache/iceberg/TestTransaction.java
+++ b/core/src/test/java/org/apache/iceberg/TestTransaction.java
@@ -31,8 +31,23 @@ import org.apache.iceberg.exceptions.CommitFailedException;
 import org.apache.iceberg.io.OutputFile;
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+@RunWith(Parameterized.class)
 public class TestTransaction extends TableTestBase {
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestTransaction(int formatVersion) {
+    super(formatVersion);
+  }
+
   @Test
   public void testEmptyTransaction() {
     Assert.assertEquals("Table should be on version 0", 0, (int) version());
diff --git a/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java b/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java
index 9f14860..b6c75f3 100644
--- a/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java
+++ b/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java
@@ -27,8 +27,22 @@ import org.apache.iceberg.exceptions.ValidationException;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+@RunWith(Parameterized.class)
 public class TestWapWorkflow extends TableTestBase {
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestWapWorkflow(int formatVersion) {
+    super(formatVersion);
+  }
 
   @Before
   public void setupTableProperties() {
diff --git a/core/src/test/java/org/apache/iceberg/mapping/TestMappingUpdates.java b/core/src/test/java/org/apache/iceberg/mapping/TestMappingUpdates.java
index 8f23b64..ae30bc6 100644
--- a/core/src/test/java/org/apache/iceberg/mapping/TestMappingUpdates.java
+++ b/core/src/test/java/org/apache/iceberg/mapping/TestMappingUpdates.java
@@ -25,10 +25,25 @@ import org.apache.iceberg.TableTestBase;
 import org.apache.iceberg.types.Types;
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import static org.apache.iceberg.types.Types.NestedField.required;
 
+@RunWith(Parameterized.class)
 public class TestMappingUpdates extends TableTestBase {
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { 1 },
+        new Object[] { 2 },
+    };
+  }
+
+  public TestMappingUpdates(int formatVersion) {
+    super(formatVersion);
+  }
+
   @Test
   public void testAddColumnMappingUpdate() {
     NameMapping mapping = MappingUtil.create(table.schema());