Skip to content

Commit 2059a90

Browse files
authored
[Data] Fix UnboundLocalError when read_parquet with columns and no partitioning (#55820)
<!-- Thank you for your contribution! Please review https://github.com/ray-project/ray/blob/master/CONTRIBUTING.rst before opening a pull request. --> <!-- Please add a reviewer to the assignee section when you create a PR. If you don't have the access to it, we will shortly find a reviewer and assign them to your PR. --> ## Why are these changes needed? <!-- Please give a short summary of the change and the problem this solves. --> If you call `read_parquet` with `partitioning=None` and non-empty `columns`, then Ray Data raises a error because of a missing branch. This PR fixes that issue. ``` Traceback (most recent call last): File "/root/lab42_vr/test.py", line 6, in <module> ds = ray.data.read_parquet(input_s3_path, partitioning=None, columns=columns) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ File "/root/.venv/lib/python3.12/site-packages/ray/data/read_api.py", line 950, in read_parquet datasource = ParquetDatasource( ^^^^^^^^^^^^^^^^^^ File "/root/.venv/lib/python3.12/site-packages/ray/data/_internal/datasource/parquet_datasource.py", line 262, in __init__ data_columns, partition_columns = _infer_data_and_partition_columns( ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ File "/root/.venv/lib/python3.12/site-packages/ray/data/_internal/datasource/parquet_datasource.py", line 817, in _infer_data_and_partition_columns return data_columns, partition_columns ^^^^^^^^^^^^^^^^^ UnboundLocalError: cannot access local variable 'partition_columns' where it is not associated with a value ``` ## Related issue number <!-- For example: "Closes #1234" --> Fixes #55279 ## Checks - [ ] I've signed off every commit(by using the -s flag, i.e., `git commit -s`) in this PR. - [ ] I've run `scripts/format.sh` to lint the changes in this PR. - [ ] I've included any doc changes needed for https://docs.ray.io/en/master/. - [ ] I've added any new APIs to the API Reference. For example, if I added a method in Tune, I've added it in `doc/source/tune/api/` under the corresponding `.rst` file. - [ ] I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/ - Testing Strategy - [ ] Unit tests - [ ] Release tests - [ ] This PR is not tested :( --------- Signed-off-by: Balaji Veeramani <bveeramani@berkeley.edu>
1 parent 6d27fc2 commit 2059a90

File tree

2 files changed

+14
-0
lines changed

2 files changed

+14
-0
lines changed

python/ray/data/_internal/datasource/parquet_datasource.py

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -825,4 +825,7 @@ def _infer_data_and_partition_columns(
825825
partition_columns = [
826826
column for column in user_specified_columns if column in partitions
827827
]
828+
else:
829+
partition_columns = []
830+
828831
return data_columns, partition_columns

python/ray/data/tests/test_parquet.py

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2223,6 +2223,17 @@ def test_parquet_write_parallel_overwrite(
22232223
assert result.count() == 1000
22242224

22252225

2226+
def test_read_parquet_with_none_partitioning_and_columns(tmp_path):
2227+
# Test for https://github.com/ray-project/ray/issues/55279.
2228+
table = pa.table({"column": [42]})
2229+
path = os.path.join(tmp_path, "file.parquet")
2230+
pq.write_table(table, path)
2231+
2232+
ds = ray.data.read_parquet(path, partitioning=None, columns=["column"])
2233+
2234+
assert ds.take_all() == [{"column": 42}]
2235+
2236+
22262237
if __name__ == "__main__":
22272238
import sys
22282239

0 commit comments

Comments
 (0)