Skip to content

Parquet Incremental Sync#768

Merged
vinishjail97 merged 134 commits into
apache:mainfrom
sapienza88:parquet_incr_sync
Jun 5, 2026
Merged

Parquet Incremental Sync#768
vinishjail97 merged 134 commits into
apache:mainfrom
sapienza88:parquet_incr_sync

Conversation

@sapienza88

@sapienza88 sapienza88 commented Dec 10, 2025

Copy link
Copy Markdown
Contributor

What is the purpose of the pull request

Adds incremental syncing ability to the ParquetSource

Brief change log

  • Adds a new class ParquetDataManager.java for handling the fetching of data files for Parquet Source
  • Updates IT to include incremental source

Verify this pull request

  • new tests added to ITParquetConversionSource

@sapienza88 sapienza88 changed the title Parquet Incremental Sync: Given a parquet file return data from a certain modification time Parquet Incremental Sync Dec 10, 2025
@rahil-c

rahil-c commented Dec 15, 2025

Copy link
Copy Markdown
Contributor

I can do first review for this @the-other-tim-brown @vinishjail97

@vinishjail97 vinishjail97 self-requested a review December 16, 2025 08:31
Comment thread xtable-core/src/main/java/org/apache/xtable/parquet/ParquetDataManager.java Outdated
Comment thread xtable-core/src/main/java/org/apache/xtable/parquet/ParquetDataManager.java Outdated
Comment thread xtable-core/src/main/java/org/apache/xtable/parquet/ParquetFileConfig.java Outdated
Comment thread xtable-core/src/main/java/org/apache/xtable/parquet/ParquetDataManager.java Outdated
Comment thread xtable-core/src/main/java/org/apache/xtable/parquet/ParquetDataManager.java Outdated
@sapienza88

Copy link
Copy Markdown
Contributor Author

@vinishjail97 I added some comments on the functions so that the approach is clearer. All above suggestions were also taken into account in my last commit.

@vinishjail97

Copy link
Copy Markdown
Contributor

XTable shouldn't be writing any new data or parquet files it operators at a metadata level. Can you see this comment for reference? I had written few approaches on how to do incremental parquet sync.
#550 (comment)

@vinishjail97

Copy link
Copy Markdown
Contributor

@sapienza88 I'm adding a more detailed design and a class level structure to unblock this PR.

Design Principle
XTable operates at a metadata level only. The current PR approach of writing new Parquet files with filtered data is incorrect. XTable should:

  • Discover existing Parquet files from storage
  • Generate table format metadata (Hudi, Iceberg, Delta) for those files
  • NEVER write new Parquet files or transform data.

Architecture

  ┌────────────────────────────────────────────────────────────┐
  │                  ParquetConversionSource                   │
  │  - Uses ParquetFileDiscovery to find files                 │
  │  - Converts file metadata to InternalDataFile              │
  │  - Returns snapshots and table changes                     │
  └────────────────────────────────────────────────────────────┘
                              │
                              ▼
  ┌────────────────────────────────────────────────────────────┐
  │              ParquetFileDiscovery (new class)              │
  │  - Lists all .parquet files from filesystem                │
  │  - Filters files by modification time                      │
  │  - Returns lightweight file metadata                       │
  └────────────────────────────────────────────────────────────┘
                              │
                              ▼
  ┌────────────────────────────────────────────────────────────┐
  │            FileSystem (HDFS/S3/GCS/Azure)                  │
  │  - fs.listFiles(basePath, recursive=true)                  │
  └────────────────────────────────────────────────────────────┘

Use file modification time as commit identifier, you will be able to identify which files have been synced and which haven't been synced. The files not synced need to have metadata generated. The future functionality like making it optimized, handling deleted parquet files in storage can be handled incrementally, hoping to scope low for this PR.

}
}

@Test

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Line 184 needs to be updated to include INCREMENTAL as well

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done!

Comment thread xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java Outdated
Comment thread xtable-core/src/main/java/org/apache/xtable/parquet/ParquetDataManager.java Outdated
Comment thread xtable-core/src/main/java/org/apache/xtable/parquet/ParquetDataManager.java Outdated
Comment thread xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetDataManager.java Outdated
Comment thread xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java Outdated
Comment thread xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java Outdated
Comment thread xtable-core/src/main/java/org/apache/xtable/parquet/ParquetDataManager.java Outdated
…uetDataManager methods

- ParquetConversionSource: materialize the file listing once per operation in
  getTableChangeForCommit and getCurrentSnapshot instead of re-listing the
  filesystem 2-3 times (addresses the 'fetch the list once' review comment).
- ParquetDataManager: remove getParquetDataFileAt and
  getParquetFilesMetadataAfterTime; both had no production caller (only tests).
- TestParquetDataManager: drop tests for the removed methods and repoint the
  real-filesystem tests to getCurrentFilesInfo.
getTableChangeForCommit computed filesAdded and the table from one
materialized listing but derived the committed sourceIdentifier from a
separate parquetDataManager.getMostRecentParquetFile() listing. A file
landing between the two listings could advance the committed identifier
past a file that was never included in filesAdded, permanently skipping
it on the next incremental sync.

Use the most recent file from the same snapshot for both the table and
the source identifier. Add an assertion in ITParquetConversionSource
that the committed identifier matches that snapshot's latest mod time.
@vinishjail97 vinishjail97 merged commit 754fd27 into apache:main Jun 5, 2026
2 checks passed
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants