Skip to content

Commit

Permalink
Fixup
Browse files Browse the repository at this point in the history
  • Loading branch information
phofl committed Jul 24, 2024
1 parent ffa770a commit f5ba5a6
Show file tree
Hide file tree
Showing 2 changed files with 14 additions and 4 deletions.
16 changes: 13 additions & 3 deletions dask_bigquery/core.py
Original file line number Diff line number Diff line change
Expand Up @@ -97,6 +97,7 @@ def bigquery_read(
read_kwargs: dict,
arrow_options: dict,
credentials: dict = None,
convert_string: bool = False,
) -> pd.DataFrame:
"""Read a single batch of rows via BQ Storage API, in Arrow binary format.
Expand All @@ -116,7 +117,15 @@ def bigquery_read(
BigQuery Storage API Stream "name"
NOTE: Please set if reading from Storage API without any `row_restriction`.
https://cloud.google.com/bigquery/docs/reference/storage/rpc/google.cloud.bigquery.storage.v1beta1#stream
convert_string: bool
Whether to convert arrow strings directly to arrpw strings in the DataFrame
"""
arrow_options = arrow_options.copy()
if convert_string:
types_mapper = _get_types_mapper(arrow_options.get("types_mapper", {}.get))
if types_mapper is not None:
arrow_options["types_mapper"] = types_mapper

with bigquery_clients(project_id, credentials=credentials) as (_, bqs_client):
session = bqs_client.create_read_session(make_create_read_session_request())
schema = pyarrow.ipc.read_schema(
Expand Down Expand Up @@ -229,19 +238,19 @@ def make_create_read_session_request():
),
)

arrow_options = arrow_options.copy()
arrow_options_meta = arrow_options.copy()
if pyarrow_strings_enabled():
types_mapper = _get_types_mapper(arrow_options.get("types_mapper", {}.get))
if types_mapper is not None:
arrow_options["types_mapper"] = types_mapper
arrow_options_meta["types_mapper"] = types_mapper

# Create a read session in order to detect the schema.
# Read sessions are light weight and will be auto-deleted after 24 hours.
session = bqs_client.create_read_session(make_create_read_session_request())
schema = pyarrow.ipc.read_schema(
pyarrow.py_buffer(session.arrow_schema.serialized_schema)
)
meta = schema.empty_table().to_pandas(**arrow_options)
meta = schema.empty_table().to_pandas(**arrow_options_meta)
print(meta.dtypes)

return dd.from_map(
Expand All @@ -252,6 +261,7 @@ def make_create_read_session_request():
read_kwargs=read_kwargs,
arrow_options=arrow_options,
credentials=credentials,
convert_string=pyarrow_strings_enabled(),
),
[stream.name for stream in session.streams],
meta=meta,
Expand Down
2 changes: 1 addition & 1 deletion dask_bigquery/tests/test_core.py
Original file line number Diff line number Diff line change
Expand Up @@ -391,7 +391,7 @@ def test_arrow_options(table):
@pytest.mark.parametrize("convert_string", [True, False])
def test_convert_string(table, convert_string):
project_id, dataset_id, table_id = table
with dask.config.set({"dask.dataframe.convert-string": convert_string}):
with dask.config.set({"dataframe.convert-string": convert_string}):
ddf = read_gbq(
project_id=project_id,
dataset_id=dataset_id,
Expand Down

0 comments on commit f5ba5a6

Please sign in to comment.