Skip to content

Commit

Permalink
[SPARK-13534][PYSPARK] Using Apache Arrow to increase performance of …
Browse files Browse the repository at this point in the history
…DataFrame.toPandas

## What changes were proposed in this pull request?
Integrate Apache Arrow with Spark to increase performance of `DataFrame.toPandas`.  This has been done by using Arrow to convert data partitions on the executor JVM to Arrow payload byte arrays where they are then served to the Python process.  The Python DataFrame can then collect the Arrow payloads where they are combined and converted to a Pandas DataFrame.  Data types except complex, date, timestamp, and decimal  are currently supported, otherwise an `UnsupportedOperation` exception is thrown.

Additions to Spark include a Scala package private method `Dataset.toArrowPayload` that will convert data partitions in the executor JVM to `ArrowPayload`s as byte arrays so they can be easily served.  A package private class/object `ArrowConverters` that provide data type mappings and conversion routines.  In Python, a private method `DataFrame._collectAsArrow` is added to collect Arrow payloads and a SQLConf "spark.sql.execution.arrow.enable" can be used in `toPandas()` to enable using Arrow (uses the old conversion by default).

## How was this patch tested?
Added a new test suite `ArrowConvertersSuite` that will run tests on conversion of Datasets to Arrow payloads for supported types.  The suite will generate a Dataset and matching Arrow JSON data, then the dataset is converted to an Arrow payload and finally validated against the JSON data.  This will ensure that the schema and data has been converted correctly.

Added PySpark tests to verify the `toPandas` method is producing equal DataFrames with and without pyarrow.  A roundtrip test to ensure the pandas DataFrame produced by pyspark is equal to a one made directly with pandas.

Author: Bryan Cutler <cutlerb@gmail.com>
Author: Li Jin <ice.xelloss@gmail.com>
Author: Li Jin <li.jin@twosigma.com>
Author: Wes McKinney <wes.mckinney@twosigma.com>

Closes #18459 from BryanCutler/toPandas_with_arrow-SPARK-13534.
  • Loading branch information
BryanCutler authored and holdenk committed Jul 10, 2017
1 parent 2bfd5ac commit d03aebb
Show file tree
Hide file tree
Showing 12 changed files with 1,859 additions and 13 deletions.
2 changes: 1 addition & 1 deletion bin/pyspark
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,7 @@ if [[ -n "$SPARK_TESTING" ]]; then
unset YARN_CONF_DIR
unset HADOOP_CONF_DIR
export PYTHONHASHSEED=0
exec "$PYSPARK_DRIVER_PYTHON" -m "$1"
exec "$PYSPARK_DRIVER_PYTHON" -m "$@"
exit
fi

Expand Down
5 changes: 5 additions & 0 deletions dev/deps/spark-deps-hadoop-2.6
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,9 @@ apacheds-kerberos-codec-2.0.0-M15.jar
api-asn1-api-1.0.0-M20.jar
api-util-1.0.0-M20.jar
arpack_combined_all-0.1.jar
arrow-format-0.4.0.jar
arrow-memory-0.4.0.jar
arrow-vector-0.4.0.jar
avro-1.7.7.jar
avro-ipc-1.7.7.jar
avro-mapred-1.7.7-hadoop2.jar
Expand Down Expand Up @@ -55,6 +58,7 @@ datanucleus-core-3.2.10.jar
datanucleus-rdbms-3.2.9.jar
derby-10.12.1.1.jar
eigenbase-properties-1.1.5.jar
flatbuffers-1.2.0-3f79e055.jar
gson-2.2.4.jar
guava-14.0.1.jar
guice-3.0.jar
Expand All @@ -77,6 +81,7 @@ hadoop-yarn-server-web-proxy-2.6.5.jar
hk2-api-2.4.0-b34.jar
hk2-locator-2.4.0-b34.jar
hk2-utils-2.4.0-b34.jar
hppc-0.7.1.jar
htrace-core-3.0.4.jar
httpclient-4.5.2.jar
httpcore-4.4.4.jar
Expand Down
5 changes: 5 additions & 0 deletions dev/deps/spark-deps-hadoop-2.7
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,9 @@ apacheds-kerberos-codec-2.0.0-M15.jar
api-asn1-api-1.0.0-M20.jar
api-util-1.0.0-M20.jar
arpack_combined_all-0.1.jar
arrow-format-0.4.0.jar
arrow-memory-0.4.0.jar
arrow-vector-0.4.0.jar
avro-1.7.7.jar
avro-ipc-1.7.7.jar
avro-mapred-1.7.7-hadoop2.jar
Expand Down Expand Up @@ -55,6 +58,7 @@ datanucleus-core-3.2.10.jar
datanucleus-rdbms-3.2.9.jar
derby-10.12.1.1.jar
eigenbase-properties-1.1.5.jar
flatbuffers-1.2.0-3f79e055.jar
gson-2.2.4.jar
guava-14.0.1.jar
guice-3.0.jar
Expand All @@ -77,6 +81,7 @@ hadoop-yarn-server-web-proxy-2.7.3.jar
hk2-api-2.4.0-b34.jar
hk2-locator-2.4.0-b34.jar
hk2-utils-2.4.0-b34.jar
hppc-0.7.1.jar
htrace-core-3.1.0-incubating.jar
httpclient-4.5.2.jar
httpcore-4.4.4.jar
Expand Down
20 changes: 20 additions & 0 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -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>
Expand Down Expand Up @@ -1878,6 +1879,25 @@
<artifactId>paranamer</artifactId>
<version>${paranamer.version}</version>
</dependency>
<dependency>
<groupId>org.apache.arrow</groupId>
<artifactId>arrow-vector</artifactId>
<version>${arrow.version}</version>
<exclusions>
<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>

