-
Notifications
You must be signed in to change notification settings - Fork 86
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
feat: reads using global ctx #982
Open
ion-elgreco
wants to merge
1
commit into
apache:main
Choose a base branch
from
ion-elgreco:feat/global_read
base: main
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,181 @@ | ||
# Licensed to the Apache Software Foundation (ASF) under one | ||
# or more contributor license agreements. See the NOTICE file | ||
# distributed with this work for additional information | ||
# regarding copyright ownership. The ASF licenses this file | ||
# to you under the Apache License, Version 2.0 (the | ||
# "License"); you may not use this file except in compliance | ||
# with the License. You may obtain a copy of the License at | ||
# | ||
# http://www.apache.org/licenses/LICENSE-2.0 | ||
# | ||
# Unless required by applicable law or agreed to in writing, | ||
# software distributed under the License is distributed on an | ||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
# KIND, either express or implied. See the License for the | ||
# specific language governing permissions and limitations | ||
# under the License. | ||
|
||
"""IO read functions using global context.""" | ||
|
||
import pathlib | ||
|
||
from datafusion.dataframe import DataFrame | ||
from datafusion.expr import Expr | ||
import pyarrow | ||
from ._internal import SessionContext as SessionContextInternal | ||
|
||
|
||
def read_parquet( | ||
path: str | pathlib.Path, | ||
table_partition_cols: list[tuple[str, str]] | None = None, | ||
parquet_pruning: bool = True, | ||
file_extension: str = ".parquet", | ||
skip_metadata: bool = True, | ||
schema: pyarrow.Schema | None = None, | ||
file_sort_order: list[list[Expr]] | None = None, | ||
) -> DataFrame: | ||
"""Read a Parquet source into a :py:class:`~datafusion.dataframe.Dataframe`. | ||
|
||
Args: | ||
path: Path to the Parquet file. | ||
table_partition_cols: Partition columns. | ||
parquet_pruning: Whether the parquet reader should use the predicate | ||
to prune row groups. | ||
file_extension: File extension; only files with this extension are | ||
selected for data input. | ||
skip_metadata: Whether the parquet reader should skip any metadata | ||
that may be in the file schema. This can help avoid schema | ||
conflicts due to metadata. | ||
schema: An optional schema representing the parquet files. If None, | ||
the parquet reader will try to infer it based on data in the | ||
file. | ||
file_sort_order: Sort order for the file. | ||
|
||
Returns: | ||
DataFrame representation of the read Parquet files | ||
""" | ||
if table_partition_cols is None: | ||
table_partition_cols = [] | ||
return DataFrame( | ||
SessionContextInternal._global_ctx().read_parquet( | ||
str(path), | ||
table_partition_cols, | ||
parquet_pruning, | ||
file_extension, | ||
skip_metadata, | ||
schema, | ||
file_sort_order, | ||
) | ||
) | ||
|
||
|
||
def read_json( | ||
path: str | pathlib.Path, | ||
schema: pyarrow.Schema | None = None, | ||
schema_infer_max_records: int = 1000, | ||
file_extension: str = ".json", | ||
table_partition_cols: list[tuple[str, str]] | None = None, | ||
file_compression_type: str | None = None, | ||
) -> DataFrame: | ||
"""Read a line-delimited JSON data source. | ||
|
||
Args: | ||
path: Path to the JSON file. | ||
schema: The data source schema. | ||
schema_infer_max_records: Maximum number of rows to read from JSON | ||
files for schema inference if needed. | ||
file_extension: File extension; only files with this extension are | ||
selected for data input. | ||
table_partition_cols: Partition columns. | ||
file_compression_type: File compression type. | ||
|
||
Returns: | ||
DataFrame representation of the read JSON files. | ||
""" | ||
if table_partition_cols is None: | ||
table_partition_cols = [] | ||
return DataFrame( | ||
SessionContextInternal._global_ctx().read_json( | ||
str(path), | ||
schema, | ||
schema_infer_max_records, | ||
file_extension, | ||
table_partition_cols, | ||
file_compression_type, | ||
) | ||
) | ||
|
||
|
||
def read_csv( | ||
path: str | pathlib.Path | list[str] | list[pathlib.Path], | ||
schema: pyarrow.Schema | None = None, | ||
has_header: bool = True, | ||
delimiter: str = ",", | ||
schema_infer_max_records: int = 1000, | ||
file_extension: str = ".csv", | ||
table_partition_cols: list[tuple[str, str]] | None = None, | ||
file_compression_type: str | None = None, | ||
) -> DataFrame: | ||
"""Read a CSV data source. | ||
|
||
Args: | ||
path: Path to the CSV file | ||
schema: An optional schema representing the CSV files. If None, the | ||
CSV reader will try to infer it based on data in file. | ||
has_header: Whether the CSV file have a header. If schema inference | ||
is run on a file with no headers, default column names are | ||
created. | ||
delimiter: An optional column delimiter. | ||
schema_infer_max_records: Maximum number of rows to read from CSV | ||
files for schema inference if needed. | ||
file_extension: File extension; only files with this extension are | ||
selected for data input. | ||
table_partition_cols: Partition columns. | ||
file_compression_type: File compression type. | ||
|
||
Returns: | ||
DataFrame representation of the read CSV files | ||
""" | ||
if table_partition_cols is None: | ||
table_partition_cols = [] | ||
|
||
path = [str(p) for p in path] if isinstance(path, list) else str(path) | ||
|
||
return DataFrame( | ||
SessionContextInternal._global_ctx().read_csv( | ||
path, | ||
schema, | ||
has_header, | ||
delimiter, | ||
schema_infer_max_records, | ||
file_extension, | ||
table_partition_cols, | ||
file_compression_type, | ||
) | ||
) | ||
|
||
|
||
def read_avro( | ||
path: str | pathlib.Path, | ||
schema: pyarrow.Schema | None = None, | ||
file_partition_cols: list[tuple[str, str]] | None = None, | ||
file_extension: str = ".avro", | ||
) -> DataFrame: | ||
"""Create a :py:class:`DataFrame` for reading Avro data source. | ||
|
||
Args: | ||
path: Path to the Avro file. | ||
schema: The data source schema. | ||
file_partition_cols: Partition columns. | ||
file_extension: File extension to select. | ||
|
||
Returns: | ||
DataFrame representation of the read Avro file | ||
""" | ||
if file_partition_cols is None: | ||
file_partition_cols = [] | ||
return DataFrame( | ||
SessionContextInternal._global_ctx().read_avro( | ||
str(path), schema, file_partition_cols, file_extension | ||
) | ||
) |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -23,6 +23,7 @@ | |
import pyarrow.dataset as ds | ||
import pytest | ||
|
||
|
||
from datafusion import ( | ||
DataFrame, | ||
RuntimeConfig, | ||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,97 @@ | ||
# Licensed to the Apache Software Foundation (ASF) under one | ||
# or more contributor license agreements. See the NOTICE file | ||
# distributed with this work for additional information | ||
# regarding copyright ownership. The ASF licenses this file | ||
# to you under the Apache License, Version 2.0 (the | ||
# "License"); you may not use this file except in compliance | ||
# with the License. You may obtain a copy of the License at | ||
# | ||
# http://www.apache.org/licenses/LICENSE-2.0 | ||
# | ||
# Unless required by applicable law or agreed to in writing, | ||
# software distributed under the License is distributed on an | ||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
# KIND, either express or implied. See the License for the | ||
# specific language governing permissions and limitations | ||
# under the License. | ||
import os | ||
import pathlib | ||
|
||
from datafusion import column | ||
import pyarrow as pa | ||
|
||
|
||
from datafusion.io import read_avro, read_csv, read_json, read_parquet | ||
|
||
|
||
def test_read_json_global_ctx(ctx): | ||
path = os.path.dirname(os.path.abspath(__file__)) | ||
|
||
# Default | ||
test_data_path = os.path.join(path, "data_test_context", "data.json") | ||
df = read_json(test_data_path) | ||
result = df.collect() | ||
|
||
assert result[0].column(0) == pa.array(["a", "b", "c"]) | ||
assert result[0].column(1) == pa.array([1, 2, 3]) | ||
|
||
# Schema | ||
schema = pa.schema( | ||
[ | ||
pa.field("A", pa.string(), nullable=True), | ||
] | ||
) | ||
df = read_json(test_data_path, schema=schema) | ||
result = df.collect() | ||
|
||
assert result[0].column(0) == pa.array(["a", "b", "c"]) | ||
assert result[0].schema == schema | ||
|
||
# File extension | ||
test_data_path = os.path.join(path, "data_test_context", "data.json") | ||
df = read_json(test_data_path, file_extension=".json") | ||
result = df.collect() | ||
|
||
assert result[0].column(0) == pa.array(["a", "b", "c"]) | ||
assert result[0].column(1) == pa.array([1, 2, 3]) | ||
|
||
|
||
def test_read_parquet_global(): | ||
parquet_df = read_parquet(path="parquet/data/alltypes_plain.parquet") | ||
parquet_df.show() | ||
assert parquet_df is not None | ||
|
||
path = pathlib.Path.cwd() / "parquet/data/alltypes_plain.parquet" | ||
parquet_df = read_parquet(path=path) | ||
assert parquet_df is not None | ||
|
||
|
||
def test_read_csv(): | ||
csv_df = read_csv(path="testing/data/csv/aggregate_test_100.csv") | ||
csv_df.select(column("c1")).show() | ||
|
||
|
||
def test_read_csv_list(): | ||
csv_df = read_csv(path=["testing/data/csv/aggregate_test_100.csv"]) | ||
expected = csv_df.count() * 2 | ||
|
||
double_csv_df = read_csv( | ||
path=[ | ||
"testing/data/csv/aggregate_test_100.csv", | ||
"testing/data/csv/aggregate_test_100.csv", | ||
] | ||
) | ||
actual = double_csv_df.count() | ||
|
||
double_csv_df.select(column("c1")).show() | ||
assert actual == expected | ||
|
||
|
||
def test_read_avro(): | ||
avro_df = read_avro(path="testing/data/avro/alltypes_plain.avro") | ||
avro_df.show() | ||
assert avro_df is not None | ||
|
||
path = pathlib.Path.cwd() / "testing/data/avro/alltypes_plain.avro" | ||
avro_df = read_avro(path=path) | ||
assert avro_df is not None |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Side note: it would be great to use ruff (https://stackoverflow.com/a/77876298) or isort to deterministically and programmatically sort python imports, and validate that in CI. I think isort/ruff would have a newline here between the third-party and first-party imports.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
there a pre-commit config for ruff linter and formatter
datafusion-python/.pre-commit-config.yaml
Lines 23 to 30 in 79c22d6
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
As the SO answer above explains, import sorting isn't currently part of the default
ruff-format
behavior. We'd need to opt-in by adding anI
element here:datafusion-python/pyproject.toml
Line 66 in 79c22d6