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 2022/10/27 04:31:37 UTC

[iceberg] branch master updated: Spec, Core: Add file seq number to ManifestEntry (#6002)

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/iceberg.git


The following commit(s) were added to refs/heads/master by this push:
     new 8bff303038 Spec, Core: Add file seq number to ManifestEntry (#6002)
8bff303038 is described below

commit 8bff3030386b2479985fc0ac0dc02e4f1ba45620
Author: Anton Okolnychyi <ao...@apple.com>
AuthorDate: Wed Oct 26 21:31:31 2022 -0700

    Spec, Core: Add file seq number to ManifestEntry (#6002)
---
 .../org/apache/iceberg/GenericManifestEntry.java   |  39 +++++-
 .../apache/iceberg/InheritableMetadataFactory.java |   7 ++
 .../java/org/apache/iceberg/ManifestEntry.java     |  31 ++++-
 .../java/org/apache/iceberg/ManifestWriter.java    |  46 +++++--
 .../main/java/org/apache/iceberg/V1Metadata.java   |  10 ++
 .../main/java/org/apache/iceberg/V2Metadata.java   |  13 ++
 .../java/org/apache/iceberg/TableTestBase.java     |  62 +++++++---
 .../java/org/apache/iceberg/TestFastAppend.java    |   6 +-
 .../org/apache/iceberg/TestManifestWriter.java     | 100 +++++++++++++++
 .../apache/iceberg/TestManifestWriterVersions.java |  42 +++++--
 .../java/org/apache/iceberg/TestMergeAppend.java   | 137 +++++++++++++++------
 .../org/apache/iceberg/TestReplacePartitions.java  |   3 +-
 .../java/org/apache/iceberg/TestRewriteFiles.java  |  39 ++++--
 .../test/java/org/apache/iceberg/TestRowDelta.java |  97 ++++++++++-----
 .../iceberg/TestSequenceNumberForV2Table.java      |  60 +++++----
 .../apache/iceberg/TestV1ToV2RowDeltaDelete.java   |  15 ++-
 format/spec.md                                     |  36 +++---
 .../spark/source/TestIcebergSourceTablesBase.java  |   6 +-
 .../spark/source/TestIcebergSourceTablesBase.java  |   6 +-
 .../spark/source/TestIcebergSourceTablesBase.java  |   6 +-
 .../spark/source/TestIcebergSourceTablesBase.java  |   6 +-
 .../spark/source/TestIcebergSourceTablesBase.java  |   6 +-
 22 files changed, 594 insertions(+), 179 deletions(-)

diff --git a/core/src/main/java/org/apache/iceberg/GenericManifestEntry.java b/core/src/main/java/org/apache/iceberg/GenericManifestEntry.java
index 71545f28cb..45457760d8 100644
--- a/core/src/main/java/org/apache/iceberg/GenericManifestEntry.java
+++ b/core/src/main/java/org/apache/iceberg/GenericManifestEntry.java
@@ -31,6 +31,7 @@ class GenericManifestEntry<F extends ContentFile<F>>
   private Long snapshotId = null;
   private Long dataSequenceNumber = null;
   private Long sequenceNumber = null;
+  private Long fileSequenceNumber = null;
   private F file = null;
 
   GenericManifestEntry(org.apache.avro.Schema schema) {
@@ -47,14 +48,22 @@ class GenericManifestEntry<F extends ContentFile<F>>
     this.snapshotId = toCopy.snapshotId;
     this.dataSequenceNumber = toCopy.dataSequenceNumber;
     this.sequenceNumber = toCopy.sequenceNumber;
+    this.fileSequenceNumber = toCopy.fileSequenceNumber;
     this.file = toCopy.file().copy(fullCopy);
   }
 
-  ManifestEntry<F> wrapExisting(Long newSnapshotId, Long newDataSequenceNumber, F newFile) {
+  ManifestEntry<F> wrapExisting(ManifestEntry<F> entry) {
+    return wrapExisting(
+        entry.snapshotId(), entry.dataSequenceNumber(), entry.fileSequenceNumber(), entry.file());
+  }
+
+  ManifestEntry<F> wrapExisting(
+      Long newSnapshotId, Long newDataSequenceNumber, Long newFileSequenceNumber, F newFile) {
     this.status = Status.EXISTING;
     this.snapshotId = newSnapshotId;
     this.dataSequenceNumber = newDataSequenceNumber;
     this.sequenceNumber = newDataSequenceNumber;
+    this.fileSequenceNumber = newFileSequenceNumber;
     this.file = newFile;
     return this;
   }
@@ -68,15 +77,23 @@ class GenericManifestEntry<F extends ContentFile<F>>
     this.snapshotId = newSnapshotId;
     this.dataSequenceNumber = newDataSequenceNumber;
     this.sequenceNumber = newDataSequenceNumber;
+    this.fileSequenceNumber = null;
     this.file = newFile;
     return this;
   }
 
-  ManifestEntry<F> wrapDelete(Long newSnapshotId, Long newDataSequenceNumber, F newFile) {
+  ManifestEntry<F> wrapDelete(Long newSnapshotId, ManifestEntry<F> entry) {
+    return wrapDelete(
+        newSnapshotId, entry.dataSequenceNumber(), entry.fileSequenceNumber(), entry.file());
+  }
+
+  ManifestEntry<F> wrapDelete(
+      Long newSnapshotId, Long newDataSequenceNumber, Long newFileSequenceNumber, F newFile) {
     this.status = Status.DELETED;
     this.snapshotId = newSnapshotId;
     this.dataSequenceNumber = newDataSequenceNumber;
     this.sequenceNumber = null;
+    this.fileSequenceNumber = newFileSequenceNumber;
     this.file = newFile;
     return this;
   }
@@ -107,6 +124,11 @@ class GenericManifestEntry<F extends ContentFile<F>>
     }
   }
 
+  @Override
+  public Long fileSequenceNumber() {
+    return fileSequenceNumber;
+  }
+
   /** @return a file */
   @Override
   public F file() {
@@ -140,6 +162,11 @@ class GenericManifestEntry<F extends ContentFile<F>>
     this.sequenceNumber = newSequenceNumber;
   }
 
+  @Override
+  public void setFileSequenceNumber(long newFileSequenceNumber) {
+    this.fileSequenceNumber = newFileSequenceNumber;
+  }
+
   @Override
   @SuppressWarnings("unchecked")
   public void put(int i, Object v) {
@@ -156,6 +183,9 @@ class GenericManifestEntry<F extends ContentFile<F>>
         this.sequenceNumber = null;
         return;
       case 3:
+        this.fileSequenceNumber = (Long) v;
+        return;
+      case 4:
         this.file = (F) v;
         return;
       default:
@@ -178,6 +208,8 @@ class GenericManifestEntry<F extends ContentFile<F>>
       case 2:
         return dataSequenceNumber;
       case 3:
+        return fileSequenceNumber;
+      case 4:
         return file;
       default:
         throw new UnsupportedOperationException("Unknown field ordinal: " + i);
@@ -196,7 +228,7 @@ class GenericManifestEntry<F extends ContentFile<F>>
 
   @Override
   public int size() {
-    return 4;
+    return 5;
   }
 
   @Override
@@ -205,6 +237,7 @@ class GenericManifestEntry<F extends ContentFile<F>>
         .add("status", status)
         .add("snapshot_id", snapshotId)
         .add("data_sequence_number", dataSequenceNumber)
+        .add("file_sequence_number", fileSequenceNumber)
         .add("file", file)
         .toString();
   }
diff --git a/core/src/main/java/org/apache/iceberg/InheritableMetadataFactory.java b/core/src/main/java/org/apache/iceberg/InheritableMetadataFactory.java
index 3f48c05b18..789f325d8c 100644
--- a/core/src/main/java/org/apache/iceberg/InheritableMetadataFactory.java
+++ b/core/src/main/java/org/apache/iceberg/InheritableMetadataFactory.java
@@ -75,6 +75,13 @@ class InheritableMetadataFactory {
         manifestEntry.setSequenceNumber(sequenceNumber);
       }
 
+      // in v1 tables, the file sequence number is not persisted and can be safely defaulted to 0
+      // in v2 tables, the file sequence number should be inherited iff the entry status is ADDED
+      if (manifestEntry.fileSequenceNumber() == null
+          && (sequenceNumber == 0 || manifestEntry.status() == ManifestEntry.Status.ADDED)) {
+        manifestEntry.setFileSequenceNumber(sequenceNumber);
+      }
+
       return manifestEntry;
     }
   }