Expand Down
17 changes: 17 additions & 0 deletions python/pyspark/serializers.py
Original file line number Diff line number Diff line change
Expand Up @@ -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()

def __repr__(self):
return "ArrowSerializer"


class BatchedSerializer(Serializer):

"""
Expand Down
48 changes: 37 additions & 11 deletions python/pyspark/sql/dataframe.py
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,8 @@

from pyspark import copy_func, since
from pyspark.rdd import RDD, _load_from_socket, ignore_unicode_prefix
from pyspark.serializers import BatchedSerializer, PickleSerializer, UTF8Deserializer
from pyspark.serializers import ArrowSerializer, BatchedSerializer, PickleSerializer, \
UTF8Deserializer
from pyspark.storagelevel import StorageLevel
from pyspark.traceback_utils import SCCallSiteSync
from pyspark.sql.types import _parse_datatype_json_string
Expand Down Expand Up @@ -1710,7 +1711,8 @@ def toDF(self, *cols):

@since(1.3)
def toPandas(self):
"""Returns the contents of this :class:`DataFrame` as Pandas ``pandas.DataFrame``.
"""
Returns the contents of this :class:`DataFrame` as Pandas ``pandas.DataFrame``.
This is only available if Pandas is installed and available.
Expand All @@ -1723,18 +1725,42 @@ def toPandas(self):
1 5 Bob
"""
import pandas as pd
if self.sql_ctx.getConf("spark.sql.execution.arrow.enable", "false").lower() == "true":
try:
import pyarrow
tables = self._collectAsArrow()
if tables:
table = pyarrow.concat_tables(tables)
return table.to_pandas()
else:
return pd.DataFrame.from_records([], columns=self.columns)
except ImportError as e:
msg = "note: pyarrow must be installed and available on calling Python process " \
"if using spark.sql.execution.arrow.enable=true"
raise ImportError("%s\n%s" % (e.message, msg))
else:
dtype = {}
for field in self.schema:
pandas_type = _to_corrected_pandas_type(field.dataType)
if pandas_type is not None:
dtype[field.name] = pandas_type

dtype = {}
for field in self.schema:
pandas_type = _to_corrected_pandas_type(field.dataType)
if pandas_type is not None:
dtype[field.name] = pandas_type
pdf = pd.DataFrame.from_records(self.collect(), columns=self.columns)

pdf = pd.DataFrame.from_records(self.collect(), columns=self.columns)
for f, t in dtype.items():
pdf[f] = pdf[f].astype(t, copy=False)
return pdf

for f, t in dtype.items():
pdf[f] = pdf[f].astype(t, copy=False)
return pdf
def _collectAsArrow(self):
"""
Returns all records as list of deserialized ArrowPayloads, pyarrow must be installed
and available.
.. note:: Experimental.
"""
with SCCallSiteSync(self._sc) as css:
port = self._jdf.collectAsArrowToPython()
return list(_load_from_socket(port, ArrowSerializer()))

