-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-41114] [CONNECT] [PYTHON] [FOLLOW-UP] Python Client support for local data #38803
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 4 commits
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 |
|---|---|---|
|
|
@@ -25,7 +25,8 @@ | |
| TYPE_CHECKING, | ||
| Mapping, | ||
| ) | ||
|
|
||
| import pandas | ||
| import pyarrow as pa | ||
| import pyspark.sql.connect.proto as proto | ||
| from pyspark.sql.connect.column import ( | ||
| Column, | ||
|
|
@@ -177,6 +178,37 @@ def _repr_html_(self) -> str: | |
| """ | ||
|
|
||
|
|
||
| class LocalRelation(LogicalPlan): | ||
| """Creates a LocalRelation plan object based on a Pandas DataFrame.""" | ||
|
|
||
| def __init__(self, pdf: "pandas.DataFrame") -> None: | ||
| super().__init__(None) | ||
| self._pdf = pdf | ||
|
|
||
| def plan(self, session: "SparkConnectClient") -> proto.Relation: | ||
| assert self._pdf is not None | ||
|
|
||
| sink = pa.BufferOutputStream() | ||
| table = pa.Table.from_pandas(self._pdf) | ||
| with pa.ipc.new_stream(sink, table.schema) as writer: | ||
|
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. I am not familiar here so a question: any possible that an empty panda dataframe are used here (e.g. has schema but no data). If so maybe have a test case?
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. I'll add a test for that, thanks for the proposal! |
||
| for b in table.to_batches(): | ||
| writer.write_batch(b) | ||
|
|
||
| plan = proto.Relation() | ||
| plan.local_relation.data = sink.getvalue().to_pybytes() | ||
| return plan | ||
|
|
||
| def print(self, indent: int = 0) -> str: | ||
| return f"{' ' * indent}<LocalRelation>\n" | ||
|
|
||
| def _repr_html_(self) -> str: | ||
| return """ | ||
| <ul> | ||
| <li>LocalRelation</li> | ||
| </ul> | ||
| """ | ||
|
|
||
|
|
||
| class ShowString(LogicalPlan): | ||
| def __init__( | ||
| self, child: Optional["LogicalPlan"], numRows: int, truncate: int, vertical: bool | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,13 +17,15 @@ | |
|
|
||
| from threading import RLock | ||
| from typing import Optional, Any, Union, Dict, cast, overload | ||
| import pandas as pd | ||
|
|
||
| import pyspark.sql.types | ||
| from pyspark.sql.connect.client import SparkConnectClient | ||
| from pyspark.sql.connect.dataframe import DataFrame | ||
| from pyspark.sql.connect.plan import SQL, Range | ||
| from pyspark.sql.connect.readwriter import DataFrameReader | ||
| from pyspark.sql.utils import to_str | ||
| from . import plan | ||
| from ._typing import OptionalPrimitiveType | ||
|
|
||
|
|
||
|
|
@@ -205,6 +207,31 @@ def __init__(self, connectionString: str, userId: Optional[str] = None): | |
| # Create the reader | ||
| self.read = DataFrameReader(self) | ||
|
|
||
| def createDataFrame(self, data: "pd.DataFrame") -> "DataFrame": | ||
|
Member
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. Actually, the implementation here isn't matched to what we have in By default, the Arrow message conversion (more specifically in https://github.com/apache/spark/pull/38659/files#diff-d630cc4be6c65a3c3f7d6dbfe990f99ba992ccc26d9c3aaf6cfe46e163cb7389R514-R521) have to happen in RDD so we can parallelize this. For a bit of history, PySpark added the initial version with RDD first, and added this local relation as an optimization for small dataset (see also #36683) later.
Member
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. I am fine with the current approach but the main problem here is that 1. we can't stream the input, 2. it will have the size limit (likely 4KB). cc @hvanhovell FYI
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. It is impossible to match the implementation because in Pyspark to parallelize a first serialization is already happening to pass the input DF to the executors. In our case to even send the data to spark we have to serialize it. That said you're right that this currently does not support streaming of local data to the client. But the limit is not 4kb but probably whatever the max message size of GRPC is so in the megabytes. I think we need to add the client side streaming APIs at some point but I'd like to defer that for a bit.
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. For a large |
||
| """ | ||
| Creates a :class:`DataFrame` from a :class:`pandas.DataFrame`. | ||
|
|
||
| .. versionadded:: 3.4.0 | ||
|
|
||
|
|
||
| Parameters | ||
| ---------- | ||
| data : :class:`pandas.DataFrame` | ||
|
|
||
| Returns | ||
| ------- | ||
| :class:`DataFrame` | ||
|
|
||
| Examples | ||
| -------- | ||
| >>> import pandas | ||
| >>> pdf = pandas.DataFrame({"a": [1, 2, 3], "b": ["a", "b", "c"]}) | ||
| >>> self.connect.createDataFrame(pdf).collect() | ||
| [Row(a=1, b='a'), Row(a=2, b='b'), Row(a=3, b='c')] | ||
|
|
||
| """ | ||
| return DataFrame.withPlan(plan.LocalRelation(data), self) | ||
|
|
||
| @property | ||
| def client(self) -> "SparkConnectClient": | ||
| """ | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.
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.
Nit: is this a bit redundant though that
plan.pyis internal API, the constructor does not accepts Optional pandas dataframe and we have mypy to do type checking?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.
I think you're right. It makes sense to move the assertion into the session.
As an FYI, all of the mypy checks are really just for the code that we write. During runtime, the user can pass whatever they want and we should make sure that we have proper checks for it. But since plan is internal API it makes a lot of sense to have the checking on the public API instead.