diff --git a/core/src/main/java/org/apache/iceberg/ManifestEntry.java b/core/src/main/java/org/apache/iceberg/ManifestEntry.java
index 6d80c06543..bf319d688f 100644
--- a/core/src/main/java/org/apache/iceberg/ManifestEntry.java
+++ b/core/src/main/java/org/apache/iceberg/ManifestEntry.java
@@ -45,8 +45,10 @@ interface ManifestEntry<F extends ContentFile<F>> {
   Types.NestedField STATUS = required(0, "status", Types.IntegerType.get());
   Types.NestedField SNAPSHOT_ID = optional(1, "snapshot_id", Types.LongType.get());
   Types.NestedField SEQUENCE_NUMBER = optional(3, "sequence_number", Types.LongType.get());
+  Types.NestedField FILE_SEQUENCE_NUMBER =
+      optional(4, "file_sequence_number", Types.LongType.get());
   int DATA_FILE_ID = 2;
-  // next ID to assign: 4
+  // next ID to assign: 5
 
   static Schema getSchema(StructType partitionType) {
     return wrapFileSchema(DataFile.getType(partitionType));
@@ -54,7 +56,11 @@ interface ManifestEntry<F extends ContentFile<F>> {
 
   static Schema wrapFileSchema(StructType fileType) {
     return new Schema(
-        STATUS, SNAPSHOT_ID, SEQUENCE_NUMBER, required(DATA_FILE_ID, "data_file", fileType));
+        STATUS,
+        SNAPSHOT_ID,
+        SEQUENCE_NUMBER,
+        FILE_SEQUENCE_NUMBER,
+        required(DATA_FILE_ID, "data_file", fileType));
   }
 
   /** Returns the status of the file, whether EXISTING, ADDED, or DELETED. */
@@ -118,6 +124,27 @@ interface ManifestEntry<F extends ContentFile<F>> {
   @Deprecated
   void setSequenceNumber(long sequenceNumber);
 
+  /**
+   * Returns the file sequence number.
+   *
+   * <p>The file sequence number represents the sequence number of the snapshot in which the
+   * underlying file was added. The file sequence number is always assigned at commit and cannot be
+   * provided explicitly, unlike the data sequence number. The file sequence number does not change
+   * upon assigning and must be preserved in existing and deleted entries.
+   *
+   * <p>This method can return null if the file sequence number is unknown. This may happen while
+   * reading a v2 manifest that did not persist the file sequence number for manifest entries with
+   * status EXISTING or DELETED (older Iceberg versions).
+   */
+  Long fileSequenceNumber();
+
+  /**
+   * Sets the file sequence number for this manifest entry.
+   *
+   * @param fileSequenceNumber a file sequence number
+   */
+  void setFileSequenceNumber(long fileSequenceNumber);
+
   /** Returns a file. */
   F file();
 
diff --git a/core/src/main/java/org/apache/iceberg/ManifestWriter.java b/core/src/main/java/org/apache/iceberg/ManifestWriter.java
index bfaa9e9cc3..52cf1a6261 100644
--- a/core/src/main/java/org/apache/iceberg/ManifestWriter.java
+++ b/core/src/main/java/org/apache/iceberg/ManifestWriter.java
@@ -100,7 +100,8 @@ public abstract class ManifestWriter<F extends ContentFile<F>> implements FileAp
   /**
    * Add an added entry for a file.
    *
-   * <p>The entry's snapshot ID will be this manifest's snapshot ID.
+   * <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
    */
@@ -112,8 +113,9 @@ public abstract class ManifestWriter<F extends ContentFile<F>> implements FileAp
   /**
    * 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 sequence number
-   * will be the provided 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
@@ -139,13 +141,33 @@ public abstract class ManifestWriter<F extends ContentFile<F>> implements FileAp
    * @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)
+   * @deprecated since 1.1.0, will be removed in 1.2.0; use {@link #existing(ContentFile, long,
+   *     long, Long)}.
    */
+  @Deprecated
   public void existing(F existingFile, long fileSnapshotId, long dataSequenceNumber) {
-    addEntry(reused.wrapExisting(fileSnapshotId, dataSequenceNumber, existingFile));
+    addEntry(reused.wrapExisting(fileSnapshotId, dataSequenceNumber, null, existingFile));
+  }
+
+  /**
+   * 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) {
+    reused.wrapExisting(fileSnapshotId, dataSequenceNumber, fileSequenceNumber, existingFile);
+    addEntry(reused);
   }
 
   void existing(ManifestEntry<F> entry) {
-    addEntry(reused.wrapExisting(entry.snapshotId(), entry.dataSequenceNumber(), entry.file()));
+    addEntry(reused.wrapExisting(entry));
   }
 
   /**
@@ -155,7 +177,8 @@ public abstract class ManifestWriter<F extends ContentFile<F>> implements FileAp
    * preserved when the file is marked as deleted.
    *
    * @param deletedFile a file
-   * @deprecated since 1.0.0, will be removed in 1.1.0; use {@link #delete(ContentFile, long)}.
+   * @deprecated since 1.1.0, will be removed in 1.2.0; use {@link #delete(ContentFile, long,
+   *     Long)}.
    */
   @Deprecated
   public void delete(F deletedFile) {
@@ -166,20 +189,21 @@ public abstract class ManifestWriter<F extends ContentFile<F>> implements FileAp
   /**
    * Add a delete entry for a file.
    *
-   * <p>The entry's snapshot ID will be this manifest's snapshot ID. However, the original data
-   * sequence number of the file must be preserved when the file is marked as deleted.
+   * <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) {
-    addEntry(reused.wrapDelete(snapshotId, dataSequenceNumber, deletedFile));
+  public void delete(F deletedFile, long dataSequenceNumber, Long fileSequenceNumber) {
+    addEntry(reused.wrapDelete(snapshotId, dataSequenceNumber, fileSequenceNumber, deletedFile));
   }
 
   void delete(ManifestEntry<F> entry) {
     // Use the current Snapshot ID for the delete. It is safe to delete the data file from disk
     // when this Snapshot has been removed or when there are no Snapshots older than this one.
-    addEntry(reused.wrapDelete(snapshotId, entry.dataSequenceNumber(), entry.file()));
+    addEntry(reused.wrapDelete(snapshotId, entry));
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/iceberg/V1Metadata.java b/core/src/main/java/org/apache/iceberg/V1Metadata.java
index 5985a172c6..d45d7770fe 100644
--- a/core/src/main/java/org/apache/iceberg/V1Metadata.java
+++ b/core/src/main/java/org/apache/iceberg/V1Metadata.java
@@ -305,6 +305,16 @@ class V1Metadata {
       wrapped.setSequenceNumber(sequenceNumber);
     }
 
+    @Override
+    public Long fileSequenceNumber() {
+      return wrapped.fileSequenceNumber();
+    }
+
+    @Override
+    public void setFileSequenceNumber(long fileSequenceNumber) {
+      wrapped.setFileSequenceNumber(fileSequenceNumber);
+    }
+
     @Override
     public DataFile file() {
       return wrapped.file();
diff --git a/core/src/main/java/org/apache/iceberg/V2Metadata.java b/core/src/main/java/org/apache/iceberg/V2Metadata.java
index 9aefcab41d..64ab0fe94b 100644
--- a/core/src/main/java/org/apache/iceberg/V2Metadata.java
+++ b/core/src/main/java/org/apache/iceberg/V2Metadata.java
@@ -250,6 +250,7 @@ class V2Metadata {
         ManifestEntry.STATUS,
         ManifestEntry.SNAPSHOT_ID,
         ManifestEntry.SEQUENCE_NUMBER,
+        ManifestEntry.FILE_SEQUENCE_NUMBER,
         required(ManifestEntry.DATA_FILE_ID, "data_file", fileSchema));
   }
 
@@ -329,6 +330,8 @@ class V2Metadata {
           }
           return wrapped.dataSequenceNumber();
         case 3:
+          return wrapped.fileSequenceNumber();
+        case 4:
           return fileWrapper.wrap(wrapped.file());
         default:
           throw new UnsupportedOperationException("Unknown field ordinal: " + i);
@@ -370,6 +373,16 @@ class V2Metadata {
       wrapped.setSequenceNumber(sequenceNumber);
     }
 
+    @Override
+    public Long fileSequenceNumber() {
+      return wrapped.fileSequenceNumber();
+    }
+
+    @Override
+    public void setFileSequenceNumber(long fileSequenceNumber) {
+      wrapped.setFileSequenceNumber(fileSequenceNumber);
+    }
+
     @Override
     public F file() {
       return wrapped.file();
diff --git a/core/src/test/java/org/apache/iceberg/TableTestBase.java b/core/src/test/java/org/apache/iceberg/TableTestBase.java
index 5cc35279ad..4828c2c0c6 100644
--- a/core/src/test/java/org/apache/iceberg/TableTestBase.java
+++ b/core/src/test/java/org/apache/iceberg/TableTestBase.java
@@ -30,7 +30,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
 import java.util.UUID;
-import java.util.stream.LongStream;
 import org.apache.iceberg.deletes.PositionDelete;
 import org.apache.iceberg.io.FileIO;
 import org.apache.iceberg.io.OutputFile;
@@ -328,14 +327,28 @@ public class TableTestBase {
 
   ManifestEntry<DataFile> manifestEntry(
       ManifestEntry.Status status, Long snapshotId, DataFile file) {
+    return manifestEntry(status, snapshotId, 0L, 0L, file);
+  }
+
+  ManifestEntry<DataFile> manifestEntry(
+      ManifestEntry.Status status,
+      Long snapshotId,
+      Long dataSequenceNumber,
+      Long fileSequenceNumber,
+      DataFile file) {
+
     GenericManifestEntry<DataFile> entry = new GenericManifestEntry<>(table.spec().partitionType());
     switch (status) {
       case ADDED:
-        return entry.wrapAppend(snapshotId, file);
+        if (dataSequenceNumber != null && dataSequenceNumber != 0) {
+          return entry.wrapAppend(snapshotId, dataSequenceNumber, file);
+        } else {
+          return entry.wrapAppend(snapshotId, file);
+        }
       case EXISTING:
-        return entry.wrapExisting(snapshotId, 0L, file);
+        return entry.wrapExisting(snapshotId, dataSequenceNumber, fileSequenceNumber, file);
       case DELETED:
-        return entry.wrapDelete(snapshotId, 0L, file);
+        return entry.wrapDelete(snapshotId, dataSequenceNumber, fileSequenceNumber, file);
       default:
         throw new IllegalArgumentException("Unexpected entry status: " + status);
     }
@@ -476,33 +489,35 @@ public class TableTestBase {
 
   void validateManifest(
       ManifestFile manifest, Iterator<Long> ids, Iterator<DataFile> expectedFiles) {
-    validateManifest(manifest, null, ids, expectedFiles, null);
+    validateManifest(manifest, null, null, ids, expectedFiles, null);
   }
 
   void validateManifest(
       ManifestFile manifest,
-      Iterator<Long> seqs,
+      Iterator<Long> dataSeqs,
+      Iterator<Long> fileSeqs,
       Iterator<Long> ids,
       Iterator<DataFile> expectedFiles) {
-    validateManifest(manifest, seqs, ids, expectedFiles, null);
+    validateManifest(manifest, dataSeqs, fileSeqs, ids, expectedFiles, null);
   }
 
   void validateManifest(
       ManifestFile manifest,
-      Iterator<Long> seqs,
+      Iterator<Long> dataSeqs,
+      Iterator<Long> fileSeqs,
       Iterator<Long> ids,
       Iterator<DataFile> expectedFiles,
       Iterator<ManifestEntry.Status> statuses) {
     for (ManifestEntry<DataFile> entry : ManifestFiles.read(manifest, FILE_IO).entries()) {
       DataFile file = entry.file();
       DataFile expected = expectedFiles.next();
-      if (seqs != null) {
+      if (dataSeqs != null) {
         V1Assert.assertEquals(
             "Data sequence number should default to 0", 0, entry.dataSequenceNumber().longValue());
         V1Assert.assertEquals(
             "Sequence number should default to 0", 0, entry.sequenceNumber().longValue());
 
-        Long expectedSequenceNumber = seqs.next();
+        Long expectedSequenceNumber = dataSeqs.next();
         V2Assert.assertEquals(
             "Data sequence number should match expected",
             expectedSequenceNumber,
@@ -519,6 +534,12 @@ public class TableTestBase {
               entry.sequenceNumber().longValue());
         }
       }
+      if (fileSeqs != null) {
+        V1Assert.assertEquals(
+            "File sequence number should default to 0", (Long) 0L, entry.fileSequenceNumber());
+        V2Assert.assertEquals(
+            "File sequence number should match", fileSeqs.next(), entry.fileSequenceNumber());
+      }
       Assert.assertEquals(
           "Path should match expected", expected.path().toString(), file.path().toString());
       Assert.assertEquals("Snapshot ID should match expected ID", ids.next(), entry.snapshotId());
@@ -532,7 +553,8 @@ public class TableTestBase {
 
   void validateDeleteManifest(
       ManifestFile manifest,
-      Iterator<Long> seqs,
+      Iterator<Long> dataSeqs,
+      Iterator<Long> fileSeqs,
       Iterator<Long> ids,
       Iterator<DeleteFile> expectedFiles,
       Iterator<ManifestEntry.Status> statuses) {
@@ -540,13 +562,13 @@ public class TableTestBase {
         ManifestFiles.readDeleteManifest(manifest, FILE_IO, null).entries()) {
       DeleteFile file = entry.file();
       DeleteFile expected = expectedFiles.next();
-      if (seqs != null) {
+      if (dataSeqs != null) {
         V1Assert.assertEquals(
             "Data sequence number should default to 0", 0, entry.dataSequenceNumber().longValue());
         V1Assert.assertEquals(
             "Sequence number should default to 0", 0, entry.sequenceNumber().longValue());
 
-        Long expectedSequenceNumber = seqs.next();
+        Long expectedSequenceNumber = dataSeqs.next();
         V2Assert.assertEquals(
             "Data sequence number should match expected",
             expectedSequenceNumber,
@@ -563,6 +585,12 @@ public class TableTestBase {
               entry.sequenceNumber().longValue());
         }
       }
+      if (fileSeqs != null) {
+        V1Assert.assertEquals(
+            "File sequence number should default to 0", 0, entry.fileSequenceNumber().longValue());
+        V2Assert.assertEquals(
+            "File sequence number should match", fileSeqs.next(), entry.fileSequenceNumber());
+      }
       Assert.assertEquals(
           "Path should match expected", expected.path().toString(), file.path().toString());
       Assert.assertEquals("Snapshot ID should match expected ID", ids.next(), entry.snapshotId());
@@ -655,8 +683,12 @@ public class TableTestBase {
     return Iterators.forArray(statuses);
   }
 
-  static Iterator<Long> seqs(long... seqs) {
-    return LongStream.of(seqs).iterator();
+  static Iterator<Long> dataSeqs(Long... seqs) {
+    return Iterators.forArray(seqs);
+  }
+
+  static Iterator<Long> fileSeqs(Long... seqs) {
+    return Iterators.forArray(seqs);
   }
 
   static Iterator<Long> ids(Long... ids) {
diff --git a/core/src/test/java/org/apache/iceberg/TestFastAppend.java b/core/src/test/java/org/apache/iceberg/TestFastAppend.java
index 508c90255e..ecc673f530 100644
--- a/core/src/test/java/org/apache/iceberg/TestFastAppend.java
+++ b/core/src/test/java/org/apache/iceberg/TestFastAppend.java
@@ -114,12 +114,14 @@ public class TestFastAppend extends TableTestBase {
 
     validateManifest(
         snap.allManifests(FILE_IO).get(0),
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(commitId, commitId),
         files(FILE_C, FILE_D));
     validateManifest(
         snap.allManifests(FILE_IO).get(1),
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(commitId, commitId),
         files(FILE_A, FILE_B));
 
diff --git a/core/src/test/java/org/apache/iceberg/TestManifestWriter.java b/core/src/test/java/org/apache/iceberg/TestManifestWriter.java
index 344348c43f..ac5bc521b5 100644
--- a/core/src/test/java/org/apache/iceberg/TestManifestWriter.java
+++ b/core/src/test/java/org/apache/iceberg/TestManifestWriter.java
@@ -119,9 +119,109 @@ public class TestManifestWriter extends TableTestBase {
           "Custom sequence number should be used for all manifest entries",
           1000L,
           (long) entry.sequenceNumber());
+      Assert.assertEquals(
+          "File sequence number must be unassigned",
+          ManifestWriter.UNASSIGNED_SEQ,
+          entry.fileSequenceNumber().longValue());
     }
   }
 
+  @Test
+  public void testCommitManifestWithExplicitDataSequenceNumber() throws IOException {
+    Assume.assumeTrue("Sequence numbers are valid for format version > 1", formatVersion > 1);
+
+    DataFile file1 = newFile(50);
+    DataFile file2 = newFile(50);
+
+    long dataSequenceNumber = 25L;
+
+    ManifestFile manifest =
+        writeManifest(
+            "manifest.avro",
+            manifestEntry(Status.ADDED, null, dataSequenceNumber, null, file1),
+            manifestEntry(Status.ADDED, null, dataSequenceNumber, null, file2));
+
+    Assert.assertEquals(
+        "Manifest should have no sequence number before commit",
+        ManifestWriter.UNASSIGNED_SEQ,
+        manifest.sequenceNumber());
+
+    table.newFastAppend().appendManifest(manifest).commit();
+
+    long commitSnapshotId = table.currentSnapshot().snapshotId();
+
+    ManifestFile committedManifest = table.currentSnapshot().dataManifests(table.io()).get(0);
+
+    Assert.assertEquals(
+        "Committed manifest sequence number must be correct",
+        1L,
+        committedManifest.sequenceNumber());
+
+    Assert.assertEquals(
+        "Committed manifest min sequence number must be correct",
+        dataSequenceNumber,
+        committedManifest.minSequenceNumber());
+
+    validateManifest(
+        committedManifest,
+        dataSeqs(dataSequenceNumber, dataSequenceNumber),
+        fileSeqs(1L, 1L),
+        ids(commitSnapshotId, commitSnapshotId),
+        files(file1, file2),
+        statuses(Status.ADDED, Status.ADDED));
+  }
+
+  @Test
+  public void testCommitManifestWithExistingEntriesWithoutFileSequenceNumber() throws IOException {
+    Assume.assumeTrue("Sequence numbers are valid for format version > 1", formatVersion > 1);
+
+    DataFile file1 = newFile(50);
+    DataFile file2 = newFile(50);
+
+    table.newFastAppend().appendFile(file1).appendFile(file2).commit();
+
+    Snapshot appendSnapshot = table.currentSnapshot();
+    long appendSequenceNumber = appendSnapshot.sequenceNumber();
+    long appendSnapshotId = appendSnapshot.snapshotId();
+
+    ManifestFile originalManifest = appendSnapshot.dataManifests(table.io()).get(0);
+
+    ManifestFile newManifest =
+        writeManifest(
+            "manifest.avro",
+            manifestEntry(Status.EXISTING, appendSnapshotId, appendSequenceNumber, null, file1),
+            manifestEntry(Status.EXISTING, appendSnapshotId, appendSequenceNumber, null, file2));
+
+    Assert.assertEquals(
+        "Manifest should have no sequence number before commit",
+        ManifestWriter.UNASSIGNED_SEQ,
+        newManifest.sequenceNumber());
+
+    table.rewriteManifests().deleteManifest(originalManifest).addManifest(newManifest).commit();
+
+    Snapshot rewriteSnapshot = table.currentSnapshot();
+
+    ManifestFile committedManifest = table.currentSnapshot().dataManifests(table.io()).get(0);
+
+    Assert.assertEquals(
+        "Committed manifest sequence number must be correct",
+        rewriteSnapshot.sequenceNumber(),
+        committedManifest.sequenceNumber());
+
+    Assert.assertEquals(
+        "Committed manifest min sequence number must be correct",
+        appendSequenceNumber,
+        committedManifest.minSequenceNumber());
+
+    validateManifest(
+        committedManifest,
+        dataSeqs(appendSequenceNumber, appendSequenceNumber),
+        fileSeqs(null, null),
+        ids(appendSnapshotId, appendSnapshotId),
+        files(file1, file2),
+        statuses(Status.EXISTING, Status.EXISTING));
+  }
+
   private DataFile newFile(long recordCount) {
     return newFile(recordCount, null);
   }
diff --git a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java
index b5ca55e992..8f9cff0196 100644
--- a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java
+++ b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java
@@ -101,7 +101,11 @@ public class TestManifestWriterVersions {
   public void testV1Write() throws IOException {
     ManifestFile manifest = writeManifest(1);
     checkManifest(manifest, ManifestWriter.UNASSIGNED_SEQ);
-    checkEntry(readManifest(manifest), ManifestWriter.UNASSIGNED_SEQ, FileContent.DATA);
+    checkEntry(
+        readManifest(manifest),
+        ManifestWriter.UNASSIGNED_SEQ,
+        ManifestWriter.UNASSIGNED_SEQ,
+        FileContent.DATA);
   }
 
   @Test
@@ -119,7 +123,7 @@ public class TestManifestWriterVersions {
     checkManifest(manifest, 0L);
 
     // v1 should be read using sequence number 0 because it was missing from the manifest list file
-    checkEntry(readManifest(manifest), 0L, FileContent.DATA);
+    checkEntry(readManifest(manifest), 0L, 0L, FileContent.DATA);
   }
 
   @Test
@@ -127,7 +131,11 @@ public class TestManifestWriterVersions {
     ManifestFile manifest = writeManifest(2);
     checkManifest(manifest, ManifestWriter.UNASSIGNED_SEQ);
     Assert.assertEquals("Content", ManifestContent.DATA, manifest.content());
-    checkEntry(readManifest(manifest), ManifestWriter.UNASSIGNED_SEQ, FileContent.DATA);
+    checkEntry(
+        readManifest(manifest),
+        ManifestWriter.UNASSIGNED_SEQ,
+        ManifestWriter.UNASSIGNED_SEQ,
+        FileContent.DATA);
   }
 
   @Test
@@ -137,7 +145,7 @@ public class TestManifestWriterVersions {
     Assert.assertEquals("Content", ManifestContent.DATA, manifest.content());
 
     // v2 should use the correct sequence number by inheriting it
-    checkEntry(readManifest(manifest), SEQUENCE_NUMBER, FileContent.DATA);
+    checkEntry(readManifest(manifest), SEQUENCE_NUMBER, SEQUENCE_NUMBER, FileContent.DATA);
   }
 
   @Test
@@ -146,7 +154,10 @@ public class TestManifestWriterVersions {
     checkManifest(manifest, ManifestWriter.UNASSIGNED_SEQ);
     Assert.assertEquals("Content", ManifestContent.DELETES, manifest.content());
     checkEntry(
-        readDeleteManifest(manifest), ManifestWriter.UNASSIGNED_SEQ, FileContent.EQUALITY_DELETES);
+        readDeleteManifest(manifest),
+        ManifestWriter.UNASSIGNED_SEQ,
+        ManifestWriter.UNASSIGNED_SEQ,
+        FileContent.EQUALITY_DELETES);
   }
 
   @Test
@@ -156,7 +167,11 @@ public class TestManifestWriterVersions {
     Assert.assertEquals("Content", ManifestContent.DELETES, manifest.content());
 
     // v2 should use the correct sequence number by inheriting it
-    checkEntry(readDeleteManifest(manifest), SEQUENCE_NUMBER, FileContent.EQUALITY_DELETES);
+    checkEntry(
+        readDeleteManifest(manifest),
+        SEQUENCE_NUMBER,
+        SEQUENCE_NUMBER,
+        FileContent.EQUALITY_DELETES);
   }
 
   @Test
@@ -171,7 +186,7 @@ public class TestManifestWriterVersions {
     checkManifest(manifest2, 0L);
 
     // should not inherit the v2 sequence number because it was a rewrite
-    checkEntry(readManifest(manifest2), 0L, FileContent.DATA);
+    checkEntry(readManifest(manifest2), 0L, 0L, FileContent.DATA);
   }
 
   @Test
@@ -194,11 +209,18 @@ public class TestManifestWriterVersions {
     checkRewrittenEntry(readManifest(manifest2), 0L, FileContent.DATA);
   }
 
-  void checkEntry(ManifestEntry<?> entry, Long expectedSequenceNumber, FileContent content) {
+  void checkEntry(
+      ManifestEntry<?> entry,
+      Long expectedDataSequenceNumber,
+      Long expectedFileSequenceNumber,
+      FileContent content) {
     Assert.assertEquals("Status", ManifestEntry.Status.ADDED, entry.status());
     Assert.assertEquals("Snapshot ID", (Long) SNAPSHOT_ID, entry.snapshotId());
-    Assert.assertEquals("Data sequence number", expectedSequenceNumber, entry.dataSequenceNumber());
-    Assert.assertEquals("Sequence number", expectedSequenceNumber, entry.sequenceNumber());
+    Assert.assertEquals(
+        "Data sequence number", expectedDataSequenceNumber, entry.dataSequenceNumber());
+    Assert.assertEquals("Sequence number", expectedDataSequenceNumber, entry.sequenceNumber());
+    Assert.assertEquals(
+        "File sequence number", expectedFileSequenceNumber, entry.fileSequenceNumber());
     checkDataFile(entry.file(), content);
   }
 
diff --git a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java
index 8e9d4ab136..212bab3c13 100644
--- a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java
+++ b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java
@@ -82,7 +82,8 @@ public class TestMergeAppend extends TableTestBase {
 
     validateManifest(
         committedSnapshot.allManifests(table.io()).get(0),
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(snapshotId, snapshotId),
         files(FILE_A, FILE_B),
         statuses(Status.ADDED, Status.ADDED));
@@ -112,7 +113,8 @@ public class TestMergeAppend extends TableTestBase {
     long snapshotId = committedSnapshot.snapshotId();
     validateManifest(
         committedSnapshot.allManifests(table.io()).get(0),
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(snapshotId, snapshotId),
         files(FILE_A, FILE_B),
         statuses(Status.ADDED, Status.ADDED));
@@ -153,14 +155,16 @@ public class TestMergeAppend extends TableTestBase {
 
     validateManifest(
         committedSnapshot.allManifests(table.io()).get(0),
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(snapshotId, snapshotId),
         files(FILE_C, FILE_D),
         statuses(Status.ADDED, Status.ADDED));
 
     validateManifest(
         committedSnapshot.allManifests(table.io()).get(1),
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(snapshotId, snapshotId),
         files(FILE_A, FILE_B),
         statuses(Status.ADDED, Status.ADDED));
@@ -228,7 +232,8 @@ public class TestMergeAppend extends TableTestBase {
 
     validateManifest(
         committedSnapshot.allManifests(table.io()).get(0),
-        seqs(1, 1, 1, 1),
+        dataSeqs(1L, 1L, 1L, 1L),
+        fileSeqs(1L, 1L, 1L, 1L),
         ids(snapshotId, snapshotId, snapshotId, snapshotId),
         files(FILE_C, FILE_D, FILE_A, FILE_B),
         statuses(Status.ADDED, Status.ADDED, Status.ADDED, Status.ADDED));
@@ -261,7 +266,8 @@ public class TestMergeAppend extends TableTestBase {
     ManifestFile initialManifest = commitBefore.allManifests(table.io()).get(0);
     validateManifest(
         initialManifest,
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(baseId, baseId),
         files(FILE_A, FILE_B),
         statuses(Status.ADDED, Status.ADDED));
@@ -285,7 +291,8 @@ public class TestMergeAppend extends TableTestBase {
 
     validateManifest(
         newManifest,
-        seqs(2, 2, 1, 1),
+        dataSeqs(2L, 2L, 1L, 1L),
+        fileSeqs(2L, 2L, 1L, 1L),
         ids(snapshotId, snapshotId, baseId, baseId),
         concat(files(FILE_C, FILE_D), files(initialManifest)),
         statuses(Status.ADDED, Status.ADDED, Status.EXISTING, Status.EXISTING));
@@ -333,13 +340,15 @@ public class TestMergeAppend extends TableTestBase {
         snap1.allManifests(table.io()).size());
     validateManifest(
         snap1.allManifests(table.io()).get(0),
-        seqs(1),
+        dataSeqs(1L),
+        fileSeqs(1L),
         ids(commitId1),
         files(FILE_A),
         statuses(Status.ADDED));
     validateManifest(
         snap1.allManifests(table.io()).get(1),
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(commitId1, commitId1),
         files(FILE_C, FILE_D),
         statuses(Status.ADDED, Status.ADDED));
@@ -366,19 +375,22 @@ public class TestMergeAppend extends TableTestBase {
         snap2.allManifests(table.io()).size());
     validateManifest(
         snap2.allManifests(table.io()).get(0),
-        seqs(2),
+        dataSeqs(2L),
+        fileSeqs(2L),
         ids(commitId2),
         files(FILE_A),
         statuses(Status.ADDED));
     validateManifest(
         snap2.allManifests(table.io()).get(1),
-        seqs(2, 2),
+        dataSeqs(2L, 2L),
+        fileSeqs(2L, 2L),
         ids(commitId2, commitId2),
         files(FILE_C, FILE_D),
         statuses(Status.ADDED, Status.ADDED));
     validateManifest(
         snap2.allManifests(table.io()).get(2),
-        seqs(1, 1, 1),
+        dataSeqs(1L, 1L, 1L),
+        fileSeqs(1L, 1L, 1L),
         ids(commitId1, commitId1, commitId1),
         files(FILE_A, FILE_C, FILE_D),
         statuses(Status.EXISTING, Status.EXISTING, Status.EXISTING));
@@ -404,7 +416,8 @@ public class TestMergeAppend extends TableTestBase {
     Assert.assertEquals("Should contain 1 manifest", 1, snap1.allManifests(table.io()).size());
     validateManifest(
         snap1.allManifests(table.io()).get(0),
-        seqs(1),
+        dataSeqs(1L),
+        fileSeqs(1L),
         ids(commitId1),
         files(FILE_A),
         statuses(Status.ADDED));
@@ -420,13 +433,15 @@ public class TestMergeAppend extends TableTestBase {
     Assert.assertEquals("Should contain 2 manifests", 2, snap2.allManifests(table.io()).size());
     validateManifest(
         snap2.allManifests(table.io()).get(0),
-        seqs(2),
+        dataSeqs(2L),
+        fileSeqs(2L),
         ids(commitId2),
         files(FILE_B),
         statuses(Status.ADDED));
     validateManifest(
         snap2.allManifests(table.io()).get(1),
-        seqs(1),
+        dataSeqs(1L),
+        fileSeqs(1L),
         ids(commitId1),
         files(FILE_A),
         statuses(Status.ADDED));
@@ -451,19 +466,22 @@ public class TestMergeAppend extends TableTestBase {
     long commitId3 = snap3.snapshotId();
     validateManifest(
         snap3.allManifests(table.io()).get(0),
-        seqs(3),
+        dataSeqs(3L),
+        fileSeqs(3L),
         ids(commitId3),
         files(FILE_C),
         statuses(Status.ADDED));
     validateManifest(
         snap3.allManifests(table.io()).get(1),
-        seqs(2),
+        dataSeqs(2L),
+        fileSeqs(2L),
         ids(commitId2),
         files(FILE_B),
         statuses(Status.ADDED));
     validateManifest(
         snap3.allManifests(table.io()).get(2),
-        seqs(1),
+        dataSeqs(1L),
+        fileSeqs(1L),
         ids(commitId1),
         files(FILE_A),
         statuses(Status.ADDED));
@@ -491,7 +509,8 @@ public class TestMergeAppend extends TableTestBase {
         "Should only contains 1 merged manifest", 1, snap4.allManifests(table.io()).size());
     validateManifest(
         snap4.allManifests(table.io()).get(0),
-        seqs(4, 3, 2, 1),
+        dataSeqs(4L, 3L, 2L, 1L),
+        fileSeqs(4L, 3L, 2L, 1L),
         ids(commitId4, commitId3, commitId2, commitId1),
         files(FILE_D, FILE_C, FILE_B, FILE_A),
         statuses(Status.ADDED, Status.EXISTING, Status.EXISTING, Status.EXISTING));
@@ -534,19 +553,22 @@ public class TestMergeAppend extends TableTestBase {
 
     validateManifest(
         committed.allManifests(table.io()).get(0),
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(snapshotId, snapshotId),
         files(FILE_A, FILE_B),
         statuses(Status.ADDED, Status.ADDED));
     validateManifest(
         committed.allManifests(table.io()).get(1),
-        seqs(1),
+        dataSeqs(1L),
+        fileSeqs(1L),
         ids(snapshotId),
         files(FILE_C),
         statuses(Status.ADDED));
     validateManifest(
         committed.allManifests(table.io()).get(2),
-        seqs(1),
+        dataSeqs(1L),
+        fileSeqs(1L),
         ids(snapshotId),
         files(FILE_D),
         statuses(Status.ADDED));
@@ -577,7 +599,8 @@ public class TestMergeAppend extends TableTestBase {
     ManifestFile initialManifest = snap.allManifests(table.io()).get(0);
     validateManifest(
         initialManifest,
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(baseId, baseId),
         files(FILE_A, FILE_B),
         statuses(Status.ADDED, Status.ADDED));
@@ -601,7 +624,8 @@ public class TestMergeAppend extends TableTestBase {
 
     validateManifest(
         deleteManifest,
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(deleteId, baseId),
         files(FILE_A, FILE_B),
         statuses(Status.DELETED, Status.EXISTING));
@@ -683,7 +707,8 @@ public class TestMergeAppend extends TableTestBase {
 
     validateManifest(
         newManifest,
-        seqs(4, 3, 2, 1),
+        dataSeqs(4L, 3L, 2L, 1L),
+        fileSeqs(4L, 3L, 2L, 1L),
         ids(lastSnapshotId, idFileC, idFileB, idFileA),
         files(FILE_D, FILE_C, FILE_B, FILE_A),
         statuses(Status.ADDED, Status.EXISTING, Status.EXISTING, Status.EXISTING));
@@ -708,7 +733,8 @@ public class TestMergeAppend extends TableTestBase {
     ManifestFile initialManifest = snap.allManifests(table.io()).get(0);
     validateManifest(
         initialManifest,
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(baseId, baseId),
         files(FILE_A, FILE_B),
         statuses(Status.ADDED, Status.ADDED));
@@ -733,14 +759,16 @@ public class TestMergeAppend extends TableTestBase {
     long pendingId = committed.snapshotId();
     validateManifest(
         newManifest,
-        seqs(2, 2),
+        dataSeqs(2L, 2L),
+        fileSeqs(2L, 2L),
         ids(pendingId, pendingId),
         files(FILE_C, FILE_D),
         statuses(Status.ADDED, Status.ADDED));
 
     validateManifest(
         committed.allManifests(table.io()).get(1),
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(baseId, baseId),
         files(initialManifest),
         statuses(Status.ADDED, Status.ADDED));
@@ -759,7 +787,8 @@ public class TestMergeAppend extends TableTestBase {
     ManifestFile initialManifest = snap.allManifests(table.io()).get(0);
     validateManifest(
         initialManifest,
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(commitId, commitId),
         files(FILE_A, FILE_B),
         statuses(Status.ADDED, Status.ADDED));
@@ -799,7 +828,8 @@ public class TestMergeAppend extends TableTestBase {
     // new manifest comes first
     validateManifest(
         lastSnapshot.allManifests(table.io()).get(0),
-        seqs(2),
+        dataSeqs(2L),
+        fileSeqs(2L),
         ids(lastSnapshot.snapshotId()),
         files(newFileY),
         statuses(Status.ADDED));
@@ -863,13 +893,15 @@ public class TestMergeAppend extends TableTestBase {
 
     validateManifest(
         lastSnapshot.allManifests(table.io()).get(0),
-        seqs(3),
+        dataSeqs(3L),
+        fileSeqs(3L),
         ids(lastSnapshot.snapshotId()),
         files(newFileY),
         statuses(Status.ADDED));
     validateManifest(
         lastSnapshot.allManifests(table.io()).get(1),
-        seqs(2, 1),
+        dataSeqs(2L, 1L),
+        fileSeqs(2L, 1L),
         ids(id2, id1),
         files(FILE_B, FILE_A),
         statuses(Status.EXISTING, Status.EXISTING));
@@ -890,7 +922,13 @@ public class TestMergeAppend extends TableTestBase {
         "Table should end with last-sequence-number 0", 0, base.lastSequenceNumber());
 
     ManifestFile initialManifest = snap.allManifests(table.io()).get(0);
-    validateManifest(initialManifest, seqs(1), ids(baseId), files(FILE_A), statuses(Status.ADDED));
+    validateManifest(
+        initialManifest,
+        dataSeqs(1L),
+        fileSeqs(1L),
+        ids(baseId),
+        files(FILE_A),
+        statuses(Status.ADDED));
 
     table.ops().failCommits(5);
 
@@ -923,7 +961,8 @@ public class TestMergeAppend extends TableTestBase {
 
     validateManifest(
         latestSnapshot(table, branch).allManifests(table.io()).get(0),
-        seqs(1),
+        dataSeqs(1L),
+        fileSeqs(1L),
         ids(baseId),
         files(initialManifest),
         statuses(Status.ADDED));
@@ -972,7 +1011,13 @@ public class TestMergeAppend extends TableTestBase {
     V1Assert.assertEquals(
         "Table should end with last-sequence-number 0", 0, readMetadata().lastSequenceNumber());
     ManifestFile initialManifest = current.allManifests(table.io()).get(0);
-    validateManifest(initialManifest, seqs(1), ids(baseId), files(FILE_A), statuses(Status.ADDED));
+    validateManifest(
+        initialManifest,
+        dataSeqs(1L),
+        fileSeqs(1L),
+        ids(baseId),
+        files(FILE_A),
+        statuses(Status.ADDED));
 
     table.ops().failCommits(3);
 
@@ -1015,7 +1060,8 @@ public class TestMergeAppend extends TableTestBase {
     ManifestFile manifestFile = snapshot.allManifests(table.io()).get(0);
     validateManifest(
         manifestFile,
-        seqs(2, 1),
+        dataSeqs(2L, 1L),
+        fileSeqs(2L, 1L),
         ids(snapshotId, baseId),
         files(FILE_B, FILE_A),
         statuses(Status.ADDED, Status.EXISTING));
@@ -1042,7 +1088,8 @@ public class TestMergeAppend extends TableTestBase {
     ManifestFile manifestFile = snapshot.allManifests(table.io()).get(0);
     validateManifest(
         manifestFile,
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(snapshotId, snapshotId),
         files(FILE_A, FILE_B),
         statuses(Status.ADDED, Status.ADDED));
@@ -1087,7 +1134,8 @@ public class TestMergeAppend extends TableTestBase {
     Assert.assertEquals("Should have only 1 manifest", 1, snap1.allManifests(table.io()).size());
     validateManifest(
         snap1.allManifests(table.io()).get(0),
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(commitId1, commitId1),
         files(FILE_A, FILE_B),
         statuses(Status.ADDED, Status.ADDED));
@@ -1108,7 +1156,8 @@ public class TestMergeAppend extends TableTestBase {
         "Manifests should be merged into 1", 1, snap2.allManifests(table.io()).size());
     validateManifest(
         latestSnapshot(table, branch).allManifests(table.io()).get(0),
-        seqs(2, 2, 1, 1),
+        dataSeqs(2L, 2L, 1L, 1L),
+        fileSeqs(2L, 2L, 1L, 1L),
         ids(commitId2, commitId2, commitId1, commitId1),
         files(FILE_C, FILE_D, FILE_A, FILE_B),
         statuses(Status.ADDED, Status.ADDED, Status.EXISTING, Status.EXISTING));
@@ -1227,7 +1276,12 @@ public class TestMergeAppend extends TableTestBase {
         "Should create 1 manifest for initial write", 1, snap.allManifests(table.io()).size());
     ManifestFile initialManifest = snap.allManifests(table.io()).get(0);
     validateManifest(
-        initialManifest, seqs(1), ids(commitId), files(FILE_A), statuses(Status.ADDED));
+        initialManifest,
+        dataSeqs(1L),
+        fileSeqs(1L),
+        ids(commitId),
+        files(FILE_A),
+        statuses(Status.ADDED));
 
     // build the new spec using the table's schema, which uses fresh IDs
     PartitionSpec newSpec =
@@ -1264,7 +1318,8 @@ public class TestMergeAppend extends TableTestBase {
     // new manifest comes first
     validateManifest(
         committedSnapshot.allManifests(table.io()).get(0),
-        seqs(2),
+        dataSeqs(2L),
+        fileSeqs(2L),
         ids(committedSnapshot.snapshotId()),
         files(newFile),
         statuses(Status.ADDED));
diff --git a/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java b/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java
index d5007bf6de..6a23596aee 100644
--- a/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java
+++ b/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java
@@ -604,7 +604,8 @@ public class TestReplacePartitions extends TableTestBase {
         statuses(Status.ADDED));
     validateDeleteManifest(
         table.currentSnapshot().allManifests(table.io()).get(2),
-        seqs(2),
+        dataSeqs(2L),
+        fileSeqs(2L),
         ids(id2),
         files(FILE_A_DELETES),
         statuses(Status.ADDED));
diff --git a/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java b/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java
index 2d4c956c90..4af7fc887a 100644
--- a/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java
+++ b/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java
@@ -256,7 +256,8 @@ public class TestRewriteFiles extends TableTestBase {
         statuses(ADDED, ADDED, ADDED));
     validateDeleteManifest(
         initialManifests.get(1),
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(baseSnapshotId, baseSnapshotId),
         files(FILE_A_DELETES, FILE_B_DELETES),
         statuses(ADDED, ADDED));
@@ -290,7 +291,8 @@ public class TestRewriteFiles extends TableTestBase {
 
     validateDeleteManifest(
         pending.allManifests(table.io()).get(2),
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(pendingId, baseSnapshotId),
         files(FILE_A_DELETES, FILE_B_DELETES),
         statuses(DELETED, EXISTING));
@@ -328,7 +330,8 @@ public class TestRewriteFiles extends TableTestBase {
         statuses(ADDED, ADDED, ADDED));
     validateDeleteManifest(
         initialManifests.get(1),
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(baseSnapshotId, baseSnapshotId),
         files(FILE_A_DELETES, FILE_B_DELETES),
         statuses(ADDED, ADDED));
@@ -369,7 +372,8 @@ public class TestRewriteFiles extends TableTestBase {
 
     validateDeleteManifest(
         pending.allManifests(table.io()).get(2),
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(baseSnapshotId, baseSnapshotId),
         files(FILE_A_DELETES, FILE_B_DELETES),
         statuses(ADDED, ADDED));
@@ -455,7 +459,8 @@ public class TestRewriteFiles extends TableTestBase {
 
     validateDeleteManifest(
         pending.allManifests(table.io()).get(2),
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(pending.snapshotId(), pending.snapshotId()),
         files(FILE_A_DELETES, FILE_B_DELETES),
         statuses(DELETED, DELETED));
@@ -549,7 +554,8 @@ public class TestRewriteFiles extends TableTestBase {
 
     validateDeleteManifest(
         manifest3,
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(pending.snapshotId(), pending.snapshotId()),
         files(FILE_A_DELETES, FILE_B_DELETES),
         statuses(DELETED, DELETED));
@@ -598,10 +604,20 @@ public class TestRewriteFiles extends TableTestBase {
     validateManifestEntries(manifest1, ids(baseSnapshotId), files(FILE_A2), statuses(ADDED));
 
     validateDeleteManifest(
-        manifest2, seqs(2), ids(pending.snapshotId()), files(FILE_B_DELETES), statuses(ADDED));
+        manifest2,
+        dataSeqs(2L),
+        fileSeqs(2L),
+        ids(pending.snapshotId()),
+        files(FILE_B_DELETES),
+        statuses(ADDED));
 
     validateDeleteManifest(
-        manifest3, seqs(1), ids(pending.snapshotId()), files(FILE_A2_DELETES), statuses(DELETED));
+        manifest3,
+        dataSeqs(1L),
+        fileSeqs(1L),
+        ids(pending.snapshotId()),
+        files(FILE_A2_DELETES),
+        statuses(DELETED));
 
     rewrite.commit();
 
@@ -644,7 +660,12 @@ public class TestRewriteFiles extends TableTestBase {
     validateManifestEntries(manifest1, ids(pending.snapshotId()), files(FILE_A), statuses(DELETED));
 
     validateDeleteManifest(
-        manifest2, seqs(1), ids(pending.snapshotId()), files(FILE_A_DELETES), statuses(DELETED));
+        manifest2,
+        dataSeqs(1L),
+        fileSeqs(1L),
+        ids(pending.snapshotId()),
+        files(FILE_A_DELETES),
+        statuses(DELETED));
 
     rewrite.commit();
 
diff --git a/core/src/test/java/org/apache/iceberg/TestRowDelta.java b/core/src/test/java/org/apache/iceberg/TestRowDelta.java
index 11f1eec679..cce58f3e62 100644
--- a/core/src/test/java/org/apache/iceberg/TestRowDelta.java
+++ b/core/src/test/java/org/apache/iceberg/TestRowDelta.java
@@ -62,7 +62,8 @@ public class TestRowDelta extends V2TableTestBase {
     Assert.assertEquals("Should produce 1 data manifest", 1, snap.dataManifests(table.io()).size());
     validateManifest(
         snap.dataManifests(table.io()).get(0),
-        seqs(1),
+        dataSeqs(1L),
+        fileSeqs(1L),
         ids(snap.snapshotId()),
         files(FILE_A),
         statuses(Status.ADDED));
@@ -71,7 +72,8 @@ public class TestRowDelta extends V2TableTestBase {
         "Should produce 1 delete manifest", 1, snap.deleteManifests(table.io()).size());
     validateDeleteManifest(
         snap.deleteManifests(table.io()).get(0),
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(snap.snapshotId(), snap.snapshotId()),
         files(FILE_A_DELETES, FILE_B_DELETES),
         statuses(Status.ADDED, Status.ADDED));
@@ -128,7 +130,8 @@ public class TestRowDelta extends V2TableTestBase {
     ManifestFile deletes = table.currentSnapshot().deleteManifests(table.io()).get(0);
     validateDeleteManifest(
         deletes,
-        seqs(4),
+        dataSeqs(4L),
+        fileSeqs(4L),
         ids(table.currentSnapshot().snapshotId()),
         files(FILE_B_DELETES),
         statuses(Status.ADDED));
@@ -235,7 +238,8 @@ public class TestRowDelta extends V2TableTestBase {
     // manifest with FILE_A2 added
     validateManifest(
         snap.dataManifests(table.io()).get(0),
-        seqs(2),
+        dataSeqs(2L),
+        fileSeqs(2L),
         ids(replaceSnapshotId),
         files(FILE_A2),
         statuses(Status.ADDED));
@@ -243,7 +247,8 @@ public class TestRowDelta extends V2TableTestBase {
     // manifest with FILE_A deleted
     validateManifest(
         snap.dataManifests(table.io()).get(1),
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(replaceSnapshotId, appendSnapshotId),
         files(FILE_A, FILE_B),
         statuses(Status.DELETED, Status.EXISTING));
@@ -252,7 +257,8 @@ public class TestRowDelta extends V2TableTestBase {
         "Should have 1 delete manifest", 1, snap.deleteManifests(table.io()).size());
     validateDeleteManifest(
         snap.deleteManifests(table.io()).get(0),
-        seqs(3),
+        dataSeqs(3L),
+        fileSeqs(3L),
         ids(snap.snapshotId()),
         files(FILE_A_DELETES),
         statuses(Status.ADDED));
@@ -397,7 +403,8 @@ public class TestRowDelta extends V2TableTestBase {
     // manifest with FILE_A2 added
     validateManifest(
         snap.dataManifests(table.io()).get(0),
-        seqs(2),
+        dataSeqs(2L),
+        fileSeqs(2L),
         ids(validateFromSnapshotId),
         files(FILE_A2),
         statuses(Status.ADDED));
@@ -405,7 +412,8 @@ public class TestRowDelta extends V2TableTestBase {
     // manifest with FILE_A added
     validateManifest(
         snap.dataManifests(table.io()).get(1),
-        seqs(1),
+        dataSeqs(1L),
+        fileSeqs(1L),
         ids(appendSnapshotId),
         files(FILE_A),
         statuses(Status.ADDED));
@@ -414,7 +422,8 @@ public class TestRowDelta extends V2TableTestBase {
         "Should have 1 delete manifest", 1, snap.deleteManifests(table.io()).size());
     validateDeleteManifest(
         snap.deleteManifests(table.io()).get(0),
-        seqs(3),
+        dataSeqs(3L),
+        fileSeqs(3L),
         ids(snap.snapshotId()),
         files(FILE_A_DELETES),
         statuses(Status.ADDED));
@@ -450,7 +459,8 @@ public class TestRowDelta extends V2TableTestBase {
     Assert.assertEquals("Should produce 1 data manifest", 1, snap.dataManifests(table.io()).size());
     validateManifest(
         snap.dataManifests(table.io()).get(0),
-        seqs(1),
+        dataSeqs(1L),
+        fileSeqs(1L),
         ids(snap.snapshotId()),
         files(FILE_A),
         statuses(Status.DELETED));
@@ -459,7 +469,8 @@ public class TestRowDelta extends V2TableTestBase {
         "Should produce 1 delete manifest", 1, snap.deleteManifests(table.io()).size());
     validateDeleteManifest(
         snap.deleteManifests(table.io()).get(0),
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(snap.snapshotId(), deltaSnapshotId),
         files(FILE_A_DELETES, FILE_B_DELETES),
         statuses(Status.DELETED, Status.EXISTING));
@@ -494,14 +505,16 @@ public class TestRowDelta extends V2TableTestBase {
     int deleteManifestPos = snap.dataManifests(table.io()).get(0).deletedFilesCount() > 0 ? 0 : 1;
     validateManifest(
         snap.dataManifests(table.io()).get(deleteManifestPos),
-        seqs(1),
+        dataSeqs(1L),
+        fileSeqs(1L),
         ids(snap.snapshotId()),
         files(FILE_A),
         statuses(Status.DELETED));
     int appendManifestPos = deleteManifestPos == 0 ? 1 : 0;
     validateManifest(
         snap.dataManifests(table.io()).get(appendManifestPos),
-        seqs(2),
+        dataSeqs(2L),
+        fileSeqs(2L),
         ids(snap.snapshotId()),
         files(FILE_A2),
         statuses(Status.ADDED));
@@ -510,7 +523,8 @@ public class TestRowDelta extends V2TableTestBase {
         "Should produce 1 delete manifest", 1, snap.deleteManifests(table.io()).size());
     validateDeleteManifest(
         snap.deleteManifests(table.io()).get(0),
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(snap.snapshotId(), deltaSnapshotId),
         files(FILE_A_DELETES, FILE_B_DELETES),
         statuses(Status.DELETED, Status.EXISTING));
@@ -543,7 +557,8 @@ public class TestRowDelta extends V2TableTestBase {
     Assert.assertEquals("Should produce 1 data manifest", 1, snap.dataManifests(table.io()).size());
     validateManifest(
         snap.dataManifests(table.io()).get(0),
-        seqs(1),
+        dataSeqs(1L),
+        fileSeqs(1L),
         ids(snap.snapshotId()),
         files(FILE_A),
         statuses(Status.DELETED));
@@ -552,7 +567,8 @@ public class TestRowDelta extends V2TableTestBase {
         "Should produce 1 delete manifest", 1, snap.deleteManifests(table.io()).size());
     validateDeleteManifest(
         snap.deleteManifests(table.io()).get(0),
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
         ids(snap.snapshotId(), snap.snapshotId()),
         files(FILE_A_DELETES, FILE_B_DELETES),
         statuses(Status.DELETED, Status.DELETED));
@@ -580,7 +596,8 @@ public class TestRowDelta extends V2TableTestBase {
         "Should produce 1 data manifest", 1, deleteSnap.dataManifests(table.io()).size());
     validateManifest(
         deleteSnap.dataManifests(table.io()).get(0),
-        seqs(1),
+        dataSeqs(1L),
+        fileSeqs(1L),
         ids(deleteSnap.snapshotId()),
         files(FILE_A),
         statuses(Status.DELETED));
@@ -589,7 +606,8 @@ public class TestRowDelta extends V2TableTestBase {
         "Should produce 1 delete manifest", 1, deleteSnap.deleteManifests(table.io()).size());
     validateDeleteManifest(
         deleteSnap.deleteManifests(table.io()).get(0),
-        seqs(1),
+        dataSeqs(1L),
+        fileSeqs(1L),
         ids(deltaSnapshotId),
         files(FILE_A_DELETES),
         statuses(Status.ADDED));
@@ -612,7 +630,8 @@ public class TestRowDelta extends V2TableTestBase {
         "Should produce 1 delete manifest", 1, nextSnap.deleteManifests(table.io()).size());
     validateDeleteManifest(
         nextSnap.deleteManifests(table.io()).get(0),
-        seqs(1),
+        dataSeqs(1L),
+        fileSeqs(1L),
         ids(nextSnap.snapshotId()),
         files(FILE_A_DELETES),
         statuses(Status.DELETED));
@@ -640,7 +659,8 @@ public class TestRowDelta extends V2TableTestBase {
         "Should produce 1 data manifest", 1, deleteSnap.dataManifests(table.io()).size());
     validateManifest(
         deleteSnap.dataManifests(table.io()).get(0),
-        seqs(1),
+        dataSeqs(1L),
+        fileSeqs(1L),
         ids(deleteSnap.snapshotId()),
         files(FILE_A),
         statuses(Status.DELETED));
@@ -649,7 +669,8 @@ public class TestRowDelta extends V2TableTestBase {
         "Should produce 1 delete manifest", 1, deleteSnap.deleteManifests(table.io()).size());
     validateDeleteManifest(
         deleteSnap.deleteManifests(table.io()).get(0),
-        seqs(1),
+        dataSeqs(1L),
+        fileSeqs(1L),
         ids(deltaSnapshotId),
         files(FILE_A_DELETES),
         statuses(Status.ADDED));
@@ -669,14 +690,16 @@ public class TestRowDelta extends V2TableTestBase {
         nextSnap.dataManifests(table.io()).get(0).deletedFilesCount() > 0 ? 0 : 1;
     validateManifest(
         nextSnap.dataManifests(table.io()).get(deleteManifestPos),
-        seqs(1),
+        dataSeqs(1L),
+        fileSeqs(1L),
         ids(deleteSnap.snapshotId()),
         files(FILE_A),
         statuses(Status.DELETED));
     int appendManifestPos = deleteManifestPos == 0 ? 1 : 0;
     validateManifest(
         nextSnap.dataManifests(table.io()).get(appendManifestPos),
-        seqs(3),
+        dataSeqs(3L),
+        fileSeqs(3L),
         ids(nextSnap.snapshotId()),
         files(FILE_B),
         statuses(Status.ADDED));
@@ -685,7 +708,8 @@ public class TestRowDelta extends V2TableTestBase {
         "Should produce 1 delete manifest", 1, nextSnap.deleteManifests(table.io()).size());
     validateDeleteManifest(
         nextSnap.deleteManifests(table.io()).get(0),
-        seqs(1),
+        dataSeqs(1L),
+        fileSeqs(1L),
         ids(deltaSnapshotId),
         files(FILE_A_DELETES),
         statuses(Status.ADDED));
@@ -759,7 +783,8 @@ public class TestRowDelta extends V2TableTestBase {
     ManifestFile deletes = table.currentSnapshot().deleteManifests(table.io()).get(0);
     validateDeleteManifest(
         deletes,
-        seqs(4),
+        dataSeqs(4L),
+        fileSeqs(4L),
         ids(table.currentSnapshot().snapshotId()),
         files(deleteFile),
         statuses(Status.ADDED));
@@ -901,7 +926,8 @@ public class TestRowDelta extends V2TableTestBase {
         "Should have 4 data manifest", 4, snapshot.dataManifests(table.io()).size());
     validateManifest(
         snapshot.dataManifests(table.io()).get(0),
-        seqs(4),
+        dataSeqs(4L),
+        fileSeqs(4L),
         ids(snapshot.snapshotId()),
         files(dataFile),
         statuses(Status.ADDED));
@@ -915,7 +941,8 @@ public class TestRowDelta extends V2TableTestBase {
         "Spec must match", firstSnapshotDataFile.specId(), firstDeleteManifest.partitionSpecId());
     validateDeleteManifest(
         firstDeleteManifest,
-        seqs(4),
+        dataSeqs(4L),
+        fileSeqs(4L),
         ids(snapshot.snapshotId()),
         files(firstDeleteFile),
         statuses(Status.ADDED));
@@ -925,7 +952,8 @@ public class TestRowDelta extends V2TableTestBase {
         "Spec must match", secondSnapshotDataFile.specId(), secondDeleteManifest.partitionSpecId());
     validateDeleteManifest(
         secondDeleteManifest,
-        seqs(4),
+        dataSeqs(4L),
+        fileSeqs(4L),
         ids(snapshot.snapshotId()),
         files(secondDeleteFile),
         statuses(Status.ADDED));
@@ -935,7 +963,8 @@ public class TestRowDelta extends V2TableTestBase {
         "Spec must match", thirdSnapshotDataFile.specId(), thirdDeleteManifest.partitionSpecId());
     validateDeleteManifest(
         thirdDeleteManifest,
-        seqs(4),
+        dataSeqs(4L),
+        fileSeqs(4L),
         ids(snapshot.snapshotId()),
         files(thirdDeleteFile),
         statuses(Status.ADDED));
@@ -996,7 +1025,8 @@ public class TestRowDelta extends V2TableTestBase {
         "Spec must match", firstSnapshotDataFile.specId(), firstDeleteManifest.partitionSpecId());
     validateDeleteManifest(
         firstDeleteManifest,
-        seqs(4, 3),
+        dataSeqs(4L, 3L),
+        fileSeqs(4L, 3L),
         ids(fourthSnapshot.snapshotId(), thirdSnapshot.snapshotId()),
         files(thirdDeleteFile, firstDeleteFile),
         statuses(Status.ADDED, Status.EXISTING));
@@ -1006,7 +1036,8 @@ public class TestRowDelta extends V2TableTestBase {
         "Spec must match", secondSnapshotDataFile.specId(), secondDeleteManifest.partitionSpecId());
     validateDeleteManifest(
         secondDeleteManifest,
-        seqs(4, 3),
+        dataSeqs(4L, 3L),
+        fileSeqs(4L, 3L),
         ids(fourthSnapshot.snapshotId(), thirdSnapshot.snapshotId()),
         files(fourthDeleteFile, secondDeleteFile),
         statuses(Status.ADDED, Status.EXISTING));
@@ -1299,12 +1330,14 @@ public class TestRowDelta extends V2TableTestBase {
     List<ManifestFile> dataManifests = table.currentSnapshot().dataManifests(table.io());
     Assert.assertEquals("should have 1 data manifest", 1, dataManifests.size());
     ManifestFile mergedDataManifest = dataManifests.get(0);
+    Assert.assertEquals("Manifest seq number must match", 3L, mergedDataManifest.sequenceNumber());
 
     long currentSnapshotId = table.currentSnapshot().snapshotId();
 
     validateManifest(
         mergedDataManifest,
-        seqs(1, 1),
+        dataSeqs(1L, 1L),
+        fileSeqs(3L, 1L),
         ids(currentSnapshotId, currentSnapshotId),
         files(dataFile2, dataFile1),
         statuses(Status.ADDED, Status.DELETED));
diff --git a/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java b/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java
index de0a643f9d..8379e6b29b 100644
--- a/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java
+++ b/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java
@@ -38,7 +38,7 @@ public class TestSequenceNumberForV2Table extends TableTestBase {
     long commitId1 = snap1.snapshotId();
     ManifestFile manifestFile = table.currentSnapshot().allManifests(table.io()).get(0);
     validateSnapshot(null, snap1, 1, FILE_A);
-    validateManifest(manifestFile, seqs(1), ids(commitId1), files(FILE_A));
+    validateManifest(manifestFile, dataSeqs(1L), fileSeqs(1L), ids(commitId1), files(FILE_A));
     V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap1.sequenceNumber());
     V2Assert.assertEquals(
         "Last sequence number should be 1", 1, readMetadata().lastSequenceNumber());
@@ -48,7 +48,7 @@ public class TestSequenceNumberForV2Table extends TableTestBase {
     long commitId2 = snap2.snapshotId();
     manifestFile = table.currentSnapshot().allManifests(table.io()).get(0);
     validateSnapshot(snap1, snap2, 2, FILE_B);
-    validateManifest(manifestFile, seqs(2), ids(commitId2), files(FILE_B));
+    validateManifest(manifestFile, dataSeqs(2L), fileSeqs(2L), ids(commitId2), files(FILE_B));
     V2Assert.assertEquals("Snapshot sequence number should be 2", 2, snap2.sequenceNumber());
     V2Assert.assertEquals(
         "Last sequence number should be 2", 2, readMetadata().lastSequenceNumber());
@@ -70,11 +70,15 @@ public class TestSequenceNumberForV2Table extends TableTestBase {
       if (entry.file().path().equals(FILE_A.path())) {
         V2Assert.assertEquals(
             "FILE_A sequence number should be 1", 1, entry.dataSequenceNumber().longValue());
+        V2Assert.assertEquals(
+            "FILE_A file sequence number should be 1", 1, entry.fileSequenceNumber().longValue());
       }
 
       if (entry.file().path().equals(FILE_B.path())) {
         V2Assert.assertEquals(
             "FILE_b sequence number should be 2", 2, entry.dataSequenceNumber().longValue());
+        V2Assert.assertEquals(
+            "FILE_B file sequence number should be 2", 2, entry.fileSequenceNumber().longValue());
       }
     }
   }
@@ -101,7 +105,7 @@ public class TestSequenceNumberForV2Table extends TableTestBase {
     long commitId1 = snap1.snapshotId();
     ManifestFile manifestFile = table.currentSnapshot().allManifests(table.io()).get(0);
     validateSnapshot(null, snap1, 1, FILE_A);
-    validateManifest(manifestFile, seqs(1), ids(commitId1), files(FILE_A));
+    validateManifest(manifestFile, dataSeqs(1L), fileSeqs(1L), ids(commitId1), files(FILE_A));
     V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap1.sequenceNumber());
     V2Assert.assertEquals(
         "Last sequence number should be 1", 1, readMetadata().lastSequenceNumber());
@@ -113,7 +117,7 @@ public class TestSequenceNumberForV2Table extends TableTestBase {
     long commitId2 = snap2.snapshotId();
     manifestFile = table.currentSnapshot().allManifests(table.io()).get(0);
     validateSnapshot(snap1, snap2, 2, FILE_D);
-    validateManifest(manifestFile, seqs(2), ids(commitId2), files(FILE_D));
+    validateManifest(manifestFile, dataSeqs(2L), fileSeqs(2L), ids(commitId2), files(FILE_D));
     V2Assert.assertEquals("Snapshot sequence number should be 2", 2, snap2.sequenceNumber());
     V2Assert.assertEquals(
         "Last sequence number should be 2", 2, readMetadata().lastSequenceNumber());
@@ -122,7 +126,7 @@ public class TestSequenceNumberForV2Table extends TableTestBase {
     Snapshot snap3 = table.currentSnapshot();
     long commitId3 = snap3.snapshotId();
     manifestFile = table.currentSnapshot().allManifests(table.io()).get(0);
-    validateManifest(manifestFile, seqs(3), ids(commitId3), files(FILE_C));
+    validateManifest(manifestFile, dataSeqs(3L), fileSeqs(3L), ids(commitId3), files(FILE_C));
     validateSnapshot(snap2, snap3, 3, FILE_C);
     V2Assert.assertEquals("Snapshot sequence number should be 3", 3, snap3.sequenceNumber());
     V2Assert.assertEquals(
@@ -136,7 +140,7 @@ public class TestSequenceNumberForV2Table extends TableTestBase {
     long commitId1 = snap1.snapshotId();
     ManifestFile manifestFile = table.currentSnapshot().allManifests(table.io()).get(0);
     validateSnapshot(null, snap1, 1, FILE_A);
-    validateManifest(manifestFile, seqs(1), ids(commitId1), files(FILE_A));
+    validateManifest(manifestFile, dataSeqs(1L), fileSeqs(1L), ids(commitId1), files(FILE_A));
     V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap1.sequenceNumber());
     V2Assert.assertEquals(
         "Last sequence number should be 1", 1, readMetadata().lastSequenceNumber());
@@ -146,7 +150,7 @@ public class TestSequenceNumberForV2Table extends TableTestBase {
     long commitId2 = snap2.snapshotId();
     manifestFile = table.currentSnapshot().allManifests(table.io()).get(0);
     validateSnapshot(snap1, snap2, 2, FILE_B);
-    validateManifest(manifestFile, seqs(2), ids(commitId2), files(FILE_B));
+    validateManifest(manifestFile, dataSeqs(2L), fileSeqs(2L), ids(commitId2), files(FILE_B));
     V2Assert.assertEquals("Snapshot sequence number should be 2", 2, snap2.sequenceNumber());
     V2Assert.assertEquals(
         "Last sequence number should be 2", 2, readMetadata().lastSequenceNumber());
@@ -162,7 +166,7 @@ public class TestSequenceNumberForV2Table extends TableTestBase {
     long commitId4 = snap4.snapshotId();
     manifestFile = table.currentSnapshot().allManifests(table.io()).get(0);
     validateSnapshot(snap3, snap4, 3, FILE_C);
-    validateManifest(manifestFile, seqs(3), ids(commitId4), files(FILE_C));
+    validateManifest(manifestFile, dataSeqs(3L), fileSeqs(3L), ids(commitId4), files(FILE_C));
     V2Assert.assertEquals("Snapshot sequence number should be 1", 3, snap4.sequenceNumber());
     V2Assert.assertEquals(
         "Last sequence number should be 3", 3, readMetadata().lastSequenceNumber());
@@ -177,7 +181,7 @@ public class TestSequenceNumberForV2Table extends TableTestBase {
     long commitId = snap.snapshotId();
     ManifestFile manifestFile = table.currentSnapshot().allManifests(table.io()).get(0);
     validateSnapshot(null, snap, 1, FILE_A);
-    validateManifest(manifestFile, seqs(1), ids(commitId), files(FILE_A));
+    validateManifest(manifestFile, dataSeqs(1L), fileSeqs(1L), ids(commitId), files(FILE_A));
     V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap.sequenceNumber());
     V2Assert.assertEquals(
         "Last sequence number should be 1", 1, readMetadata().lastSequenceNumber());
@@ -200,7 +204,7 @@ public class TestSequenceNumberForV2Table extends TableTestBase {
     long commitId1 = snap1.snapshotId();
     ManifestFile manifestFile1 = table.currentSnapshot().allManifests(table.io()).get(0);
     validateSnapshot(null, snap1, 1, FILE_A);
-    validateManifest(manifestFile1, seqs(1), ids(commitId1), files(FILE_A));
+    validateManifest(manifestFile1, dataSeqs(1L), fileSeqs(1L), ids(commitId1), files(FILE_A));
     V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap1.sequenceNumber());
     V2Assert.assertEquals(
         "Last sequence number should be 1", 1, readMetadata().lastSequenceNumber());
@@ -210,7 +214,7 @@ public class TestSequenceNumberForV2Table extends TableTestBase {
     long commitId2 = snap2.snapshotId();
     ManifestFile manifestFile = table.currentSnapshot().allManifests(table.io()).get(0);
     validateSnapshot(snap1, snap2, 2, FILE_B);
-    validateManifest(manifestFile, seqs(2), ids(commitId2), files(FILE_B));
+    validateManifest(manifestFile, dataSeqs(2L), fileSeqs(2L), ids(commitId2), files(FILE_B));
     V2Assert.assertEquals("Snapshot sequence number should be 2", 2, snap2.sequenceNumber());
     V2Assert.assertEquals(
         "Last sequence number should be 2", 2, readMetadata().lastSequenceNumber());
@@ -223,7 +227,7 @@ public class TestSequenceNumberForV2Table extends TableTestBase {
             .filter(manifest -> manifest.snapshotId() == commitId3)
             .collect(Collectors.toList())
             .get(0);
-    validateManifest(manifestFile, seqs(3), ids(commitId3), files(FILE_C));
+    validateManifest(manifestFile, dataSeqs(3L), fileSeqs(3L), ids(commitId3), files(FILE_C));
     validateSnapshot(snap2, snap3, 3, FILE_C);
     V2Assert.assertEquals("Snapshot sequence number should be 3", 3, snap3.sequenceNumber());
     V2Assert.assertEquals(
@@ -238,7 +242,12 @@ public class TestSequenceNumberForV2Table extends TableTestBase {
             .collect(Collectors.toList())
             .get(0);
     validateManifest(
-        manifestFile, seqs(1), ids(commitId4), files(FILE_A), statuses(Status.DELETED));
+        manifestFile,
+        dataSeqs(1L),
+        fileSeqs(1L),
+        ids(commitId4),
+        files(FILE_A),
+        statuses(Status.DELETED));
     V2Assert.assertEquals("Snapshot sequence number should be 4", 4, snap4.sequenceNumber());
     V2Assert.assertEquals(
         "Last sequence number should be 4", 4, readMetadata().lastSequenceNumber());
@@ -252,7 +261,7 @@ public class TestSequenceNumberForV2Table extends TableTestBase {
     long commitId1 = snap1.snapshotId();
     ManifestFile manifestFile = snap1.allManifests(table.io()).get(0);
     validateSnapshot(null, snap1, 1, FILE_A);
-    validateManifest(manifestFile, seqs(1), ids(commitId1), files(FILE_A));
+    validateManifest(manifestFile, dataSeqs(1L), fileSeqs(1L), ids(commitId1), files(FILE_A));
     V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap1.sequenceNumber());
     V2Assert.assertEquals(
         "Last sequence number should be 0", 0, readMetadata().lastSequenceNumber());
@@ -272,7 +281,7 @@ public class TestSequenceNumberForV2Table extends TableTestBase {
             .collect(Collectors.toList())
             .get(0);
 
-    validateManifest(manifestFile, seqs(2), ids(commitId2), files(FILE_B));
+    validateManifest(manifestFile, dataSeqs(2L), fileSeqs(2L), ids(commitId2), files(FILE_B));
     V2Assert.assertEquals("Snapshot sequence number should be 2", 2, snap2.sequenceNumber());
     V2Assert.assertEquals(
         "Last sequence number should be 2", 2, readMetadata().lastSequenceNumber());
@@ -285,7 +294,7 @@ public class TestSequenceNumberForV2Table extends TableTestBase {
     long commitId1 = snap1.snapshotId();
     ManifestFile manifestFile = table.currentSnapshot().allManifests(table.io()).get(0);
     validateSnapshot(null, snap1, 1, FILE_A);
-    validateManifest(manifestFile, seqs(1), ids(commitId1), files(FILE_A));
+    validateManifest(manifestFile, dataSeqs(1L), fileSeqs(1L), ids(commitId1), files(FILE_A));
     V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap1.sequenceNumber());
     V2Assert.assertEquals(
         "Last sequence number should be 1", 1, readMetadata().lastSequenceNumber());
@@ -295,7 +304,7 @@ public class TestSequenceNumberForV2Table extends TableTestBase {
     long commitId2 = snap2.snapshotId();
     manifestFile = table.currentSnapshot().allManifests(table.io()).get(0);
     validateSnapshot(snap1, snap2, 2, FILE_B);
-    validateManifest(manifestFile, seqs(2), ids(commitId2), files(FILE_B));
+    validateManifest(manifestFile, dataSeqs(2L), fileSeqs(2L), ids(commitId2), files(FILE_B));
     V2Assert.assertEquals("Snapshot sequence number should be 2", 2, snap2.sequenceNumber());
     V2Assert.assertEquals(
         "Last sequence number should be 2", 2, readMetadata().lastSequenceNumber());
@@ -314,7 +323,12 @@ public class TestSequenceNumberForV2Table extends TableTestBase {
     long commitId1 = snap1.snapshotId();
     ManifestFile manifestFile = table.currentSnapshot().allManifests(table.io()).get(0);
     validateSnapshot(null, snap1, 1, FILE_A, FILE_B);
-    validateManifest(manifestFile, seqs(1, 1), ids(commitId1, commitId1), files(FILE_A, FILE_B));
+    validateManifest(
+        manifestFile,
+        dataSeqs(1L, 1L),
+        fileSeqs(1L, 1L),
+        ids(commitId1, commitId1),
+        files(FILE_A, FILE_B));
     V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap1.sequenceNumber());
     V2Assert.assertEquals(
         "Last sequence number should be 1", 1, readMetadata().lastSequenceNumber());
@@ -343,7 +357,7 @@ public class TestSequenceNumberForV2Table extends TableTestBase {
     long commitId1 = snap1.snapshotId();
     ManifestFile manifestFile = snap1.allManifests(table.io()).get(0);
     validateSnapshot(null, snap1, 1, FILE_A);
-    validateManifest(manifestFile, seqs(1), ids(commitId1), files(FILE_A));
+    validateManifest(manifestFile, dataSeqs(1L), fileSeqs(1L), ids(commitId1), files(FILE_A));
     V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap1.sequenceNumber());
     V2Assert.assertEquals(
         "Last sequence number should be 1", 1, readMetadata().lastSequenceNumber());
@@ -366,7 +380,7 @@ public class TestSequenceNumberForV2Table extends TableTestBase {
     Snapshot snap3 = table.currentSnapshot();
     long commitId3 = snap3.snapshotId();
     manifestFile = snap3.allManifests(table.io()).get(0);
-    validateManifest(manifestFile, seqs(3), ids(commitId3), files(FILE_C));
+    validateManifest(manifestFile, dataSeqs(3L), fileSeqs(3L), ids(commitId3), files(FILE_C));
     validateSnapshot(snap2, snap3, 3, FILE_C);
     V2Assert.assertEquals("Snapshot sequence number should be 3", 3, snap3.sequenceNumber());
     V2Assert.assertEquals(
@@ -377,7 +391,7 @@ public class TestSequenceNumberForV2Table extends TableTestBase {
     Snapshot snap4 = table.currentSnapshot();
     long commitId4 = snap4.snapshotId();
     manifestFile = table.currentSnapshot().allManifests(table.io()).get(0);
-    validateManifest(manifestFile, seqs(4), ids(commitId4), files(FILE_B));
+    validateManifest(manifestFile, dataSeqs(4L), fileSeqs(4L), ids(commitId4), files(FILE_B));
     validateSnapshot(snap3, snap4, 4, FILE_B);
     V2Assert.assertEquals("Snapshot sequence number should be 4", 4, snap4.sequenceNumber());
     V2Assert.assertEquals(
@@ -391,7 +405,7 @@ public class TestSequenceNumberForV2Table extends TableTestBase {
     long commitId1 = snap1.snapshotId();
     ManifestFile manifestFile = snap1.allManifests(table.io()).get(0);
     validateSnapshot(null, snap1, 1, FILE_A);
-    validateManifest(manifestFile, seqs(1), ids(commitId1), files(FILE_A));
+    validateManifest(manifestFile, dataSeqs(1L), fileSeqs(1L), ids(commitId1), files(FILE_A));
     V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap1.sequenceNumber());
     V2Assert.assertEquals(
         "Last sequence number should be 1", 1, readMetadata().lastSequenceNumber());
@@ -412,7 +426,7 @@ public class TestSequenceNumberForV2Table extends TableTestBase {
     Snapshot snap3 = table.currentSnapshot();
     long commitId3 = snap3.snapshotId();
     manifestFile = snap3.allManifests(table.io()).get(0);
-    validateManifest(manifestFile, seqs(2), ids(commitId3), files(FILE_B));
+    validateManifest(manifestFile, dataSeqs(2L), fileSeqs(2L), ids(commitId3), files(FILE_B));
     validateSnapshot(snap2, snap3, 2, FILE_B);
     V2Assert.assertEquals("Snapshot sequence number should be 2", 2, snap3.sequenceNumber());
     V2Assert.assertEquals(
diff --git a/core/src/test/java/org/apache/iceberg/TestV1ToV2RowDeltaDelete.java b/core/src/test/java/org/apache/iceberg/TestV1ToV2RowDeltaDelete.java
index f2f0602b3f..6d38ab4273 100644
--- a/core/src/test/java/org/apache/iceberg/TestV1ToV2RowDeltaDelete.java
+++ b/core/src/test/java/org/apache/iceberg/TestV1ToV2RowDeltaDelete.java
@@ -235,7 +235,8 @@ public class TestV1ToV2RowDeltaDelete extends TableTestBase {
 
     validateManifest(
         appendSnapshot.dataManifests(table.io()).get(0),
-        seqs(0, 0),
+        dataSeqs(0L, 0L),
+        fileSeqs(0L, 0L),
         ids(appendSnapshot.snapshotId(), appendSnapshot.snapshotId()),
         files(FILE_A, FILE_B),
         statuses(ManifestEntry.Status.ADDED, ManifestEntry.Status.ADDED));
@@ -253,14 +254,16 @@ public class TestV1ToV2RowDeltaDelete extends TableTestBase {
 
     validateManifest(
         deltaSnapshot.dataManifests(table.io()).get(0),
-        seqs(0, 0),
+        dataSeqs(0L, 0L),
+        fileSeqs(0L, 0L),
         ids(appendSnapshot.snapshotId(), appendSnapshot.snapshotId()),
         files(FILE_A, FILE_B),
         statuses(ManifestEntry.Status.ADDED, ManifestEntry.Status.ADDED));
 
     validateDeleteManifest(
         deltaSnapshot.deleteManifests(table.io()).get(0),
-        seqs(1),
+        dataSeqs(1L),
+        fileSeqs(1L),
         ids(deltaSnapshot.snapshotId()),
         files(FILE_A_DELETES),
         statuses(ManifestEntry.Status.ADDED));
@@ -272,14 +275,16 @@ public class TestV1ToV2RowDeltaDelete extends TableTestBase {
 
     validateManifest(
         deleteSnapshot.dataManifests(table.io()).get(0),
-        seqs(0, 0),
+        dataSeqs(0L, 0L),
+        fileSeqs(0L, 0L),
         ids(appendSnapshot.snapshotId(), deleteSnapshot.snapshotId()),
         files(FILE_A, FILE_B),
         statuses(ManifestEntry.Status.EXISTING, ManifestEntry.Status.DELETED));
 
     validateDeleteManifest(
         deleteSnapshot.deleteManifests(table.io()).get(0),
-        seqs(1),
+        dataSeqs(1L),
+        fileSeqs(1L),
         ids(deltaSnapshot.snapshotId()),
         files(FILE_A_DELETES),
         statuses(ManifestEntry.Status.ADDED));
diff --git a/format/spec.md b/format/spec.md
index 96b335ffbb..5b88e8335c 100644
--- a/format/spec.md
+++ b/format/spec.md
@@ -411,12 +411,13 @@ A manifest file must store the partition spec and other metadata as properties i
 
 The schema of a manifest file is a struct called `manifest_entry` with the following fields:
 
-| v1         | v2         | Field id, name           | Type                                                      | Description                                                                                  |
-| ---------- | ---------- |--------------------------|-----------------------------------------------------------|----------------------------------------------------------------------------------------------|
-| _required_ | _required_ | **`0  status`**          | `int` with meaning: `0: EXISTING` `1: ADDED` `2: DELETED` | Used to track additions and deletions. Deletes are informational only and not used in scans. |
-| _required_ | _optional_ | **`1  snapshot_id`**     | `long`                                                    | Snapshot id where the file was added, or deleted if status is 2. Inherited when null.        |
-|            | _optional_ | **`3  sequence_number`** | `long`                                                    | Data sequence number of the file. Inherited when null.                                       |
-| _required_ | _required_ | **`2  data_file`**       | `data_file` `struct` (see below)                          | File path, partition tuple, metrics, ...                                                     |
+| v1         | v2         | Field id, name                | Type                                                      | Description |
+| ---------- | ---------- |-------------------------------|-----------------------------------------------------------|-------------|
+| _required_ | _required_ | **`0  status`**               | `int` with meaning: `0: EXISTING` `1: ADDED` `2: DELETED` | Used to track additions and deletions. Deletes are informational only and not used in scans. |
+| _required_ | _optional_ | **`1  snapshot_id`**          | `long`                                                    | Snapshot id where the file was added, or deleted if status is 2. Inherited when null. |
+|            | _optional_ | **`3  sequence_number`**      | `long`                                                    | Data sequence number of the file. Inherited when null and status is 1 (added). |
+|            | _optional_ | **`4  file_sequence_number`** | `long`                                                    | File sequence number indicating when the file was added. Inherited when null and status is 1 (added). |
+| _required_ | _required_ | **`2  data_file`**            | `data_file` `struct` (see below)                          | File path, partition tuple, metrics, ... |
 
 `data_file` is a struct with the following fields:
 
@@ -463,8 +464,10 @@ When a file is added to the dataset, its manifest entry should store the snapsho
 
 When a file is replaced or deleted from the dataset, its manifest entry fields store the snapshot ID in which the file was deleted and status 2 (deleted). The file may be deleted from the file system when the snapshot in which it was deleted is garbage collected, assuming that older snapshots have also been garbage collected [1].
 
-Iceberg v2 adds a sequence number to the entry and makes the snapshot id optional. Both fields, `sequence_number` and `snapshot_id`, are inherited from manifest metadata when `null`. That is, if the field is `null` for an entry, then the entry must inherit its value from the manifest file's metadata, stored in the manifest list [2].
-The `sequence_number` field represents the data sequence number and must never change after a file is added to the dataset. 
+Iceberg v2 adds data and file sequence numbers to the entry and makes the snapshot ID optional. Values for these fields are inherited from manifest metadata when `null`. That is, if the field is `null` for an entry, then the entry must inherit its value from the manifest file's metadata, stored in the manifest list.
+The `sequence_number` field represents the data sequence number and must never change after a file is added to the dataset. The data sequence number represents a relative age of the file content and should be used for planning which delete files apply to a data file.
+The `file_sequence_number` field represents the sequence number of the snapshot that added the file and must also remain unchanged upon assigning at commit. The file sequence number can't be used for pruning delete files as the data within the file may have an older data sequence number. 
+The data and file sequence numbers are inherited only if the entry status is 1 (added). If the entry status is 0 (existing) or 2 (deleted), the entry must include both sequence numbers explicitly.
 
 Notes:
 
@@ -475,10 +478,10 @@ Notes:
 
 Manifests track the sequence number when a data or delete file was added to the table.
 
-When adding a new file, its sequence number is set to `null` because the snapshot's sequence number is not assigned until the snapshot is successfully committed. When reading, sequence numbers are inherited by replacing `null` with the manifest's sequence number from the manifest list.
-It is also possible to add a new file that logically belongs to an older sequence number. In that case, the sequence number must be provided explicitly and not inherited.
+When adding a new file, its data and file sequence numbers are set to `null` because the snapshot's sequence number is not assigned until the snapshot is successfully committed. When reading, sequence numbers are inherited by replacing `null` with the manifest's sequence number from the manifest list.
+It is also possible to add a new file with data that logically belongs to an older sequence number. In that case, the data sequence number must be provided explicitly and not inherited. However, the file sequence number must be always assigned when the snapshot is successfully committed.
 
-When writing an existing file to a new manifest or marking an existing file as deleted, the sequence number must be non-null and set to the original data sequence number of the file that was either inherited or provided at the commit time.
+When writing an existing file to a new manifest or marking an existing file as deleted, the data and file sequence numbers must be non-null and set to the original values that were either inherited or provided at the commit time.
 
 Inheriting sequence numbers through the metadata tree allows writing a new manifest without a known sequence number, so that a manifest can be written once and reused in commit retries. To change a sequence number for a retry, only the manifest list must be rewritten.
 
@@ -537,7 +540,7 @@ Manifest list files store `manifest_file`, a struct with the following fields:
 | _required_ | _required_ | **`502 partition_spec_id`**    | `int`                                       | ID of a partition spec used to write the manifest; must be listed in table metadata `partition-specs` |
 |            | _required_ | **`517 content`**              | `int` with meaning: `0: data`, `1: deletes` | The type of files tracked by the manifest, either data or delete files; 0 for all v1 manifests |
 |            | _required_ | **`515 sequence_number`**      | `long`                                      | The sequence number when the manifest was added to the table; use 0 when reading v1 manifest lists |
-|            | _required_ | **`516 min_sequence_number`**  | `long`                                      | The minimum sequence number of all live data or delete files in the manifest; use 0 when reading v1 manifest lists |
+|            | _required_ | **`516 min_sequence_number`**  | `long`                                      | The minimum data sequence number of all live data or delete files in the manifest; use 0 when reading v1 manifest lists |
 | _required_ | _required_ | **`503 added_snapshot_id`**    | `long`                                      | ID of the snapshot where the  manifest file was added |
 | _optional_ | _required_ | **`504 added_files_count`**    | `int`                                       | Number of entries in the manifest that have status `ADDED` (1), when `null` this is assumed to be non-zero |
 | _optional_ | _required_ | **`505 existing_files_count`** | `int`                                       | Number of entries in the manifest that have status `EXISTING` (0), when `null` this is assumed to be non-zero |
@@ -584,16 +587,16 @@ Note that for any snapshot, all file paths marked with "ADDED" or "EXISTING" may
 Delete files that match the query filter must be applied to data files at read time, limited by the scope of the delete file using the following rules.
 
 * A _position_ delete file must be applied to a data file when all of the following are true:
-    - The data file's sequence number is _less than or equal to_ the delete file's sequence number
+    - The data file's data sequence number is _less than or equal to_ the delete file's data sequence number
     - The data file's partition (both spec and partition values) is equal to the delete file's partition
 * An _equality_ delete file must be applied to a data file when all of the following are true:
-    - The data file's sequence number is _strictly less than_ the delete's sequence number
+    - The data file's data sequence number is _strictly less than_ the delete's data sequence number
     - The data file's partition (both spec and partition values) is equal to the delete file's partition _or_ the delete file's partition spec is unpartitioned
 
 In general, deletes are applied only to data files that are older and in the same partition, except for two special cases:
 
 * Equality delete files stored with an unpartitioned spec are applied as global deletes. Otherwise, delete files do not apply to files in other partitions.
-* Position delete files must be applied to data files from the same commit, when the data and delete file sequence numbers are equal. This allows deleting rows that were added in the same commit.
+* Position delete files must be applied to data files from the same commit, when the data and delete file data sequence numbers are equal. This allows deleting rows that were added in the same commit.
 
 
 Notes:
@@ -1196,6 +1199,7 @@ Writing v1 metadata:
 * Manifest list field `min-sequence-number` should not be written
 * Manifest list field `content` must be 0 (data) or omitted
 * Manifest entry field `sequence_number` should not be written
+* Manifest entry field `file_sequence_number` should not be written
 * Data file field `content` must be 0 (data) or omitted
 
 Reading v1 metadata for v2:
@@ -1206,6 +1210,7 @@ Reading v1 metadata for v2:
 * Manifest list field `min-sequence-number` must default to 0
 * Manifest list field `content` must default to 0 (data)
 * Manifest entry field `sequence_number` must default to 0
+* Manifest entry field `file_sequence_number` must default to 0
 * Data file field `content` must default to 0 (data)
 
 Writing v2 metadata:
@@ -1244,6 +1249,7 @@ Writing v2 metadata:
 * Manifest `manifest_entry`:
     * `snapshot_id` is now optional to support inheritance
     * `sequence_number` was added and is optional, to support inheritance
+    * `file_sequence_number` was added and is optional, to support inheritance
 * Manifest `data_file`:
     * `content` was added and is required; 0=data, 1=position deletes, 2=equality deletes; default to 0 when reading v1 manifests
     * `equality_ids` was added, to be used for equality deletes only
diff --git a/spark/v2.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v2.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java
index 7fc589056d..675a330c84 100644
--- a/spark/v2.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java
+++ b/spark/v2.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java
@@ -170,7 +170,8 @@ public abstract class TestIcebergSourceTablesBase extends SparkTestBase {
       // each row must inherit snapshot_id and sequence_number
       rows.forEach(
           row -> {
-            row.put(2, 0L);
+            row.put(2, 0L); // data sequence number
+            row.put(3, 0L); // file sequence number
             GenericData.Record file = (GenericData.Record) row.get("data_file");
             asMetadataRecord(file);
             expected.add(row);
@@ -365,7 +366,8 @@ public abstract class TestIcebergSourceTablesBase extends SparkTestBase {
         // each row must inherit snapshot_id and sequence_number
         rows.forEach(
             row -> {
-              row.put(2, 0L);
+              row.put(2, 0L); // data sequence number
+              row.put(3, 0L); // file sequence number
               GenericData.Record file = (GenericData.Record) row.get("data_file");
               asMetadataRecord(file);
               expected.add(row);
diff --git a/spark/v3.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java
index dc8017838f..1976d71b6c 100644
--- a/spark/v3.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java
+++ b/spark/v3.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java
@@ -170,7 +170,8 @@ public abstract class TestIcebergSourceTablesBase extends SparkTestBase {
       // each row must inherit snapshot_id and sequence_number
       rows.forEach(
           row -> {
-            row.put(2, 0L);
+            row.put(2, 0L); // data sequence number
+            row.put(3, 0L); // file sequence number
             GenericData.Record file = (GenericData.Record) row.get("data_file");
             asMetadataRecord(file);
             expected.add(row);
@@ -363,7 +364,8 @@ public abstract class TestIcebergSourceTablesBase extends SparkTestBase {
         // each row must inherit snapshot_id and sequence_number
         rows.forEach(
             row -> {
-              row.put(2, 0L);
+              row.put(2, 0L); // data sequence number
+              row.put(3, 0L); // file sequence number
               GenericData.Record file = (GenericData.Record) row.get("data_file");
               asMetadataRecord(file);
               expected.add(row);
diff --git a/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java
index dc8017838f..1976d71b6c 100644
--- a/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java
+++ b/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java
@@ -170,7 +170,8 @@ public abstract class TestIcebergSourceTablesBase extends SparkTestBase {
       // each row must inherit snapshot_id and sequence_number
       rows.forEach(
           row -> {
-            row.put(2, 0L);
+            row.put(2, 0L); // data sequence number
+            row.put(3, 0L); // file sequence number
             GenericData.Record file = (GenericData.Record) row.get("data_file");
             asMetadataRecord(file);
             expected.add(row);
@@ -363,7 +364,8 @@ public abstract class TestIcebergSourceTablesBase extends SparkTestBase {
         // each row must inherit snapshot_id and sequence_number
         rows.forEach(
             row -> {
-              row.put(2, 0L);
+              row.put(2, 0L); // data sequence number
+              row.put(3, 0L); // file sequence number
               GenericData.Record file = (GenericData.Record) row.get("data_file");
               asMetadataRecord(file);
               expected.add(row);
diff --git a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java
index 141ecd1e8c..57ce654a71 100644
--- a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java
+++ b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java
@@ -170,7 +170,8 @@ public abstract class TestIcebergSourceTablesBase extends SparkTestBase {
       // each row must inherit snapshot_id and sequence_number
       rows.forEach(
           row -> {
-            row.put(2, 0L);
+            row.put(2, 0L); // data sequence number
+            row.put(3, 0L); // file sequence number
             GenericData.Record file = (GenericData.Record) row.get("data_file");
             asMetadataRecord(file);
             expected.add(row);
@@ -363,7 +364,8 @@ public abstract class TestIcebergSourceTablesBase extends SparkTestBase {
         // each row must inherit snapshot_id and sequence_number
         rows.forEach(
             row -> {
-              row.put(2, 0L);
+              row.put(2, 0L); // data sequence number
+              row.put(3, 0L); // file sequence number
               GenericData.Record file = (GenericData.Record) row.get("data_file");
               asMetadataRecord(file);
               expected.add(row);
diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java
index 141ecd1e8c..57ce654a71 100644
--- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java
+++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java
@@ -170,7 +170,8 @@ public abstract class TestIcebergSourceTablesBase extends SparkTestBase {
       // each row must inherit snapshot_id and sequence_number
       rows.forEach(
           row -> {
-            row.put(2, 0L);
+            row.put(2, 0L); // data sequence number
+            row.put(3, 0L); // file sequence number
             GenericData.Record file = (GenericData.Record) row.get("data_file");
             asMetadataRecord(file);
             expected.add(row);
@@ -363,7 +364,8 @@ public abstract class TestIcebergSourceTablesBase extends SparkTestBase {
         // each row must inherit snapshot_id and sequence_number
         rows.forEach(
             row -> {
-              row.put(2, 0L);
+              row.put(2, 0L); // data sequence number
+              row.put(3, 0L); // file sequence number
               GenericData.Record file = (GenericData.Record) row.get("data_file");
               asMetadataRecord(file);
               expected.add(row);