Skip to content

[lake/iceberg] IcebergLakeCommitter Can Block Indefinitely During Simultaneous Rewrite Operations #2420

@rionmonster

Description

@rionmonster

Search before asking

  • I searched in the issues and found nothing similar.

Fluss version

0.8.0 (latest release)

Please describe the bug 🐞

While investigating a separate bug, I noticed an issue that can result in transient failures during the IcebergLakeCommitter.commit() process. When the commit contains both new data files and rewrite results in the same transaction, the commitRewrite() operation can block for extremely long periods of times (observed 15+ minutes) due to it repeatedly attempting to validate.

The following chain of events can occur:

  1. New data/delete files are committed via AppendFiles or RowDelta
    This creates a new Iceberg snapshot (creating snapshot N+1).
  2. commitRewrite() is called
    This calls validateFromSnapshot(rewriteDataFileResults.get(0).snapshotId()).
    The validation expects snapshot N (the snapshot when rewrite was planned).
    But snapshot N+1 now exists from Step 1.
    Validation fails because conflicting changes are detected.
  3. Iceberg retries the rewrite commit, but it will never succeed because:
    The table was already modified in Step 1.
    Each retry re-validates against the same old snapshot.
    The conflict is permanent within this commit cycle.

We can see this demonstrated in the corresponding code as well:

// IcebergLakeCommitter.java:100-147
@Override
public long commit(IcebergCommittable committable, Map<String, String> snapshotProperties)
        throws IOException {
    try {
        icebergTable.refresh();

        // ... build snapshotUpdate ...

        // Step 1: Commit data files first (creates new snapshot)
        long snapshotId = commit(snapshotUpdate, snapshotProperties);

        // Step 2: Then try to commit rewrite (validates against OLD snapshot)
        List<RewriteDataFileResult> rewriteDataFileResults = committable.rewriteDataFileResults();
        if (!rewriteDataFileResults.isEmpty()) {
            Long rewriteCommitSnapshotId = commitRewrite(rewriteDataFileResults, snapshotProperties);
            // ...
        }
        return snapshotId;
    } catch (Exception e) {
        throw new IOException("Failed to commit to Iceberg table.", e);
    }
}
// IcebergLakeCommitter.java:149-185
private Long commitRewrite(
        List<RewriteDataFileResult> rewriteDataFileResults,
        Map<String, String> snapshotProperties) {
    icebergTable.refresh();
    RewriteFiles rewriteFiles = icebergTable.newRewrite();
    try {
        // Validates against the snapshot when rewrite was planned
        // But Step 1 already created a newer snapshot!
        rewriteFiles.validateFromSnapshot(rewriteDataFileResults.get(0).snapshotId());
        // ... add files ...
        return commit(rewriteFiles, snapshotProperties);  // This fails validation
    } catch (Exception e) {
        // Error handling - deletes added files
        LOG.error("Failed to commit rewrite files to iceberg...", e);
        CatalogUtil.deleteFiles(...);
        return null;
    }
}

Log-based Evidence

The following represents a timeline (of diagnostic logs) demonstrating the issue at hand with each entry prefixed with the timestamp:

# Epochs 1-5 Completing Normally (~500ms cycles)
2745  requestTable: epoch 1
3258  finishTableTiering: epoch 1 ✓
3755  requestTable: epoch 2
4378  finishTableTiering: epoch 2 ✓
4747  requestTable: epoch 3
5255  finishTableTiering: epoch 3 ✓
5754  requestTable: epoch 4
5989  finishTableTiering: epoch 4 ✓
6754  requestTable: epoch 5
6929  finishTableTiering: epoch 5 ✓

# Epoch 6 Started (7754ms)
7754 [fluss-netty-server-worker-thread-3] requestTable: polled table 0 path fluss.log_table state Pending epoch 6
7754 [fluss-netty-server-worker-thread-3] lakeTieringHeartbeat: requestTable=true assigned table 0 epoch 6

# Main Commit Ran BEFORE Rewrite (7970ms)
7970 [TieringCommitter] main commit completed in 82ms newSnapshotId=3432330612187421343
7970 [TieringCommitter] starting commitRewrite with 1 results

# Snapshot Mismatch Detected (7971ms)
7971 [TieringCommitter] validateFromSnapshotId=3918591728127853509 currentSnapshotId=3432330612187421343
                                               ^^^ rewrite planned here              ^^^ but main commit created this new one

# Iceberg Retry Loop - Blocked 917,869ms (7972ms → 925841ms)
7972   [TieringCommitter] commitRewrite: calling commit() after setup took 2ms
925841 [TieringCommitter] commitRewrite: FAILED after 917870ms error=InterruptedException: sleep interrupted

#finishTableTiering Never Called for Epoch 6
(No log entry exists - epoch 6 never completed)

# Test Assertion Failed (IcebergRewriteITCase.testLogTableCompaction)
expected: 4L
but was: 3L

Solution

This issue should be addressed pretty easily by adjusting the order of the operations within the commit() process to ensure that any pending rewrite operations are committed (as the snapshot should still be valid) before the introduction of new data files similar to below:

@Override
public long commit(IcebergCommittable committable, Map<String, String> snapshotProperties)
        throws IOException {
    try {
        icebergTable.refresh();

        long snapshotId;

        // Step 1: Commit rewrite FIRST (while snapshot is still valid)
        List<RewriteDataFileResult> rewriteDataFileResults = committable.rewriteDataFileResults();
        if (!rewriteDataFileResults.isEmpty()) {
            Long rewriteCommitSnapshotId = commitRewrite(rewriteDataFileResults, snapshotProperties);
            if (rewriteCommitSnapshotId != null) {
                snapshotId = rewriteCommitSnapshotId;
            }
        }

        // Step 2: Then commit new data files
        SnapshotUpdate<?> snapshotUpdate;
        if (committable.getDeleteFiles().isEmpty()) {
            AppendFiles appendFiles = icebergTable.newAppend();
            committable.getDataFiles().forEach(appendFiles::appendFile);
            snapshotUpdate = appendFiles;
        } else {
            RowDelta rowDelta = icebergTable.newRowDelta();
            committable.getDataFiles().forEach(rowDelta::addRows);
            committable.getDeleteFiles().forEach(rowDelta::addDeletes);
            snapshotUpdate = rowDelta;
        }
        snapshotId = commit(snapshotUpdate, snapshotProperties);

        return checkNotNull(snapshotId, "Iceberg committed snapshot id must be non-null.");
    } catch (Exception e) {
        throw new IOException("Failed to commit to Iceberg table.", e);
    }
}

Are you willing to submit a PR?

  • I'm willing to submit a PR!

Metadata

Metadata

Assignees

Labels

No labels
No labels

Type

Projects

No projects

Milestone

No milestone

Relationships

None yet

Development

No branches or pull requests

Issue actions