Skip to content

Commit

Permalink
Convert automatically to arrow strings
Browse files Browse the repository at this point in the history
  • Loading branch information
phofl committed Jul 24, 2024
1 parent 6e6bc41 commit 7bec049
Show file tree
Hide file tree
Showing 3 changed files with 56 additions and 0 deletions.
40 changes: 40 additions & 0 deletions dask_bigquery/core.py
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,8 @@
import pandas as pd
import pyarrow
from dask.base import tokenize
from dask.dataframe._compat import PANDAS_GE_220
from dask.dataframe.utils import pyarrow_strings_enabled
from google.api_core import client_info as rest_client_info
from google.api_core import exceptions
from google.api_core.gapic_v1 import client_info as grpc_client_info
Expand Down Expand Up @@ -130,6 +132,37 @@ def bigquery_read(
return pd.concat(shards)


def _get_types_mapper(user_mapper):
type_mappers = []

# always use the user-defined mapper first, if available
if user_mapper is not None:
type_mappers.append(user_mapper)

type_mappers.append({pyarrow.string(): pd.StringDtype("pyarrow")}.get)
if PANDAS_GE_220:
type_mappers.append({pyarrow.large_string(): pd.StringDtype("pyarrow")}.get)
type_mappers.append({pyarrow.date32(): pd.ArrowDtype(pyarrow.date32())}.get)
type_mappers.append({pyarrow.date64(): pd.ArrowDtype(pyarrow.date64())}.get)

def _convert_decimal_type(type):
if pyarrow.types.is_decimal(type):
return pd.ArrowDtype(type)
return None

type_mappers.append(_convert_decimal_type)

def default_types_mapper(pyarrow_dtype):
"""Try all type mappers in order, starting from the user type mapper."""
for type_converter in type_mappers:
converted_type = type_converter(pyarrow_dtype)
if converted_type is not None:
return converted_type

if len(type_mappers) > 0:
return default_types_mapper


def read_gbq(
project_id: str,
dataset_id: str,
Expand Down Expand Up @@ -196,13 +229,20 @@ def make_create_read_session_request():
),
)

arrow_options = arrow_options.copy()
if pyarrow_strings_enabled():
types_mapper = _get_types_mapper(arrow_options.get("types_mapper", {}))
if types_mapper is not None:
arrow_options["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)
print(meta.dtypes)

return dd.from_map(
partial(
Expand Down
Empty file added dask_bigquery/tests/__init__.py
Empty file.
16 changes: 16 additions & 0 deletions dask_bigquery/tests/test_core.py
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@
import uuid
from datetime import datetime, timedelta, timezone

import dask
import dask.dataframe as dd
import gcsfs
import google.auth
Expand Down Expand Up @@ -387,6 +388,21 @@ def test_arrow_options(table):
assert ddf.dtypes["name"] == pd.StringDtype(storage="pyarrow")


@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}):
ddf = read_gbq(
project_id=project_id,
dataset_id=dataset_id,
table_id=table_id,
)
if convert_string:
assert ddf.dtypes["name"] == pd.StringDtype(storage="pyarrow")
else:
assert ddf.dtypes["name"] == object


@pytest.mark.skipif(sys.platform == "darwin", reason="Segfaults on macOS")
def test_read_required_partition_filter(df, required_partition_filter_table):
project_id, dataset_id, table_id = required_partition_filter_table
Expand Down

0 comments on commit 7bec049

Please sign in to comment.