##########################################################################################
# Pandas compatibility
Expand Down
78 changes: 77 additions & 1 deletion python/pyspark/sql/tests.py
Original file line number Diff line number Diff line change
Expand Up @@ -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, _make_type_verifier
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
except:
# No Arrow, but that's okay, we'll skip those tests
pass


class UTCOffsetTimezone(datetime.tzinfo):
"""
Specifies timezone in UTC offset
Expand Down Expand Up @@ -2843,6 +2852,73 @@ def __init__(self, **kwargs):
_make_type_verifier(data_type, nullable=False)(obj)


@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()
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:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -855,6 +855,24 @@ object SQLConf {
.intConf
.createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt)

val ARROW_EXECUTION_ENABLE =
buildConf("spark.sql.execution.arrow.enable")
.internal()
.doc("Make use of Apache Arrow for columnar data transfers. Currently available " +
"for use with pyspark.sql.DataFrame.toPandas with the following data types: " +
"StringType, BinaryType, BooleanType, DoubleType, FloatType, ByteType, IntegerType, " +
"LongType, ShortType")
.booleanConf
.createWithDefault(false)

val ARROW_EXECUTION_MAX_RECORDS_PER_BATCH =
buildConf("spark.sql.execution.arrow.maxRecordsPerBatch")
.internal()
.doc("When using Apache Arrow, limit the maximum number of records that can be written " +
"to a single ArrowRecordBatch in memory. If set to zero or negative there is no limit.")
.intConf
.createWithDefault(10000)

object Deprecated {
val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks"
}
Expand Down Expand Up @@ -1115,6 +1133,10 @@ class SQLConf extends Serializable with Logging {

def starSchemaFTRatio: Double = getConf(STARSCHEMA_FACT_TABLE_RATIO)

def arrowEnable: Boolean = getConf(ARROW_EXECUTION_ENABLE)

def arrowMaxRecordsPerBatch: Int = getConf(ARROW_EXECUTION_MAX_RECORDS_PER_BATCH)

/** ********************** SQLConf functionality methods ************ */

/** Set Spark SQL configuration properties. */
Expand Down
4 changes: 4 additions & 0 deletions sql/core/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -103,6 +103,10 @@
<artifactId>jackson-databind</artifactId>
<version>${fasterxml.jackson.version}</version>
</dependency>
<dependency>
<groupId>org.apache.arrow</groupId>
<artifactId>arrow-vector</artifactId>
</dependency>
<dependency>
<groupId>org.apache.xbean</groupId>
<artifactId>xbean-asm5-shaded</artifactId>
Expand Down
20 changes: 20 additions & 0 deletions sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection}
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.arrow.{ArrowConverters, ArrowPayload}
import org.apache.spark.sql.execution.command._
import org.apache.spark.sql.execution.datasources.LogicalRelation
import org.apache.spark.sql.execution.python.EvaluatePython
Expand Down Expand Up @@ -2907,6 +2908,16 @@ class Dataset[T] private[sql](
}
}

/**
* Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
*/
private[sql] def collectAsArrowToPython(): Int = {
withNewExecutionId {
val iter = toArrowPayload.collect().iterator.map(_.asPythonSerializable)
PythonRDD.serveIterator(iter, "serve-Arrow")
}
}

private[sql] def toPythonIterator(): Int = {
withNewExecutionId {
PythonRDD.toLocalIteratorAndServe(javaToPython.rdd)
Expand Down Expand Up @@ -2988,4 +2999,13 @@ class Dataset[T] private[sql](
Dataset(sparkSession, logicalPlan)
}
}

/** Convert to an RDD of ArrowPayload byte arrays */
private[sql] def toArrowPayload: RDD[ArrowPayload] = {
val schemaCaptured = this.schema
val maxRecordsPerBatch = sparkSession.sessionState.conf.arrowMaxRecordsPerBatch
queryExecution.toRdd.mapPartitionsInternal { iter =>
ArrowConverters.toPayloadIterator(iter, schemaCaptured, maxRecordsPerBatch)
}
}
}
Loading

0 comments on commit d03aebb

Please sign in to comment.