-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-54317][PYTHON][CONNECT] Unify Arrow conversion logic for Classic and Connect toPandas #53045
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
Changes from all commits
6a42779
4ac2656
28e755b
2037aa8
a76043d
331eb41
9dd85d2
bd77465
1f8049f
1a5fad9
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -99,8 +99,9 @@ | |
| ) | ||
| from pyspark.sql.connect.observation import Observation | ||
| from pyspark.sql.connect.utils import get_python_ver | ||
| from pyspark.sql.pandas.types import _create_converter_to_pandas, from_arrow_schema | ||
| from pyspark.sql.types import DataType, StructType, _has_type | ||
| from pyspark.sql.pandas.types import from_arrow_schema | ||
| from pyspark.sql.pandas.conversion import _convert_arrow_table_to_pandas | ||
| from pyspark.sql.types import DataType, StructType | ||
| from pyspark.util import PythonEvalType | ||
| from pyspark.storagelevel import StorageLevel | ||
| from pyspark.errors import PySparkValueError, PySparkAssertionError, PySparkNotImplementedError | ||
|
|
@@ -984,88 +985,31 @@ def to_pandas( | |
| # Get all related configs in a batch | ||
| ( | ||
| timezone, | ||
| struct_in_pandas, | ||
| self_destruct, | ||
| structHandlingMode, | ||
| selfDestruct, | ||
| ) = self.get_configs( | ||
| "spark.sql.session.timeZone", | ||
| "spark.sql.execution.pandas.structHandlingMode", | ||
| "spark.sql.execution.arrow.pyspark.selfDestruct.enabled", | ||
| ) | ||
|
|
||
| table, schema, metrics, observed_metrics, _ = self._execute_and_fetch( | ||
| req, observations, self_destruct == "true" | ||
| req, observations, selfDestruct == "true" | ||
| ) | ||
| assert table is not None | ||
| ei = ExecutionInfo(metrics, observed_metrics) | ||
|
|
||
| schema = schema or from_arrow_schema(table.schema, prefer_timestamp_ntz=True) | ||
| assert schema is not None and isinstance(schema, StructType) | ||
|
|
||
| # Rename columns to avoid duplicated column names during processing | ||
| temp_col_names = [f"col_{i}" for i in range(len(schema.names))] | ||
| table = table.rename_columns(temp_col_names) | ||
|
|
||
| # Pandas DataFrame created from PyArrow uses datetime64[ns] for date type | ||
| # values, but we should use datetime.date to match the behavior with when | ||
| # Arrow optimization is disabled. | ||
| pandas_options = {"coerce_temporal_nanoseconds": True} | ||
| if self_destruct == "true" and table.num_rows > 0: | ||
| # Configure PyArrow to use as little memory as possible: | ||
| # self_destruct - free columns as they are converted | ||
| # split_blocks - create a separate Pandas block for each column | ||
| # use_threads - convert one column at a time | ||
| pandas_options.update( | ||
| { | ||
| "self_destruct": True, | ||
| "split_blocks": True, | ||
| "use_threads": False, | ||
| } | ||
| ) | ||
|
|
||
| if len(schema.names) > 0: | ||
| error_on_duplicated_field_names: bool = False | ||
| if struct_in_pandas == "legacy" and any( | ||
| _has_type(f.dataType, StructType) for f in schema.fields | ||
| ): | ||
| error_on_duplicated_field_names = True | ||
| struct_in_pandas = "dict" | ||
|
|
||
| # SPARK-51112: If the table is empty, we avoid using pyarrow to_pandas to create the | ||
| # DataFrame, as it may fail with a segmentation fault. | ||
| if table.num_rows == 0: | ||
| # For empty tables, create empty Series with converters to preserve dtypes | ||
| pdf = pd.concat( | ||
| [ | ||
| _create_converter_to_pandas( | ||
| field.dataType, | ||
| field.nullable, | ||
| timezone=timezone, | ||
| struct_in_pandas=struct_in_pandas, | ||
| error_on_duplicated_field_names=error_on_duplicated_field_names, | ||
| )(pd.Series([], name=temp_col_names[i], dtype="object")) | ||
| for i, field in enumerate(schema.fields) | ||
| ], | ||
| axis="columns", | ||
| ) | ||
| else: | ||
| pdf = pd.concat( | ||
| [ | ||
| _create_converter_to_pandas( | ||
| field.dataType, | ||
| field.nullable, | ||
| timezone=timezone, | ||
| struct_in_pandas=struct_in_pandas, | ||
| error_on_duplicated_field_names=error_on_duplicated_field_names, | ||
| )(arrow_col.to_pandas(**pandas_options)) | ||
| for arrow_col, field in zip(table.columns, schema.fields) | ||
| ], | ||
| axis="columns", | ||
| ) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Could we not also unify the
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yes after reading more, I moved more common logic into the helper function to unify them. Thanks! |
||
| # Restore original column names (including duplicates) | ||
| pdf.columns = schema.names | ||
| else: | ||
| # empty columns | ||
| pdf = table.to_pandas(**pandas_options) | ||
| pdf = _convert_arrow_table_to_pandas( | ||
| arrow_table=table, | ||
| schema=schema, | ||
| timezone=timezone, | ||
| struct_handling_mode=structHandlingMode, | ||
| date_as_object=False, | ||
| self_destruct=selfDestruct == "true", | ||
| ) | ||
|
|
||
| if len(metrics) > 0: | ||
| pdf.attrs["metrics"] = metrics | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.