-
Notifications
You must be signed in to change notification settings - Fork 28.1k
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
[SPARK-13534][PySpark] Using Apache Arrow to increase performance of DataFrame.toPandas #15821
Changes from all commits
f681d52
afd5739
a4b958e
be508a5
5dbad22
5837b38
bdba357
d20437f
2e81a93
1ce4f2d
ed1f0fa
202650e
fbe3b7c
f44e6d7
e0bf11b
3090a3e
54884ed
42af1d5
9c8ea63
b7c28ad
2851cd6
f8f24ab
b6c752b
cbab294
44ca3ff
33b75b9
97742b8
b821077
3d786a2
cb4c510
074f66c
7260217
a0483b8
470f33d
c144667
250b581
f667a7a
76f7ddb
89dd0f4
d7cb4ab
b6fe733
36f8127
088f79e
e0449eb
b6bfcd7
2c1af59
1d471ac
a4d6057
934c147
2e4747b
b4eebc2
d49a14d
a630bf0
748e6fb
b361bdc
8bff966
f96f555
b53e09f
44d7a2a
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 |
---|---|---|
|
@@ -181,6 +181,7 @@ | |
<paranamer.version>2.6</paranamer.version> | ||
<maven-antrun.version>1.8</maven-antrun.version> | ||
<commons-crypto.version>1.0.0</commons-crypto.version> | ||
<arrow.version>0.4.0</arrow.version> | ||
|
||
<test.java.home>${java.home}</test.java.home> | ||
<test.exclude.tags></test.exclude.tags> | ||
|
@@ -1878,6 +1879,25 @@ | |
<artifactId>paranamer</artifactId> | ||
<version>${paranamer.version}</version> | ||
</dependency> | ||
<dependency> | ||
<groupId>org.apache.arrow</groupId> | ||
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. why we add arrow dependency at root instead of only spark sql? 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 think this is just standard pom .. 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. This is just the dependency management section in the main pom. The only actual dependency is in spark-sql. |
||
<artifactId>arrow-vector</artifactId> | ||
<version>${arrow.version}</version> | ||
<exclusions> | ||
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. Should we consider explore excluding netty here, since we exclude it in most of the other related projects (like parquet), since it seems to have added some unnecessary jars to the deps list. 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 added netty to exclusions and does not seem to cause any issues |
||
<exclusion> | ||
<groupId>com.fasterxml.jackson.core</groupId> | ||
<artifactId>jackson-annotations</artifactId> | ||
</exclusion> | ||
<exclusion> | ||
<groupId>com.fasterxml.jackson.core</groupId> | ||
<artifactId>jackson-databind</artifactId> | ||
</exclusion> | ||
<exclusion> | ||
<groupId>io.netty</groupId> | ||
<artifactId>netty-handler</artifactId> | ||
</exclusion> | ||
</exclusions> | ||
</dependency> | ||
</dependencies> | ||
</dependencyManagement> | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -182,6 +182,23 @@ def loads(self, obj): | |
raise NotImplementedError | ||
|
||
|
||
class ArrowSerializer(FramedSerializer): | ||
""" | ||
Serializes an Arrow stream. | ||
""" | ||
|
||
def dumps(self, obj): | ||
raise NotImplementedError | ||
|
||
def loads(self, obj): | ||
import pyarrow as pa | ||
reader = pa.RecordBatchFileReader(pa.BufferReader(obj)) | ||
return reader.read_all() | ||
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. since we are sending multiple batches from JVM, does 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. This will read all batches in a framed byte array from a stream and return. The stream can have multiple framed byte arrays, so it repeats until end of stream. How many batches this reads depends on how it serialized. When calling |
||
|
||
def __repr__(self): | ||
return "ArrowSerializer" | ||
|
||
|
||
class BatchedSerializer(Serializer): | ||
|
||
""" | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -58,12 +58,21 @@ | |
from pyspark.sql import SparkSession, SQLContext, HiveContext, Column, Row | ||
from pyspark.sql.types import * | ||
from pyspark.sql.types import UserDefinedType, _infer_type | ||
from pyspark.tests import ReusedPySparkTestCase, SparkSubmitTests | ||
from pyspark.tests import QuietTest, ReusedPySparkTestCase, SparkSubmitTests | ||
from pyspark.sql.functions import UserDefinedFunction, sha2, lit | ||
from pyspark.sql.window import Window | ||
from pyspark.sql.utils import AnalysisException, ParseException, IllegalArgumentException | ||
|
||
|
||
_have_arrow = False | ||
try: | ||
import pyarrow | ||
_have_arrow = True | ||
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. We should do similar thing above when using Arrow required feature, e.g., ArrowSerializer. 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. Do you mean to automatically enable the Arrow functionality if pyarrow installed? Right now it is enabled manually with a flag 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 mean we should throw an exception when 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. Maybe give the param doc string as exception message? I.e., |
||
except: | ||
# No Arrow, but that's okay, we'll skip those tests | ||
pass | ||
|
||
|
||
class UTCOffsetTimezone(datetime.tzinfo): | ||
""" | ||
Specifies timezone in UTC offset | ||
|
@@ -2620,6 +2629,74 @@ def range_frame_match(): | |
|
||
importlib.reload(window) | ||
|
||
|
||
@unittest.skipIf(not _have_arrow, "Arrow not installed") | ||
class ArrowTests(ReusedPySparkTestCase): | ||
|
||
@classmethod | ||
def setUpClass(cls): | ||
ReusedPySparkTestCase.setUpClass() | ||
cls.spark = SparkSession(cls.sc) | ||
cls.spark.conf.set("spark.sql.execution.arrow.enable", "true") | ||
cls.schema = StructType([ | ||
StructField("1_str_t", StringType(), True), | ||
StructField("2_int_t", IntegerType(), True), | ||
StructField("3_long_t", LongType(), True), | ||
StructField("4_float_t", FloatType(), True), | ||
StructField("5_double_t", DoubleType(), True)]) | ||
cls.data = [("a", 1, 10, 0.2, 2.0), | ||
("b", 2, 20, 0.4, 4.0), | ||
("c", 3, 30, 0.8, 6.0)] | ||
|
||
def assertFramesEqual(self, df_with_arrow, df_without): | ||
msg = ("DataFrame from Arrow is not equal" + | ||
("\n\nWith Arrow:\n%s\n%s" % (df_with_arrow, df_with_arrow.dtypes)) + | ||
("\n\nWithout:\n%s\n%s" % (df_without, df_without.dtypes))) | ||
self.assertTrue(df_without.equals(df_with_arrow), msg=msg) | ||
|
||
def test_unsupported_datatype(self): | ||
schema = StructType([StructField("array", ArrayType(IntegerType(), False), True)]) | ||
df = self.spark.createDataFrame([([1, 2, 3],)], schema=schema) | ||
with QuietTest(self.sc): | ||
self.assertRaises(Exception, lambda: df.toPandas()) | ||
|
||
def test_null_conversion(self): | ||
df_null = self.spark.createDataFrame([tuple([None for _ in range(len(self.data[0]))])] + | ||
self.data) | ||
pdf = df_null.toPandas() | ||
null_counts = pdf.isnull().sum().tolist() | ||
self.assertTrue(all([c == 1 for c in null_counts])) | ||
|
||
def test_toPandas_arrow_toggle(self): | ||
df = self.spark.createDataFrame(self.data, schema=self.schema) | ||
self.spark.conf.set("spark.sql.execution.arrow.enable", "false") | ||
pdf = df.toPandas() | ||
self.spark.conf.set("spark.sql.execution.arrow.enable", "true") | ||
pdf_arrow = df.toPandas() | ||
self.assertFramesEqual(pdf_arrow, pdf) | ||
|
||
def test_pandas_round_trip(self): | ||
import pandas as pd | ||
import numpy as np | ||
data_dict = {} | ||
for j, name in enumerate(self.schema.names): | ||
data_dict[name] = [self.data[i][j] for i in range(len(self.data))] | ||
# need to convert these to numpy types first | ||
data_dict["2_int_t"] = np.int32(data_dict["2_int_t"]) | ||
data_dict["4_float_t"] = np.float32(data_dict["4_float_t"]) | ||
pdf = pd.DataFrame(data=data_dict) | ||
df = self.spark.createDataFrame(self.data, schema=self.schema) | ||
pdf_arrow = df.toPandas() | ||
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. call 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. that's done in the |
||
self.assertFramesEqual(pdf_arrow, pdf) | ||
|
||
def test_filtered_frame(self): | ||
df = self.spark.range(3).toDF("i") | ||
pdf = df.filter("i < 0").toPandas() | ||
self.assertEqual(len(pdf.columns), 1) | ||
self.assertEqual(pdf.columns[0], "i") | ||
self.assertTrue(pdf.empty) | ||
|
||
|
||
if __name__ == "__main__": | ||
from pyspark.sql.tests import * | ||
if xmlrunner: | ||
|
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.
This is just a temporary addition to make sure pyarrow tests run because they required a conda env, for now
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.
will we remove this at some time?
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.
pyarrow has pip packages now, does that help? @BryanCutler I can't read your comment and understand why they required a conda env but maybe that's it?
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.
Yeah, this will definitely be removed at some point. I was working with @holdenk to set this up as a temporary way to get the Python Arrow tests to run. I'll look into using the pip packages and see if that can be used instead of this. Thanks @leifwalsh!