Skip to content

Conversation

@bveeramani
Copy link
Member

Why are these changes needed?

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

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>
@bveeramani bveeramani requested a review from a team as a code owner August 21, 2025 20:39
Copy link
Contributor

@gemini-code-assist gemini-code-assist bot left a comment

Choose a reason for hiding this comment

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

Code Review

This pull request addresses an UnboundLocalError that occurs when ray.data.read_parquet is called with partitioning=None and a non-empty columns list. The fix correctly initializes partition_columns in this scenario, preventing the error. A corresponding test case has been added to verify the fix.

My review includes one suggestion to make the new test more robust by adding an assertion to verify the content of the dataset, in addition to ensuring that no exception is raised.

@ray-gardener ray-gardener bot added the data Ray Data-related issues label Aug 22, 2025
@bveeramani bveeramani marked this pull request as draft August 22, 2025 02:21
@bveeramani bveeramani changed the title [Data] [Data] Fix UnboundLocalError when read_parquet with columns and no partitioning Aug 22, 2025
@bveeramani bveeramani added the go add ONLY when ready to merge, run all tests label Aug 22, 2025
Signed-off-by: Balaji Veeramani <bveeramani@berkeley.edu>
Signed-off-by: Balaji Veeramani <bveeramani@berkeley.edu>
@bveeramani bveeramani marked this pull request as ready for review August 25, 2025 17:50
@bveeramani bveeramani merged commit 2059a90 into master Aug 25, 2025
5 checks passed
@bveeramani bveeramani deleted the fix-parquet-bug branch August 25, 2025 18:07
liulehui pushed a commit to liulehui/ray that referenced this pull request Aug 26, 2025
…o partitioning (ray-project#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 ray-project#1234" -->

Fixes ray-project#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>
Signed-off-by: Lehui Liu <lehui@anyscale.com>
jugalshah291 pushed a commit to jugalshah291/ray_fork that referenced this pull request Sep 11, 2025
…o partitioning (ray-project#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 ray-project#1234" -->

Fixes ray-project#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>
Signed-off-by: jugalshah291 <shah.jugal291@gmail.com>
dstrodtman pushed a commit that referenced this pull request Oct 6, 2025
…o 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>
Signed-off-by: Douglas Strodtman <douglas@anyscale.com>
landscapepainter pushed a commit to landscapepainter/ray that referenced this pull request Nov 17, 2025
…o partitioning (ray-project#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 ray-project#1234" -->

Fixes ray-project#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>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

data Ray Data-related issues go add ONLY when ready to merge, run all tests

Projects

None yet

Development

Successfully merging this pull request may close these issues.

[data] read_parquet with partitioning=None is incompatible with columns argument

3 participants