Skip to content

Conversation

vtk9
Copy link
Contributor

@vtk9 vtk9 commented Feb 25, 2025

  • parquet_files_to_data_files changed to parquet_file_to_data_files which processes a single parquet file and returns a DataFile
  • _parquet_files_to_data_files uses internal ExecutorFactory

resolves #1335

Copy link
Contributor

@amitgilad3 amitgilad3 left a comment

Choose a reason for hiding this comment

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

Great work!!! , i think you need to run make lint

tbl.add_files(file_paths=file_paths)


@pytest.mark.integration
Copy link
Contributor

Choose a reason for hiding this comment

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

Im not sure this test is any different from existing test (but maybe im mistaken), the test verifies that the files are correctly added to the table—by checking that the manifest counts match the expected values—but it does not directly assert that the file additions are processed concurrently.
we already have tests that add multiple file

Copy link
Contributor Author

Choose a reason for hiding this comment

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

added a better that asserts that files are processed within different threads.
let me know if this is sufficient.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

tests in CICD was failing. it should be fixed now

Copy link
Contributor

Choose a reason for hiding this comment

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

really cool way to check that files are processed within different threads. !!

f"Cannot add file {file_path} because it has field IDs. `add_files` only supports addition of files without field_ids"
)
schema = table_metadata.schema()
_check_pyarrow_schema_compatible(schema, parquet_metadata.schema.to_arrow_schema())
Copy link
Contributor

Choose a reason for hiding this comment

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

We're converting the schema multiple times. Since we're optimizing for performance now, we probably want to store this in a variable to reduce GIL congestion

Copy link
Contributor Author

Choose a reason for hiding this comment

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

i assume you mean schema = table_metadata.schema() I made that change.

assuming you meant _check_pyarrow_schema_compatible, I don't think it is possible to only compute that once since it relies on parquet_metadata which is technically unique to each parquet file

Copy link
Contributor

Choose a reason for hiding this comment

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

to_arrow_schema() is called on lines 2483 and 2479. Would be good to call this just once

Copy link
Contributor Author

Choose a reason for hiding this comment

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

got it. should be fixed now

equality_ids=None,
key_metadata=None,
**statistics.to_serialized_dict(),
def parquet_file_to_data_file(io: FileIO, table_metadata: TableMetadata, file_path: str) -> DataFile:
Copy link
Contributor

Choose a reason for hiding this comment

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

We're changing a public API here. We either have to go through the deprecation cycle, or add a new method parquet_file_to_data_file, that's being used by parquet_file_to_data_files

Copy link
Contributor Author

Choose a reason for hiding this comment

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

i re-added parquet_files_to_data_files (plural). internally it uses parquet_file_to_data_file (singular)

another alternative is to make parquet_file_to_data_file private. what do you think?

Copy link
Contributor

Choose a reason for hiding this comment

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

If we want to make parquet_file_to_date_file private, then we have to go through the deprecation cycle as well. I think we can leave it in for now.

vtk9 added 2 commits February 26, 2025 17:59
- add a better test which checks multiple threads used during execution
- re-add `parquet_files_to_data_files` back and let it use `parquet_file_to_data_file`
- move `schema = table_metadata.schema()` outside of function it is
  being used in
fix integration to be more robust
Copy link
Contributor

@Fokko Fokko left a comment

Choose a reason for hiding this comment

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

Thanks @vtk9 for adding this, and thanks @amitgilad3 for the review 🚀

@Fokko Fokko merged commit f942551 into apache:main Mar 3, 2025
7 checks passed
gabeiglio pushed a commit to Netflix/iceberg-python that referenced this pull request Aug 13, 2025
- `parquet_files_to_data_files` changed to `parquet_file_to_data_files`
which processes a single parquet file and returns a `DataFile`
- `_parquet_files_to_data_files` uses internal ExecutorFactory

resolves apache#1335
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.

improve performance of Table.add_files by parallelizing

3 participants