From 2e0b308ef8e4829171f97298fb340e060c410747 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Thu, 20 Jun 2019 18:15:17 +0100 Subject: [PATCH 01/34] initial commit of cogroup --- .../spark/api/python/PythonRunner.scala | 2 + python/pyspark/rdd.py | 1 + python/pyspark/serializers.py | 34 ++++ python/pyspark/sql/cogroup.py | 15 ++ python/pyspark/sql/functions.py | 3 + python/pyspark/sql/group.py | 4 + .../tests/test_pandas_udf_cogrouped_map.py | 113 +++++++++++ python/pyspark/worker.py | 39 +++- .../logical/pythonLogicalOperators.scala | 12 ++ .../spark/sql/RelationalGroupedDataset.scala | 31 ++- .../spark/sql/execution/SparkStrategies.scala | 3 + .../python/FlatMapCoGroupsInPandasExec.scala | 93 +++++++++ .../python/FlatMapGroupsInPandasExec.scala | 16 +- .../python/InterleavedArrowWriter.scala | 82 ++++++++ .../python/MultiDfArrowPythonRunner.scala | 191 ++++++++++++++++++ 15 files changed, 631 insertions(+), 8 deletions(-) create mode 100644 python/pyspark/sql/cogroup.py create mode 100644 python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowWriter.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/python/MultiDfArrowPythonRunner.scala diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index 4dcc5eb5fbfcd..fe28ec6fc1c29 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -46,6 +46,7 @@ private[spark] object PythonEvalType { val SQL_GROUPED_MAP_PANDAS_UDF = 201 val SQL_GROUPED_AGG_PANDAS_UDF = 202 val SQL_WINDOW_AGG_PANDAS_UDF = 203 + val SQL_COGROUPED_MAP_PANDAS_UDF = 204 def toString(pythonEvalType: Int): String = pythonEvalType match { case NON_UDF => "NON_UDF" @@ -54,6 +55,7 @@ private[spark] object PythonEvalType { case SQL_GROUPED_MAP_PANDAS_UDF => "SQL_GROUPED_MAP_PANDAS_UDF" case SQL_GROUPED_AGG_PANDAS_UDF => "SQL_GROUPED_AGG_PANDAS_UDF" case SQL_WINDOW_AGG_PANDAS_UDF => "SQL_WINDOW_AGG_PANDAS_UDF" + case SQL_COGROUPED_MAP_PANDAS_UDF => "SQL_COGROUPED_MAP_PANDAS_UDF" } } diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index f0682e71a1780..f198316d0c4a5 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -73,6 +73,7 @@ class PythonEvalType(object): SQL_GROUPED_MAP_PANDAS_UDF = 201 SQL_GROUPED_AGG_PANDAS_UDF = 202 SQL_WINDOW_AGG_PANDAS_UDF = 203 + SQL_COGROUPED_MAP_PANDAS_UDF = 204 def portable_hash(x): diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 516ee7e7b3084..e818241aaeebf 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -359,6 +359,24 @@ def __repr__(self): return "ArrowStreamPandasSerializer" +class InterleavedArrowReader(object): + + def __init__(self, stream): + import pyarrow as pa + self._schema1 = pa.read_schema(stream) + self._schema2 = pa.read_schema(stream) + self._reader = pa.MessageReader.open_stream(stream) + + def __iter__(self): + return self + + def __next__(self): + import pyarrow as pa + batch1 = pa.read_record_batch(self._reader.read_next_message(), self._schema1) + batch2 = pa.read_record_batch(self._reader.read_next_message(), self._schema2) + return batch1, batch2 + + class ArrowStreamPandasUDFSerializer(ArrowStreamPandasSerializer): """ Serializer used by Python worker to evaluate Pandas UDFs @@ -404,6 +422,22 @@ def __repr__(self): return "ArrowStreamPandasUDFSerializer" +class InterleavedArrowStreamPandasSerializer(ArrowStreamPandasUDFSerializer): + + def __init__(self, timezone, safecheck, assign_cols_by_name): + super(InterleavedArrowStreamPandasSerializer, self).__init__(timezone, safecheck, assign_cols_by_name) + + def load_stream(self, stream): + """ + Deserialize ArrowRecordBatches to an Arrow table and return as a list of pandas.Series. + """ + import pyarrow as pa + reader = InterleavedArrowReader(pa.input_stream(stream)) + for batch1, batch2 in reader: + yield ( [self.arrow_to_pandas(c) for c in pa.Table.from_batches([batch1]).itercolumns()], + [self.arrow_to_pandas(c) for c in pa.Table.from_batches([batch2]).itercolumns()]) + + class BatchedSerializer(Serializer): """ diff --git a/python/pyspark/sql/cogroup.py b/python/pyspark/sql/cogroup.py new file mode 100644 index 0000000000000..8946ab1202651 --- /dev/null +++ b/python/pyspark/sql/cogroup.py @@ -0,0 +1,15 @@ +from pyspark.sql.dataframe import DataFrame + + +class CoGroupedData(object): + + def __init__(self, gd1, gd2): + self._gd1 = gd1 + self._gd2 = gd2 + self.sql_ctx = gd1.sql_ctx + + def apply(self, udf): + df = self._gd1._df + udf_column = udf(*[df[col] for col in df.columns]) + jdf = self._gd1._jgd.flatMapCoGroupsInPandas(self._gd2._jgd, udf_column._jc.expr()) + return DataFrame(jdf, self.sql_ctx) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 613822b7edf2d..f156b9e9bd984 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -2798,6 +2798,8 @@ class PandasUDFType(object): GROUPED_MAP = PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF + COGROUPED_MAP = PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF + GROUPED_AGG = PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF @@ -3179,6 +3181,7 @@ def pandas_udf(f=None, returnType=None, functionType=None): if eval_type not in [PythonEvalType.SQL_SCALAR_PANDAS_UDF, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF, + PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF]: raise ValueError("Invalid functionType: " "functionType must be one the values from PandasUDFType") diff --git a/python/pyspark/sql/group.py b/python/pyspark/sql/group.py index cc1da8e7c1f72..04f42b1598376 100644 --- a/python/pyspark/sql/group.py +++ b/python/pyspark/sql/group.py @@ -22,6 +22,7 @@ from pyspark.sql.column import Column, _to_seq from pyspark.sql.dataframe import DataFrame from pyspark.sql.types import * +from pyspark.sql.cogroup import CoGroupedData __all__ = ["GroupedData"] @@ -220,6 +221,9 @@ def pivot(self, pivot_col, values=None): jgd = self._jgd.pivot(pivot_col, values) return GroupedData(jgd, self._df) + def cogroup(self, other): + return CoGroupedData(self, other) + @since(2.3) def apply(self, udf): """ diff --git a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py new file mode 100644 index 0000000000000..e98b66c64eaae --- /dev/null +++ b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py @@ -0,0 +1,113 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import datetime +import unittest +import sys + +from collections import OrderedDict +from decimal import Decimal + +from pyspark.sql import Row +from pyspark.sql.functions import array, explode, col, lit, udf, sum, pandas_udf, PandasUDFType +from pyspark.sql.types import * +from pyspark.testing.sqlutils import ReusedSQLTestCase, have_pandas, have_pyarrow, \ + pandas_requirement_message, pyarrow_requirement_message +from pyspark.testing.utils import QuietTest + +if have_pandas: + import pandas as pd + from pandas.util.testing import assert_frame_equal + +if have_pyarrow: + import pyarrow as pa + + +""" +Tests below use pd.DataFrame.assign that will infer mixed types (unicode/str) for column names +from kwargs w/ Python 2, so need to set check_column_type=False and avoid this check +""" +if sys.version < '3': + _check_column_type = False +else: + _check_column_type = True + + +@unittest.skipIf( + not have_pandas or not have_pyarrow, + pandas_requirement_message or pyarrow_requirement_message) +class CoGroupedMapPandasUDFTests(ReusedSQLTestCase): + + @property + def data(self): + return self.spark.range(10).toDF('id') \ + .withColumn("vs", array([lit(i) for i in range(20, 30)])) \ + .withColumn("v", explode(col('vs'))).drop('vs') + + def test_supported_types(self): + + df1 = self.spark.createDataFrame( + pd.DataFrame.from_dict({ + 'id' : [1,1,10, 10, 1,1], + 'x' : [1.0, 2.0, 3.0, 4.0, 5.0, 6.0] + })) + + df2 = self.spark.createDataFrame( + pd.DataFrame.from_dict({ + 'id2': [1,1,10, 10, 1,1], + 'a': [1.0, 2.0, 3.0, 4.0, 5.0, 6.0] + })) + + output_schema = StructType([ + StructField("id", LongType()), + StructField("x", DoubleType()), + ]) + + @pandas_udf(output_schema, functionType=PandasUDFType.COGROUPED_MAP) + def foo(left, right): + print("hello") + print(left) + print("goodbye") + print(right) + return left + + output_schema2 = StructType([ + StructField("id", LongType()) + ]) + @pandas_udf(output_schema, functionType=PandasUDFType.GROUPED_MAP) + def foo2(key, df): + print('key is ' + str(key)) + print(df) + return df + + + df1.groupby(col("id") > 5)\ + .apply(foo2)\ + .show() + + + + +if __name__ == "__main__": + from pyspark.sql.tests.test_pandas_udf_cogrouped_map import * + + try: + import xmlrunner + testRunner = xmlrunner.XMLTestRunner(output='target/test-reports') + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 16257bef6b320..2ccc28cffce78 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -38,7 +38,7 @@ from pyspark.rdd import PythonEvalType from pyspark.serializers import write_with_length, write_int, read_long, read_bool, \ write_long, read_int, SpecialLengths, UTF8Deserializer, PickleSerializer, \ - BatchedSerializer, ArrowStreamPandasUDFSerializer + BatchedSerializer, ArrowStreamPandasUDFSerializer, InterleavedArrowStreamPandasSerializer from pyspark.sql.types import to_arrow_type, StructType from pyspark.util import _get_argspec, fail_on_stopiteration from pyspark import shuffle @@ -103,8 +103,25 @@ def verify_result_length(*a): return lambda *a: (verify_result_length(*a), arrow_return_type) -def wrap_grouped_map_pandas_udf(f, return_type, argspec): +def wrap_cogrouped_map_pandas_udf(f, return_type): + def wrapped(left, right): + import pandas as pd + result = f(pd.concat(left, axis=1), pd.concat(right, axis=1)) + if not isinstance(result, pd.DataFrame): + raise TypeError("Return type of the user-defined function should be " + "pandas.DataFrame, but is {}".format(type(result))) + if not len(result.columns) == len(return_type): + raise RuntimeError( + "Number of columns of the returned pandas.DataFrame " + "doesn't match specified schema. " + "Expected: {} Actual: {}".format(len(return_type), len(result.columns))) + return result + + return lambda v: [(wrapped(v[0], v[1]), to_arrow_type(return_type))] + + +def wrap_grouped_map_pandas_udf(f, return_type, argspec): def wrapped(key_series, value_series): import pandas as pd @@ -233,6 +250,7 @@ def read_udfs(pickleSer, infile, eval_type): runner_conf = {} if eval_type in (PythonEvalType.SQL_SCALAR_PANDAS_UDF, + PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF, PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF, PythonEvalType.SQL_WINDOW_AGG_PANDAS_UDF): @@ -255,9 +273,12 @@ def read_udfs(pickleSer, infile, eval_type): # Scalar Pandas UDF handles struct type arguments as pandas DataFrames instead of # pandas Series. See SPARK-27240. - df_for_struct = eval_type == PythonEvalType.SQL_SCALAR_PANDAS_UDF - ser = ArrowStreamPandasUDFSerializer(timezone, safecheck, assign_cols_by_name, - df_for_struct) + if eval_type == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: + ser = InterleavedArrowStreamPandasSerializer(timezone, safecheck, assign_cols_by_name) + else: + df_for_struct = eval_type == PythonEvalType.SQL_SCALAR_PANDAS_UDF + ser = ArrowStreamPandasUDFSerializer(timezone, safecheck, assign_cols_by_name, + df_for_struct) else: ser = BatchedSerializer(PickleSerializer(), 100) @@ -282,6 +303,14 @@ def read_udfs(pickleSer, infile, eval_type): arg0 = ["a[%d]" % o for o in arg_offsets[1: split_offset]] arg1 = ["a[%d]" % o for o in arg_offsets[split_offset:]] mapper_str = "lambda a: f([%s], [%s])" % (", ".join(arg0), ", ".join(arg1)) + elif eval_type == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: + # We assume there is only one UDF here because cogrouped map doesn't + # support combining multiple UDFs. + assert num_udfs == 1 + arg_offsets, udf = read_single_udf( + pickleSer, infile, eval_type, runner_conf, udf_index=0) + udfs['f'] = udf + mapper_str = "lambda a: f(a)" else: # Create function like this: # lambda a: (f0(a[0]), f1(a[1], a[2]), f2(a[3])) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala index 2df30a1a53ad7..35adbae423f25 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala @@ -39,6 +39,18 @@ case class FlatMapGroupsInPandas( override val producedAttributes = AttributeSet(output) } + +case class FlatMapCoGroupsInPandas( + leftAttributes: Seq[Attribute], + rightAttributes: Seq[Attribute], + functionExpr: Expression, + output: Seq[Attribute], + left: LogicalPlan, + right: LogicalPlan) extends BinaryNode { + override val producedAttributes = AttributeSet(output) +} + + trait BaseEvalPython extends UnaryNode { def udfs: Seq[PythonUDF] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index e85636d82a62c..147cc00c0ba91 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -47,8 +47,8 @@ import org.apache.spark.sql.types.{NumericType, StructType} */ @Stable class RelationalGroupedDataset protected[sql]( - df: DataFrame, - groupingExprs: Seq[Expression], + private val df: DataFrame, + private val groupingExprs: Seq[Expression], groupType: RelationalGroupedDataset.GroupType) { private[this] def toDF(aggExprs: Seq[Expression]): DataFrame = { @@ -523,6 +523,33 @@ class RelationalGroupedDataset protected[sql]( Dataset.ofRows(df.sparkSession, plan) } + private[sql] def flatMapCoGroupsInPandas + (r: RelationalGroupedDataset, expr: PythonUDF): DataFrame = { + require(expr.evalType == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, + "Must pass a cogrouped map udf") + require(expr.dataType.isInstanceOf[StructType], + s"The returnType of the udf must be a ${StructType.simpleString}") + + val leftGroupingNamedExpressions = groupingExprs.map { + case ne: NamedExpression => ne + case other => Alias(other, other.toString)() + } + + val rightGroupingNamedExpressions = r.groupingExprs.map { + case ne: NamedExpression => ne + case other => Alias(other, other.toString)() + } + + val leftAttributes = leftGroupingNamedExpressions.map(_.toAttribute) + val rightAttributes = rightGroupingNamedExpressions.map(_.toAttribute) + val left = df.logicalPlan + val right = r.df.logicalPlan + val output = expr.dataType.asInstanceOf[StructType].toAttributes + val plan = FlatMapCoGroupsInPandas(leftAttributes, rightAttributes, expr, output, left, right) + + Dataset.ofRows(df.sparkSession, plan) + } + override def toString: String = { val builder = new StringBuilder builder.append("RelationalGroupedDataset: [grouping expressions: [") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index c4031496f610f..965f04c058966 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -679,6 +679,9 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { f, p, b, is, ot, planLater(child)) :: Nil case logical.FlatMapGroupsInPandas(grouping, func, output, child) => execution.python.FlatMapGroupsInPandasExec(grouping, func, output, planLater(child)) :: Nil + case logical.FlatMapCoGroupsInPandas(leftGroup, rightGroup, func, output, left, right) => + execution.python.FlatMapCoGroupsInPandasExec( + leftGroup, rightGroup, func, output, planLater(left), planLater(right)) :: Nil case logical.MapElements(f, _, _, objAttr, child) => execution.MapElementsExec(f, objAttr, planLater(child)) :: Nil case logical.AppendColumns(f, _, _, in, out, child) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala new file mode 100644 index 0000000000000..2dc517354812a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.python + +import scala.collection.JavaConverters._ + +import org.apache.spark.TaskContext +import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Distribution, Partitioning} +import org.apache.spark.sql.execution.{BinaryExecNode, CoGroupedIterator, GroupedIterator, SparkPlan} +import org.apache.spark.sql.util.ArrowUtils +import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch} + +case class FlatMapCoGroupsInPandasExec( + leftGroup: Seq[Attribute], + rightGroup: Seq[Attribute], + func: Expression, + output: Seq[Attribute], + left: SparkPlan, + right: SparkPlan) + extends BinaryExecNode { + + private val pandasFunction = func.asInstanceOf[PythonUDF].func + + override def outputPartitioning: Partitioning = left.outputPartitioning + + override def producedAttributes: AttributeSet = AttributeSet(output) + + override def requiredChildDistribution: Seq[Distribution] = { + ClusteredDistribution(leftGroup) :: ClusteredDistribution(rightGroup) :: Nil + } + + override def requiredChildOrdering: Seq[Seq[SortOrder]] = { + leftGroup + .map(SortOrder(_, Ascending)) :: rightGroup.map(SortOrder(_, Ascending)) :: Nil + } + + + override protected def doExecute(): RDD[InternalRow] = { + + val chainedFunc = Seq(ChainedPythonFunctions(Seq(pandasFunction))) + val sessionLocalTimeZone = conf.sessionLocalTimeZone + val pythonRunnerConf = ArrowUtils.getPythonRunnerConfMap(conf) + + left.execute().zipPartitions(right.execute()) { (leftData, rightData) => + val leftGrouped = GroupedIterator(leftData, leftGroup, left.output) + val rightGrouped = GroupedIterator(rightData, rightGroup, right.output) + val cogroup = new CoGroupedIterator(leftGrouped, rightGrouped, leftGroup) + .map{case (k, l, r) => (l, r)} + val context = TaskContext.get() + val columnarBatchIter = new InterleavedArrowPythonRunner( + chainedFunc, + PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, + Array(Array.empty), + left.schema, + right.schema, + sessionLocalTimeZone, + pythonRunnerConf).compute(cogroup, context.partitionId(), context) + + + val unsafeProj = UnsafeProjection.create(output, output) + + columnarBatchIter.flatMap { batch => + // Grouped Map UDF returns a StructType column in ColumnarBatch, select the children here + val structVector = batch.column(0).asInstanceOf[ArrowColumnVector] + val outputVectors = output.indices.map(structVector.getChild) + val flattenedBatch = new ColumnarBatch(outputVectors.toArray) + flattenedBatch.setNumRows(batch.numRows()) + flattenedBatch.rowIterator.asScala + }.map(unsafeProj) + } + + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala index 7b0e014f9ca48..46db29a250520 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala @@ -90,10 +90,14 @@ case class FlatMapGroupsInPandasExec( // argOffsets[1 .. argOffsets[0]+1] is the arg offsets for grouping attributes // argOffsets[argOffsets[0]+1 .. ] is the arg offsets for data attributes + println("grouping attributes are " + groupingAttributes.mkString(",")) + println("data attributes are " + child.output.mkString(",")) val dataAttributes = child.output.drop(groupingAttributes.length) val groupingIndicesInData = groupingAttributes.map { attribute => dataAttributes.indexWhere(attribute.semanticEquals) } + println("dataAttributes attributes are " + dataAttributes.mkString(",")) + println("groupingIndicesInData are " + groupingIndicesInData.mkString(",")) val groupingArgOffsets = new ArrayBuffer[Int] val nonDupGroupingAttributes = new ArrayBuffer[Attribute] @@ -116,15 +120,25 @@ case class FlatMapGroupsInPandasExec( } } + println("nonDupGroupingAttributes.length: " + nonDupGroupingAttributes.length) + println("nonDupGroupingAttributes.length: " + nonDupGroupingAttributes.length) + println("dataAttributes.length: " + dataAttributes.length) + val dataArgOffsets = nonDupGroupingAttributes.length until (nonDupGroupingAttributes.length + dataAttributes.length) + val argOffsets = Array(Array(groupingAttributes.length) ++ groupingArgOffsets ++ dataArgOffsets) + println("numAttributes are " + groupingAttributes.length) + println("groupingArgOffsets are " + groupingArgOffsets.mkString(",")) + println("dataArgOffsets are " + dataArgOffsets.mkString(",")) + + // Attributes after deduplication val dedupAttributes = nonDupGroupingAttributes ++ dataAttributes val dedupSchema = StructType.fromAttributes(dedupAttributes) - + println("dedupSchema is " + dedupSchema) inputRDD.mapPartitionsInternal { iter => val grouped = if (groupingAttributes.isEmpty) { Iterator(iter) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowWriter.scala new file mode 100644 index 0000000000000..f08763532eb23 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowWriter.scala @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.python + +import java.io.OutputStream +import java.nio.channels.Channels + +import org.apache.arrow.vector.{VectorSchemaRoot, VectorUnloader} +import org.apache.arrow.vector.ipc.WriteChannel +import org.apache.arrow.vector.ipc.message.MessageSerializer + + +class InterleavedArrowWriter( leftRoot: VectorSchemaRoot, + rightRoot: VectorSchemaRoot, + out: WriteChannel) extends AutoCloseable{ + + + private var started = false + private val leftUnloader = new VectorUnloader(leftRoot) + private val rightUnloader = new VectorUnloader(rightRoot) + + def start(): Unit = { + this.ensureStarted() + } + + def writeBatch(): Unit = { + this.ensureStarted() + val leftBatch = leftUnloader.getRecordBatch + val rightBatch = rightUnloader.getRecordBatch + MessageSerializer.serialize(out, leftBatch) + MessageSerializer.serialize(out, rightBatch) + leftBatch.close() + rightBatch.close() + } + + private def ensureStarted(): Unit = { + if (!started) { + started = true + MessageSerializer.serialize(out, leftRoot.getSchema) + MessageSerializer.serialize(out, rightRoot.getSchema) + } + } + + def end(): Unit = { + ensureStarted() + ensureEnded() + } + + def ensureEnded(): Unit = { + out.writeIntLittleEndian(0) + } + + def close(): Unit = { + out.close() + } + +} + +object InterleavedArrowWriter{ + + def apply(leftRoot: VectorSchemaRoot, + rightRoot: VectorSchemaRoot, + out: OutputStream): InterleavedArrowWriter = { + new InterleavedArrowWriter(leftRoot, rightRoot, new WriteChannel(Channels.newChannel(out))) + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/MultiDfArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/MultiDfArrowPythonRunner.scala new file mode 100644 index 0000000000000..f40ae92ac284e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/MultiDfArrowPythonRunner.scala @@ -0,0 +1,191 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.python + +import java.io._ +import java.net._ +import java.util.concurrent.atomic.AtomicBoolean + +import scala.collection.JavaConverters._ + +import org.apache.arrow.vector.VectorSchemaRoot +import org.apache.arrow.vector.ipc.{ArrowStreamReader, ArrowStreamWriter} + +import org.apache.spark._ +import org.apache.spark.api.python._ +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.arrow.ArrowWriter +import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.ArrowUtils +import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch, ColumnVector} +import org.apache.spark.util.Utils + + +class InterleavedArrowPythonRunner( + funcs: Seq[ChainedPythonFunctions], + evalType: Int, + argOffsets: Array[Array[Int]], + leftSchema: StructType, + rightSchema: StructType, + timeZoneId: String, + conf: Map[String, String]) + extends BasePythonRunner[(Iterator[InternalRow], Iterator[InternalRow]), ColumnarBatch]( + funcs, evalType, argOffsets) { + + protected override def newWriterThread( + env: SparkEnv, + worker: Socket, + inputIterator: Iterator[(Iterator[InternalRow], Iterator[InternalRow])], + partitionIndex: Int, + context: TaskContext): WriterThread = { + new WriterThread(env, worker, inputIterator, partitionIndex, context) { + + protected override def writeCommand(dataOut: DataOutputStream): Unit = { + + // Write config for the worker as a number of key -> value pairs of strings + dataOut.writeInt(conf.size) + for ((k, v) <- conf) { + PythonRDD.writeUTF(k, dataOut) + PythonRDD.writeUTF(v, dataOut) + } + + PythonUDFRunner.writeUDFs(dataOut, funcs, argOffsets) + } + + protected override def writeIteratorToStream(dataOut: DataOutputStream): Unit = { + val leftArrowSchema = ArrowUtils.toArrowSchema(leftSchema, timeZoneId) + val rightArrowSchema = ArrowUtils.toArrowSchema(rightSchema, timeZoneId) + val allocator = ArrowUtils.rootAllocator.newChildAllocator( + s"stdout writer for $pythonExec", 0, Long.MaxValue) + val leftRoot = VectorSchemaRoot.create(leftArrowSchema, allocator) + val rightRoot = VectorSchemaRoot.create(rightArrowSchema, allocator) + + Utils.tryWithSafeFinally { + val leftArrowWriter = ArrowWriter.create(leftRoot) + val rightArrowWriter = ArrowWriter.create(rightRoot) + val writer = InterleavedArrowWriter(leftRoot, rightRoot, dataOut) + writer.start() + + while (inputIterator.hasNext) { + + val (nextLeft, nextRight) = inputIterator.next() + + while (nextLeft.hasNext) { + leftArrowWriter.write(nextLeft.next()) + } + while (nextRight.hasNext) { + rightArrowWriter.write(nextRight.next()) + } + leftArrowWriter.finish() + rightArrowWriter.finish() + writer.writeBatch() + leftArrowWriter.reset() + rightArrowWriter.reset() + } + // end writes footer to the output stream and doesn't clean any resources. + // It could throw exception if the output stream is closed, so it should be + // in the try block. + writer.end() + } { + // If we close root and allocator in TaskCompletionListener, there could be a race + // condition where the writer thread keeps writing to the VectorSchemaRoot while + // it's being closed by the TaskCompletion listener. + // Closing root and allocator here is cleaner because root and allocator is owned + // by the writer thread and is only visible to the writer thread. + // + // If the writer thread is interrupted by TaskCompletionListener, it should either + // (1) in the try block, in which case it will get an InterruptedException when + // performing io, and goes into the finally block or (2) in the finally block, + // in which case it will ignore the interruption and close the resources. + leftRoot.close() + rightRoot.close() + allocator.close() + } + } + } + } + + protected override def newReaderIterator( + stream: DataInputStream, + writerThread: WriterThread, + startTime: Long, + env: SparkEnv, + worker: Socket, + releasedOrClosed: AtomicBoolean, + context: TaskContext): Iterator[ColumnarBatch] = { + new ReaderIterator(stream, writerThread, startTime, env, worker, releasedOrClosed, context) { + + private val allocator = ArrowUtils.rootAllocator.newChildAllocator( + s"stdin reader for $pythonExec", 0, Long.MaxValue) + + private var reader: ArrowStreamReader = _ + private var root: VectorSchemaRoot = _ + private var schema: StructType = _ + private var vectors: Array[ColumnVector] = _ + + context.addTaskCompletionListener[Unit] { _ => + if (reader != null) { + reader.close(false) + } + allocator.close() + } + + private var batchLoaded = true + + protected override def read(): ColumnarBatch = { + if (writerThread.exception.isDefined) { + throw writerThread.exception.get + } + try { + if (reader != null && batchLoaded) { + batchLoaded = reader.loadNextBatch() + if (batchLoaded) { + val batch = new ColumnarBatch(vectors) + batch.setNumRows(root.getRowCount) + batch + } else { + reader.close(false) + allocator.close() + // Reach end of stream. Call `read()` again to read control data. + read() + } + } else { + stream.readInt() match { + case SpecialLengths.START_ARROW_STREAM => + reader = new ArrowStreamReader(stream, allocator) + root = reader.getVectorSchemaRoot() + schema = ArrowUtils.fromArrowSchema(root.getSchema()) + vectors = root.getFieldVectors().asScala.map { vector => + new ArrowColumnVector(vector) + }.toArray[ColumnVector] + read() + case SpecialLengths.TIMING_DATA => + handleTimingData() + read() + case SpecialLengths.PYTHON_EXCEPTION_THROWN => + throw handlePythonException() + case SpecialLengths.END_OF_DATA_SECTION => + handleEndOfDataSection() + null + } + } + } catch handleException + } + } + } +} From 64ff5acc55392fb2a6fb3bd320820f2800769503 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Thu, 20 Jun 2019 20:45:59 +0100 Subject: [PATCH 02/34] minor tidy up --- .../tests/test_pandas_udf_cogrouped_map.py | 83 ++++++------- python/pyspark/worker.py | 2 + .../execution/python/ArrowPythonRunner.scala | 70 +---------- .../python/BaseArrowPythonRunner.scala | 114 ++++++++++++++++++ .../python/FlatMapCoGroupsInPandasExec.scala | 4 + .../python/FlatMapGroupsInPandasExec.scala | 16 +-- ...ala => InterleavedArrowPythonRunner.scala} | 77 +----------- .../python/InterleavedArrowWriter.scala | 17 +-- .../apache/spark/sql/GroupedDataTest.scala | 33 +++++ 9 files changed, 203 insertions(+), 213 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/python/BaseArrowPythonRunner.scala rename sql/core/src/main/scala/org/apache/spark/sql/execution/python/{MultiDfArrowPythonRunner.scala => InterleavedArrowPythonRunner.scala} (62%) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/GroupedDataTest.scala diff --git a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py index e98b66c64eaae..508825123df07 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py +++ b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py @@ -52,62 +52,51 @@ pandas_requirement_message or pyarrow_requirement_message) class CoGroupedMapPandasUDFTests(ReusedSQLTestCase): - @property - def data(self): - return self.spark.range(10).toDF('id') \ - .withColumn("vs", array([lit(i) for i in range(20, 30)])) \ - .withColumn("v", explode(col('vs'))).drop('vs') - - def test_supported_types(self): - - df1 = self.spark.createDataFrame( - pd.DataFrame.from_dict({ - 'id' : [1,1,10, 10, 1,1], - 'x' : [1.0, 2.0, 3.0, 4.0, 5.0, 6.0] - })) - - df2 = self.spark.createDataFrame( - pd.DataFrame.from_dict({ - 'id2': [1,1,10, 10, 1,1], - 'a': [1.0, 2.0, 3.0, 4.0, 5.0, 6.0] - })) + def test_simple(self): + + pdf1 = pd.DataFrame.from_dict({ + 'id': ['a', 'a', 'b', 'b'], + 't': [1.0, 2.0, 1.0, 2.0], + 'x': [10, 10, 30, 40] + + }) + + pdf2 = pd.DataFrame.from_dict({ + 'id2': ['a', 'b'], + 't': [0.5, 0.5], + 'y': [7.0, 8.0] + }) output_schema = StructType([ - StructField("id", LongType()), - StructField("x", DoubleType()), + StructField("id", StringType()), + StructField("t", DoubleType()), + StructField("x", IntegerType()), + StructField("y", DoubleType()), ]) + @pandas_udf(output_schema, functionType=PandasUDFType.COGROUPED_MAP) - def foo(left, right): - print("hello") + def pandas_merge(left, right): print(left) - print("goodbye") + print("#########") print(right) - return left - - output_schema2 = StructType([ - StructField("id", LongType()) - ]) - @pandas_udf(output_schema, functionType=PandasUDFType.GROUPED_MAP) - def foo2(key, df): - print('key is ' + str(key)) - print(df) - return df - - - df1.groupby(col("id") > 5)\ - .apply(foo2)\ - .show() + print("#########") + import pandas as pd + left.sort_values(by='t', inplace=True) + right.sort_values(by='t', inplace=True) + result = pd.merge_asof(left, right, on='t').reset_index() + print(result) + return result + df1 = self.spark.createDataFrame(pdf1) + df2 = self.spark.createDataFrame(pdf2) + gd1 = df1.groupby('id') + gd2 = df2.groupby('id2') -if __name__ == "__main__": - from pyspark.sql.tests.test_pandas_udf_cogrouped_map import * + gd1\ + .cogroup(gd2)\ + .apply(pandas_merge)\ + .explain() - try: - import xmlrunner - testRunner = xmlrunner.XMLTestRunner(output='target/test-reports') - except ImportError: - testRunner = None - unittest.main(testRunner=testRunner, verbosity=2) diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 2ccc28cffce78..1a423920b77f5 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -236,6 +236,8 @@ def read_single_udf(pickleSer, infile, eval_type, runner_conf, udf_index): elif eval_type == PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF: argspec = _get_argspec(row_func) # signature was lost when wrapping it return arg_offsets, wrap_grouped_map_pandas_udf(func, return_type, argspec) + elif eval_type == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: + return arg_offsets, wrap_cogrouped_map_pandas_udf(func, return_type) elif eval_type == PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF: return arg_offsets, wrap_grouped_agg_pandas_udf(func, return_type) elif eval_type == PythonEvalType.SQL_WINDOW_AGG_PANDAS_UDF: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala index 3710218b2af5f..5e00eecf1b230 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala @@ -45,7 +45,7 @@ class ArrowPythonRunner( schema: StructType, timeZoneId: String, conf: Map[String, String]) - extends BasePythonRunner[Iterator[InternalRow], ColumnarBatch]( + extends BaseArrowPythonRunner[Iterator[InternalRow]]( funcs, evalType, argOffsets) { protected override def newWriterThread( @@ -112,72 +112,4 @@ class ArrowPythonRunner( } } - protected override def newReaderIterator( - stream: DataInputStream, - writerThread: WriterThread, - startTime: Long, - env: SparkEnv, - worker: Socket, - releasedOrClosed: AtomicBoolean, - context: TaskContext): Iterator[ColumnarBatch] = { - new ReaderIterator(stream, writerThread, startTime, env, worker, releasedOrClosed, context) { - - private val allocator = ArrowUtils.rootAllocator.newChildAllocator( - s"stdin reader for $pythonExec", 0, Long.MaxValue) - - private var reader: ArrowStreamReader = _ - private var root: VectorSchemaRoot = _ - private var schema: StructType = _ - private var vectors: Array[ColumnVector] = _ - - context.addTaskCompletionListener[Unit] { _ => - if (reader != null) { - reader.close(false) - } - allocator.close() - } - - private var batchLoaded = true - - protected override def read(): ColumnarBatch = { - if (writerThread.exception.isDefined) { - throw writerThread.exception.get - } - try { - if (reader != null && batchLoaded) { - batchLoaded = reader.loadNextBatch() - if (batchLoaded) { - val batch = new ColumnarBatch(vectors) - batch.setNumRows(root.getRowCount) - batch - } else { - reader.close(false) - allocator.close() - // Reach end of stream. Call `read()` again to read control data. - read() - } - } else { - stream.readInt() match { - case SpecialLengths.START_ARROW_STREAM => - reader = new ArrowStreamReader(stream, allocator) - root = reader.getVectorSchemaRoot() - schema = ArrowUtils.fromArrowSchema(root.getSchema()) - vectors = root.getFieldVectors().asScala.map { vector => - new ArrowColumnVector(vector) - }.toArray[ColumnVector] - read() - case SpecialLengths.TIMING_DATA => - handleTimingData() - read() - case SpecialLengths.PYTHON_EXCEPTION_THROWN => - throw handlePythonException() - case SpecialLengths.END_OF_DATA_SECTION => - handleEndOfDataSection() - null - } - } - } catch handleException - } - } - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BaseArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BaseArrowPythonRunner.scala new file mode 100644 index 0000000000000..3cba06dcf7d52 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BaseArrowPythonRunner.scala @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.python + +import java.io._ +import java.net._ +import java.util.concurrent.atomic.AtomicBoolean + +import scala.collection.JavaConverters._ + +import org.apache.arrow.vector.VectorSchemaRoot +import org.apache.arrow.vector.ipc.ArrowStreamReader + +import org.apache.spark._ +import org.apache.spark.api.python._ +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.ArrowUtils +import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch, ColumnVector} + + +abstract class BaseArrowPythonRunner[T]( + funcs: Seq[ChainedPythonFunctions], + evalType: Int, + argOffsets: Array[Array[Int]]) + extends BasePythonRunner[T, ColumnarBatch]( + funcs, evalType, argOffsets) { + + + protected override def newReaderIterator( + stream: DataInputStream, + writerThread: WriterThread, + startTime: Long, + env: SparkEnv, + worker: Socket, + releasedOrClosed: AtomicBoolean, + context: TaskContext): Iterator[ColumnarBatch] = { + + new ReaderIterator(stream, writerThread, startTime, env, worker, releasedOrClosed, context) { + + private val allocator = ArrowUtils.rootAllocator.newChildAllocator( + s"stdin reader for $pythonExec", 0, Long.MaxValue) + + private var reader: ArrowStreamReader = _ + private var root: VectorSchemaRoot = _ + private var schema: StructType = _ + private var vectors: Array[ColumnVector] = _ + + context.addTaskCompletionListener[Unit] { _ => + if (reader != null) { + reader.close(false) + } + allocator.close() + } + + private var batchLoaded = true + + protected override def read(): ColumnarBatch = { + if (writerThread.exception.isDefined) { + throw writerThread.exception.get + } + try { + if (reader != null && batchLoaded) { + batchLoaded = reader.loadNextBatch() + if (batchLoaded) { + val batch = new ColumnarBatch(vectors) + batch.setNumRows(root.getRowCount) + batch + } else { + reader.close(false) + allocator.close() + // Reach end of stream. Call `read()` again to read control data. + read() + } + } else { + stream.readInt() match { + case SpecialLengths.START_ARROW_STREAM => + reader = new ArrowStreamReader(stream, allocator) + root = reader.getVectorSchemaRoot() + schema = ArrowUtils.fromArrowSchema(root.getSchema()) + vectors = root.getFieldVectors().asScala.map { vector => + new ArrowColumnVector(vector) + }.toArray[ColumnVector] + read() + case SpecialLengths.TIMING_DATA => + handleTimingData() + read() + case SpecialLengths.PYTHON_EXCEPTION_THROWN => + throw handlePythonException() + case SpecialLengths.END_OF_DATA_SECTION => + handleEndOfDataSection() + null + } + } + } catch handleException + } + } + } +} + + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala index 2dc517354812a..22c781b9fe42c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala @@ -60,12 +60,16 @@ case class FlatMapCoGroupsInPandasExec( val sessionLocalTimeZone = conf.sessionLocalTimeZone val pythonRunnerConf = ArrowUtils.getPythonRunnerConfMap(conf) + left.execute().zipPartitions(right.execute()) { (leftData, rightData) => val leftGrouped = GroupedIterator(leftData, leftGroup, left.output) val rightGrouped = GroupedIterator(rightData, rightGroup, right.output) val cogroup = new CoGroupedIterator(leftGrouped, rightGrouped, leftGroup) .map{case (k, l, r) => (l, r)} val context = TaskContext.get() + println("in zipPartitions: left schema is " + left.schema) + println("in zipPartitions: right schema is " + right.schema) + val columnarBatchIter = new InterleavedArrowPythonRunner( chainedFunc, PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala index 46db29a250520..7b0e014f9ca48 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala @@ -90,14 +90,10 @@ case class FlatMapGroupsInPandasExec( // argOffsets[1 .. argOffsets[0]+1] is the arg offsets for grouping attributes // argOffsets[argOffsets[0]+1 .. ] is the arg offsets for data attributes - println("grouping attributes are " + groupingAttributes.mkString(",")) - println("data attributes are " + child.output.mkString(",")) val dataAttributes = child.output.drop(groupingAttributes.length) val groupingIndicesInData = groupingAttributes.map { attribute => dataAttributes.indexWhere(attribute.semanticEquals) } - println("dataAttributes attributes are " + dataAttributes.mkString(",")) - println("groupingIndicesInData are " + groupingIndicesInData.mkString(",")) val groupingArgOffsets = new ArrayBuffer[Int] val nonDupGroupingAttributes = new ArrayBuffer[Attribute] @@ -120,25 +116,15 @@ case class FlatMapGroupsInPandasExec( } } - println("nonDupGroupingAttributes.length: " + nonDupGroupingAttributes.length) - println("nonDupGroupingAttributes.length: " + nonDupGroupingAttributes.length) - println("dataAttributes.length: " + dataAttributes.length) - val dataArgOffsets = nonDupGroupingAttributes.length until (nonDupGroupingAttributes.length + dataAttributes.length) - val argOffsets = Array(Array(groupingAttributes.length) ++ groupingArgOffsets ++ dataArgOffsets) - println("numAttributes are " + groupingAttributes.length) - println("groupingArgOffsets are " + groupingArgOffsets.mkString(",")) - println("dataArgOffsets are " + dataArgOffsets.mkString(",")) - - // Attributes after deduplication val dedupAttributes = nonDupGroupingAttributes ++ dataAttributes val dedupSchema = StructType.fromAttributes(dedupAttributes) - println("dedupSchema is " + dedupSchema) + inputRDD.mapPartitionsInternal { iter => val grouped = if (groupingAttributes.isEmpty) { Iterator(iter) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/MultiDfArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala similarity index 62% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/python/MultiDfArrowPythonRunner.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala index f40ae92ac284e..b39885ee47a2d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/MultiDfArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala @@ -19,12 +19,8 @@ package org.apache.spark.sql.execution.python import java.io._ import java.net._ -import java.util.concurrent.atomic.AtomicBoolean - -import scala.collection.JavaConverters._ import org.apache.arrow.vector.VectorSchemaRoot -import org.apache.arrow.vector.ipc.{ArrowStreamReader, ArrowStreamWriter} import org.apache.spark._ import org.apache.spark.api.python._ @@ -32,7 +28,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.arrow.ArrowWriter import org.apache.spark.sql.types._ import org.apache.spark.sql.util.ArrowUtils -import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch, ColumnVector} +import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils @@ -44,7 +40,7 @@ class InterleavedArrowPythonRunner( rightSchema: StructType, timeZoneId: String, conf: Map[String, String]) - extends BasePythonRunner[(Iterator[InternalRow], Iterator[InternalRow]), ColumnarBatch]( + extends BaseArrowPythonRunner[(Iterator[InternalRow], Iterator[InternalRow])]( funcs, evalType, argOffsets) { protected override def newWriterThread( @@ -119,73 +115,4 @@ class InterleavedArrowPythonRunner( } } } - - protected override def newReaderIterator( - stream: DataInputStream, - writerThread: WriterThread, - startTime: Long, - env: SparkEnv, - worker: Socket, - releasedOrClosed: AtomicBoolean, - context: TaskContext): Iterator[ColumnarBatch] = { - new ReaderIterator(stream, writerThread, startTime, env, worker, releasedOrClosed, context) { - - private val allocator = ArrowUtils.rootAllocator.newChildAllocator( - s"stdin reader for $pythonExec", 0, Long.MaxValue) - - private var reader: ArrowStreamReader = _ - private var root: VectorSchemaRoot = _ - private var schema: StructType = _ - private var vectors: Array[ColumnVector] = _ - - context.addTaskCompletionListener[Unit] { _ => - if (reader != null) { - reader.close(false) - } - allocator.close() - } - - private var batchLoaded = true - - protected override def read(): ColumnarBatch = { - if (writerThread.exception.isDefined) { - throw writerThread.exception.get - } - try { - if (reader != null && batchLoaded) { - batchLoaded = reader.loadNextBatch() - if (batchLoaded) { - val batch = new ColumnarBatch(vectors) - batch.setNumRows(root.getRowCount) - batch - } else { - reader.close(false) - allocator.close() - // Reach end of stream. Call `read()` again to read control data. - read() - } - } else { - stream.readInt() match { - case SpecialLengths.START_ARROW_STREAM => - reader = new ArrowStreamReader(stream, allocator) - root = reader.getVectorSchemaRoot() - schema = ArrowUtils.fromArrowSchema(root.getSchema()) - vectors = root.getFieldVectors().asScala.map { vector => - new ArrowColumnVector(vector) - }.toArray[ColumnVector] - read() - case SpecialLengths.TIMING_DATA => - handleTimingData() - read() - case SpecialLengths.PYTHON_EXCEPTION_THROWN => - throw handlePythonException() - case SpecialLengths.END_OF_DATA_SECTION => - handleEndOfDataSection() - null - } - } - } catch handleException - } - } - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowWriter.scala index f08763532eb23..eb9f1d4494b91 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowWriter.scala @@ -22,7 +22,7 @@ import java.nio.channels.Channels import org.apache.arrow.vector.{VectorSchemaRoot, VectorUnloader} import org.apache.arrow.vector.ipc.WriteChannel -import org.apache.arrow.vector.ipc.message.MessageSerializer +import org.apache.arrow.vector.ipc.message.{ArrowRecordBatch, MessageSerializer} class InterleavedArrowWriter( leftRoot: VectorSchemaRoot, @@ -40,12 +40,15 @@ class InterleavedArrowWriter( leftRoot: VectorSchemaRoot, def writeBatch(): Unit = { this.ensureStarted() - val leftBatch = leftUnloader.getRecordBatch - val rightBatch = rightUnloader.getRecordBatch - MessageSerializer.serialize(out, leftBatch) - MessageSerializer.serialize(out, rightBatch) - leftBatch.close() - rightBatch.close() + writeRecordBatch(leftUnloader.getRecordBatch) + writeRecordBatch(rightUnloader.getRecordBatch) + } + + private def writeRecordBatch(b: ArrowRecordBatch): Unit = { + try + MessageSerializer.serialize(out, b) + finally + b.close() } private def ensureStarted(): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/GroupedDataTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/GroupedDataTest.scala new file mode 100644 index 0000000000000..cc9ba8bfc5325 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/GroupedDataTest.scala @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.functions.sum +import org.apache.spark.sql.test.{SharedSQLContext, SharedSparkSession} +import org.scalatest.concurrent.Eventually +import org.apache.spark.sql.functions._ + +class GroupedDataTest extends QueryTest with SharedSQLContext with Eventually { + + test("SPARK-7150 range api") { + val df = spark + .range(0, 100) + .withColumn("x", lit("a")) + } + +} From 6d039e366488945f5e673c6aba29bfd710d46ef8 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Fri, 21 Jun 2019 06:47:28 +0100 Subject: [PATCH 03/34] removed incorrect test --- python/pyspark/sql/tests/test_group.py | 46 -------------------------- 1 file changed, 46 deletions(-) delete mode 100644 python/pyspark/sql/tests/test_group.py diff --git a/python/pyspark/sql/tests/test_group.py b/python/pyspark/sql/tests/test_group.py deleted file mode 100644 index 6de1b8ea0b3ce..0000000000000 --- a/python/pyspark/sql/tests/test_group.py +++ /dev/null @@ -1,46 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -from pyspark.sql import Row -from pyspark.testing.sqlutils import ReusedSQLTestCase - - -class GroupTests(ReusedSQLTestCase): - - def test_aggregator(self): - df = self.df - g = df.groupBy() - self.assertEqual([99, 100], sorted(g.agg({'key': 'max', 'value': 'count'}).collect()[0])) - self.assertEqual([Row(**{"AVG(key#0)": 49.5})], g.mean().collect()) - - from pyspark.sql import functions - self.assertEqual((0, u'99'), - tuple(g.agg(functions.first(df.key), functions.last(df.value)).first())) - self.assertTrue(95 < g.agg(functions.approx_count_distinct(df.key)).first()[0]) - self.assertEqual(100, g.agg(functions.countDistinct(df.value)).first()[0]) - - -if __name__ == "__main__": - import unittest - from pyspark.sql.tests.test_group import * - - try: - import xmlrunner - testRunner = xmlrunner.XMLTestRunner(output='target/test-reports') - except ImportError: - testRunner = None - unittest.main(testRunner=testRunner, verbosity=2) From d8a5c5dad1acfb4efb0a6255280bc95bb0ce99ba Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Tue, 25 Jun 2019 10:50:40 +0100 Subject: [PATCH 04/34] tidies up test, fixed output cols --- python/pyspark/sql/cogroup.py | 10 ++- .../tests/test_pandas_udf_cogrouped_map.py | 81 +++++++++---------- 2 files changed, 44 insertions(+), 47 deletions(-) diff --git a/python/pyspark/sql/cogroup.py b/python/pyspark/sql/cogroup.py index 8946ab1202651..18dc397c8e348 100644 --- a/python/pyspark/sql/cogroup.py +++ b/python/pyspark/sql/cogroup.py @@ -9,7 +9,13 @@ def __init__(self, gd1, gd2): self.sql_ctx = gd1.sql_ctx def apply(self, udf): - df = self._gd1._df - udf_column = udf(*[df[col] for col in df.columns]) + all_cols = self._extract_cols(self._gd1) + self._extract_cols(self._gd2) + udf_column = udf(*all_cols) jdf = self._gd1._jgd.flatMapCoGroupsInPandas(self._gd2._jgd, udf_column._jc.expr()) return DataFrame(jdf, self.sql_ctx) + + @staticmethod + def _extract_cols(gd): + df = gd._df + return [df[col] for col in df.columns] + diff --git a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py index 508825123df07..d74f9b10325ed 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py +++ b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py @@ -52,51 +52,42 @@ pandas_requirement_message or pyarrow_requirement_message) class CoGroupedMapPandasUDFTests(ReusedSQLTestCase): + @property + def data1(self): + return self.spark.range(10).toDF('id') \ + .withColumn("ks", array([lit(i) for i in range(20, 30)])) \ + .withColumn("k", explode(col('ks')))\ + .withColumn("v", col('k') * 10)\ + .drop('ks') + + @property + def data2(self): + return self.spark.range(10).toDF('id') \ + .withColumn("ks", array([lit(i) for i in range(20, 30)])) \ + .withColumn("k", explode(col('ks'))) \ + .withColumn("v2", col('k') * 100) \ + .drop('ks') + def test_simple(self): + import pandas as pd + + l = self.data1 + r = self.data2 + + @pandas_udf('id long, k int, v int, v2 int', PandasUDFType.COGROUPED_MAP) + def merge_pandas(left, right): + return pd.merge(left, right, how='outer', on=['k', 'id']) + + # TODO: Grouping by a string fails to resolve here as analyzer cannot determine side + result = l\ + .groupby(l.id)\ + .cogroup(r.groupby(r.id))\ + .apply(merge_pandas)\ + .sort(['id', 'k'])\ + .toPandas() + + expected = pd\ + .merge(l.toPandas(), r.toPandas(), how='outer', on=['k', 'id']) - pdf1 = pd.DataFrame.from_dict({ - 'id': ['a', 'a', 'b', 'b'], - 't': [1.0, 2.0, 1.0, 2.0], - 'x': [10, 10, 30, 40] - - }) - - pdf2 = pd.DataFrame.from_dict({ - 'id2': ['a', 'b'], - 't': [0.5, 0.5], - 'y': [7.0, 8.0] - }) - - output_schema = StructType([ - StructField("id", StringType()), - StructField("t", DoubleType()), - StructField("x", IntegerType()), - StructField("y", DoubleType()), - ]) - - - @pandas_udf(output_schema, functionType=PandasUDFType.COGROUPED_MAP) - def pandas_merge(left, right): - print(left) - print("#########") - print(right) - print("#########") - import pandas as pd - left.sort_values(by='t', inplace=True) - right.sort_values(by='t', inplace=True) - result = pd.merge_asof(left, right, on='t').reset_index() - print(result) - return result - - - df1 = self.spark.createDataFrame(pdf1) - df2 = self.spark.createDataFrame(pdf2) - - gd1 = df1.groupby('id') - gd2 = df2.groupby('id2') - - gd1\ - .cogroup(gd2)\ - .apply(pandas_merge)\ - .explain() + assert_frame_equal(expected, result, check_column_type=_check_column_type) From 73188f632a85098567302d1470e8190dbe65a783 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Tue, 25 Jun 2019 10:55:02 +0100 Subject: [PATCH 05/34] removed incorrect file --- .../python/FlatMapCoGroupsInPandasExec.scala | 4 +-- .../apache/spark/sql/GroupedDataTest.scala | 33 ------------------- 2 files changed, 1 insertion(+), 36 deletions(-) delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/GroupedDataTest.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala index 22c781b9fe42c..12620264de087 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala @@ -67,8 +67,6 @@ case class FlatMapCoGroupsInPandasExec( val cogroup = new CoGroupedIterator(leftGrouped, rightGrouped, leftGroup) .map{case (k, l, r) => (l, r)} val context = TaskContext.get() - println("in zipPartitions: left schema is " + left.schema) - println("in zipPartitions: right schema is " + right.schema) val columnarBatchIter = new InterleavedArrowPythonRunner( chainedFunc, @@ -83,7 +81,7 @@ case class FlatMapCoGroupsInPandasExec( val unsafeProj = UnsafeProjection.create(output, output) columnarBatchIter.flatMap { batch => - // Grouped Map UDF returns a StructType column in ColumnarBatch, select the children here + // UDF returns a StructType column in ColumnarBatch, select the children here val structVector = batch.column(0).asInstanceOf[ArrowColumnVector] val outputVectors = output.indices.map(structVector.getChild) val flattenedBatch = new ColumnarBatch(outputVectors.toArray) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/GroupedDataTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/GroupedDataTest.scala deleted file mode 100644 index cc9ba8bfc5325..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/GroupedDataTest.scala +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql - -import org.apache.spark.sql.functions.sum -import org.apache.spark.sql.test.{SharedSQLContext, SharedSparkSession} -import org.scalatest.concurrent.Eventually -import org.apache.spark.sql.functions._ - -class GroupedDataTest extends QueryTest with SharedSQLContext with Eventually { - - test("SPARK-7150 range api") { - val df = spark - .range(0, 100) - .withColumn("x", lit("a")) - } - -} From 690fa14e4ca511b82de08e22f721caf5ec930e0b Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Tue, 25 Jun 2019 10:55:09 +0100 Subject: [PATCH 06/34] Revert: removed incorrect test --- python/pyspark/sql/tests/test_group.py | 46 ++++++++++++++++++++++++++ 1 file changed, 46 insertions(+) create mode 100644 python/pyspark/sql/tests/test_group.py diff --git a/python/pyspark/sql/tests/test_group.py b/python/pyspark/sql/tests/test_group.py new file mode 100644 index 0000000000000..6de1b8ea0b3ce --- /dev/null +++ b/python/pyspark/sql/tests/test_group.py @@ -0,0 +1,46 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from pyspark.sql import Row +from pyspark.testing.sqlutils import ReusedSQLTestCase + + +class GroupTests(ReusedSQLTestCase): + + def test_aggregator(self): + df = self.df + g = df.groupBy() + self.assertEqual([99, 100], sorted(g.agg({'key': 'max', 'value': 'count'}).collect()[0])) + self.assertEqual([Row(**{"AVG(key#0)": 49.5})], g.mean().collect()) + + from pyspark.sql import functions + self.assertEqual((0, u'99'), + tuple(g.agg(functions.first(df.key), functions.last(df.value)).first())) + self.assertTrue(95 < g.agg(functions.approx_count_distinct(df.key)).first()[0]) + self.assertEqual(100, g.agg(functions.countDistinct(df.value)).first()[0]) + + +if __name__ == "__main__": + import unittest + from pyspark.sql.tests.test_group import * + + try: + import xmlrunner + testRunner = xmlrunner.XMLTestRunner(output='target/test-reports') + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) From e3b66acf4849f3ff0278ac4c5fcfc5b11e4fc1fe Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Tue, 25 Jun 2019 17:25:39 +0100 Subject: [PATCH 07/34] fix for resolving key cols --- python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py | 3 +-- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 5 +++++ 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py index d74f9b10325ed..332bc260541a6 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py +++ b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py @@ -78,9 +78,8 @@ def test_simple(self): def merge_pandas(left, right): return pd.merge(left, right, how='outer', on=['k', 'id']) - # TODO: Grouping by a string fails to resolve here as analyzer cannot determine side result = l\ - .groupby(l.id)\ + .groupby('id')\ .cogroup(r.groupby(r.id))\ .apply(merge_pandas)\ .sort(['id', 'k'])\ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 60517f11a2491..9dad538686b30 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -970,6 +970,10 @@ class Analyzer( // To resolve duplicate expression IDs for Join and Intersect case j @ Join(left, right, _, _, _) if !j.duplicateResolved => j.copy(right = dedupRight(left, right)) + case f @ FlatMapCoGroupsInPandas(leftAttributes, rightAttributes, _, _, left, right) => + val leftAttributes2 = leftAttributes.map(x => resolveExpressionBottomUp(x, left).asInstanceOf[Attribute]) + val rightAttributes2 = rightAttributes.map(x => resolveExpressionBottomUp(x, right).asInstanceOf[Attribute]) + f.copy(leftAttributes=leftAttributes2, rightAttributes=rightAttributes2) case i @ Intersect(left, right, _) if !i.duplicateResolved => i.copy(right = dedupRight(left, right)) case e @ Except(left, right, _) if !e.duplicateResolved => @@ -2269,6 +2273,7 @@ class Analyzer( } } + /** * Removes natural or using joins by calculating output columns based on output from two sides, * Then apply a Project on a normal Join to eliminate natural or using join. From 8007fa66dd0810cd822137ae24f9ede08e498114 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Thu, 27 Jun 2019 07:54:06 +0100 Subject: [PATCH 08/34] common trait for grouped mandas udfs --- .../spark/sql/RelationalGroupedDataset.scala | 14 +- .../python/AbstractPandasGroupExec.scala | 128 ++++++++++++++++++ .../python/FlatMapCoGroupsInPandasExec.scala | 44 ++---- .../python/FlatMapGroupsInPandasExec.scala | 80 +---------- 4 files changed, 156 insertions(+), 110 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/python/AbstractPandasGroupExec.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index 147cc00c0ba91..0018f6379e8fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -47,8 +47,8 @@ import org.apache.spark.sql.types.{NumericType, StructType} */ @Stable class RelationalGroupedDataset protected[sql]( - private val df: DataFrame, - private val groupingExprs: Seq[Expression], + val df: DataFrame, + val groupingExprs: Seq[Expression], groupType: RelationalGroupedDataset.GroupType) { private[this] def toDF(aggExprs: Seq[Expression]): DataFrame = { @@ -542,11 +542,15 @@ class RelationalGroupedDataset protected[sql]( val leftAttributes = leftGroupingNamedExpressions.map(_.toAttribute) val rightAttributes = rightGroupingNamedExpressions.map(_.toAttribute) - val left = df.logicalPlan - val right = r.df.logicalPlan + + val leftChild = df.logicalPlan + val rightChild = r.df.logicalPlan + + val left = Project(leftGroupingNamedExpressions ++ leftChild.output, leftChild) + val right = Project(rightGroupingNamedExpressions ++ rightChild.output, rightChild) + val output = expr.dataType.asInstanceOf[StructType].toAttributes val plan = FlatMapCoGroupsInPandas(leftAttributes, rightAttributes, expr, output, left, right) - Dataset.ofRows(df.sparkSession, plan) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/AbstractPandasGroupExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/AbstractPandasGroupExec.scala new file mode 100644 index 0000000000000..0305f79557781 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/AbstractPandasGroupExec.scala @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.python + +import org.apache.spark.TaskContext +import org.apache.spark.api.python.{BasePythonRunner, ChainedPythonFunctions} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, PythonUDF, UnsafeProjection} +import org.apache.spark.sql.execution.{GroupedIterator, SparkPlan} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.ArrowUtils +import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch} + +import scala.collection.mutable.ArrayBuffer +import scala.collection.JavaConverters._ + +trait AbstractPandasGroupExec extends SparkPlan { + + protected val sessionLocalTimeZone = conf.sessionLocalTimeZone + + protected val pythonRunnerConf = ArrowUtils.getPythonRunnerConfMap(conf) + + protected def chainedFunc = Seq( + ChainedPythonFunctions(Seq(func.asInstanceOf[PythonUDF].func))) + + def output: Seq[Attribute] + + def func: Expression + + protected def executePython[T](data: Iterator[T], + runner: BasePythonRunner[T, ColumnarBatch]): Iterator[InternalRow] = { + + val context = TaskContext.get() + val columnarBatchIter = runner.compute(data, context.partitionId(), context) + val unsafeProj = UnsafeProjection.create(output, output) + + columnarBatchIter.flatMap { batch => + // UDF returns a StructType column in ColumnarBatch, select the children here + val structVector = batch.column(0).asInstanceOf[ArrowColumnVector] + val outputVectors = output.indices.map(structVector.getChild) + val flattenedBatch = new ColumnarBatch(outputVectors.toArray) + flattenedBatch.setNumRows(batch.numRows()) + flattenedBatch.rowIterator.asScala + }.map(unsafeProj) + + } + + protected def groupAndDedup( + input: Iterator[InternalRow], groupingAttributes: Seq[Attribute], + inputSchema: Seq[Attribute], dedupSchema: Seq[Attribute]): Iterator[Iterator[InternalRow]] = { + if (groupingAttributes.isEmpty) { + Iterator(input) + } else { + val groupedIter = GroupedIterator(input, groupingAttributes, inputSchema) + val dedupProj = UnsafeProjection.create(dedupSchema, inputSchema) + groupedIter.map { + case (_, groupedRowIter) => groupedRowIter.map(dedupProj) + } + } + } + + protected def createSchema(child: SparkPlan, groupingAttributes: Seq[Attribute]) + : (StructType, Seq[Attribute], Array[Array[Int]]) = { + + // Deduplicate the grouping attributes. + // If a grouping attribute also appears in data attributes, then we don't need to send the + // grouping attribute to Python worker. If a grouping attribute is not in data attributes, + // then we need to send this grouping attribute to python worker. + // + // We use argOffsets to distinguish grouping attributes and data attributes as following: + // + // argOffsets[0] is the length of grouping attributes + // argOffsets[1 .. argOffsets[0]+1] is the arg offsets for grouping attributes + // argOffsets[argOffsets[0]+1 .. ] is the arg offsets for data attributes + + val dataAttributes = child.output.drop(groupingAttributes.length) + val groupingIndicesInData = groupingAttributes.map { attribute => + dataAttributes.indexWhere(attribute.semanticEquals) + } + + val groupingArgOffsets = new ArrayBuffer[Int] + val nonDupGroupingAttributes = new ArrayBuffer[Attribute] + val nonDupGroupingSize = groupingIndicesInData.count(_ == -1) + + // Non duplicate grouping attributes are added to nonDupGroupingAttributes and + // their offsets are 0, 1, 2 ... + // Duplicate grouping attributes are NOT added to nonDupGroupingAttributes and + // their offsets are n + index, where n is the total number of non duplicate grouping + // attributes and index is the index in the data attributes that the grouping attribute + // is a duplicate of. + + groupingAttributes.zip(groupingIndicesInData).foreach { + case (attribute, index) => + if (index == -1) { + groupingArgOffsets += nonDupGroupingAttributes.length + nonDupGroupingAttributes += attribute + } else { + groupingArgOffsets += index + nonDupGroupingSize + } + } + + val dataArgOffsets = nonDupGroupingAttributes.length until + (nonDupGroupingAttributes.length + dataAttributes.length) + + val argOffsets = Array(Array(groupingAttributes.length) ++ groupingArgOffsets ++ dataArgOffsets) + + // Attributes after deduplication + val dedupAttributes = nonDupGroupingAttributes ++ dataAttributes + val dedupSchema = StructType.fromAttributes(dedupAttributes) + (dedupSchema, dedupAttributes, argOffsets) + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala index 12620264de087..a70b4763fc0f7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala @@ -17,17 +17,12 @@ package org.apache.spark.sql.execution.python -import scala.collection.JavaConverters._ - -import org.apache.spark.TaskContext -import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType} +import org.apache.spark.api.python.PythonEvalType import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Distribution, Partitioning} import org.apache.spark.sql.execution.{BinaryExecNode, CoGroupedIterator, GroupedIterator, SparkPlan} -import org.apache.spark.sql.util.ArrowUtils -import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch} case class FlatMapCoGroupsInPandasExec( leftGroup: Seq[Attribute], @@ -36,9 +31,7 @@ case class FlatMapCoGroupsInPandasExec( output: Seq[Attribute], left: SparkPlan, right: SparkPlan) - extends BinaryExecNode { - - private val pandasFunction = func.asInstanceOf[PythonUDF].func + extends BinaryExecNode with AbstractPandasGroupExec { override def outputPartitioning: Partitioning = left.outputPartitioning @@ -53,41 +46,30 @@ case class FlatMapCoGroupsInPandasExec( .map(SortOrder(_, Ascending)) :: rightGroup.map(SortOrder(_, Ascending)) :: Nil } - override protected def doExecute(): RDD[InternalRow] = { - val chainedFunc = Seq(ChainedPythonFunctions(Seq(pandasFunction))) - val sessionLocalTimeZone = conf.sessionLocalTimeZone - val pythonRunnerConf = ArrowUtils.getPythonRunnerConfMap(conf) - + val (schemaLeft, attrLeft, _) = createSchema(left, leftGroup) + val (schemaRight, attrRight, _) = createSchema(right, rightGroup) left.execute().zipPartitions(right.execute()) { (leftData, rightData) => val leftGrouped = GroupedIterator(leftData, leftGroup, left.output) val rightGrouped = GroupedIterator(rightData, rightGroup, right.output) - val cogroup = new CoGroupedIterator(leftGrouped, rightGrouped, leftGroup) - .map{case (k, l, r) => (l, r)} - val context = TaskContext.get() + val projLeft = UnsafeProjection.create(attrLeft, left.output) + val projRight = UnsafeProjection.create(attrRight, right.output) + val data = new CoGroupedIterator(leftGrouped, rightGrouped, leftGroup) + .map{case (k, l, r) => (l.map(projLeft), r.map(projRight))} - val columnarBatchIter = new InterleavedArrowPythonRunner( + val runner = new InterleavedArrowPythonRunner( chainedFunc, PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, Array(Array.empty), - left.schema, - right.schema, + schemaLeft, + schemaRight, sessionLocalTimeZone, - pythonRunnerConf).compute(cogroup, context.partitionId(), context) - + pythonRunnerConf) - val unsafeProj = UnsafeProjection.create(output, output) + executePython(data, runner) - columnarBatchIter.flatMap { batch => - // UDF returns a StructType column in ColumnarBatch, select the children here - val structVector = batch.column(0).asInstanceOf[ArrowColumnVector] - val outputVectors = output.indices.map(structVector.getChild) - val flattenedBatch = new ColumnarBatch(outputVectors.toArray) - flattenedBatch.setNumRows(batch.numRows()) - flattenedBatch.rowIterator.asScala - }.map(unsafeProj) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala index 267698d1bca50..474bbe04b8d62 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala @@ -53,9 +53,7 @@ case class FlatMapGroupsInPandasExec( func: Expression, output: Seq[Attribute], child: SparkPlan) - extends UnaryExecNode { - - private val pandasFunction = func.asInstanceOf[PythonUDF].func + extends UnaryExecNode with AbstractPandasGroupExec { override def outputPartitioning: Partitioning = child.outputPartitioning @@ -75,88 +73,22 @@ case class FlatMapGroupsInPandasExec( override protected def doExecute(): RDD[InternalRow] = { val inputRDD = child.execute() - val chainedFunc = Seq(ChainedPythonFunctions(Seq(pandasFunction))) - val sessionLocalTimeZone = conf.sessionLocalTimeZone - val pythonRunnerConf = ArrowUtils.getPythonRunnerConfMap(conf) - - // Deduplicate the grouping attributes. - // If a grouping attribute also appears in data attributes, then we don't need to send the - // grouping attribute to Python worker. If a grouping attribute is not in data attributes, - // then we need to send this grouping attribute to python worker. - // - // We use argOffsets to distinguish grouping attributes and data attributes as following: - // - // argOffsets[0] is the length of grouping attributes - // argOffsets[1 .. argOffsets[0]+1] is the arg offsets for grouping attributes - // argOffsets[argOffsets[0]+1 .. ] is the arg offsets for data attributes - - val dataAttributes = child.output.drop(groupingAttributes.length) - val groupingIndicesInData = groupingAttributes.map { attribute => - dataAttributes.indexWhere(attribute.semanticEquals) - } - - val groupingArgOffsets = new ArrayBuffer[Int] - val nonDupGroupingAttributes = new ArrayBuffer[Attribute] - val nonDupGroupingSize = groupingIndicesInData.count(_ == -1) - - // Non duplicate grouping attributes are added to nonDupGroupingAttributes and - // their offsets are 0, 1, 2 ... - // Duplicate grouping attributes are NOT added to nonDupGroupingAttributes and - // their offsets are n + index, where n is the total number of non duplicate grouping - // attributes and index is the index in the data attributes that the grouping attribute - // is a duplicate of. - - groupingAttributes.zip(groupingIndicesInData).foreach { - case (attribute, index) => - if (index == -1) { - groupingArgOffsets += nonDupGroupingAttributes.length - nonDupGroupingAttributes += attribute - } else { - groupingArgOffsets += index + nonDupGroupingSize - } - } - - val dataArgOffsets = nonDupGroupingAttributes.length until - (nonDupGroupingAttributes.length + dataAttributes.length) - - val argOffsets = Array(Array(groupingAttributes.length) ++ groupingArgOffsets ++ dataArgOffsets) - - // Attributes after deduplication - val dedupAttributes = nonDupGroupingAttributes ++ dataAttributes - val dedupSchema = StructType.fromAttributes(dedupAttributes) + val (dedupSchema, dedupAttributes, argOffsets) = createSchema(child, groupingAttributes) // Map grouped rows to ArrowPythonRunner results, Only execute if partition is not empty inputRDD.mapPartitionsInternal { iter => if (iter.isEmpty) iter else { - val grouped = if (groupingAttributes.isEmpty) { - Iterator(iter) - } else { - val groupedIter = GroupedIterator(iter, groupingAttributes, child.output) - val dedupProj = UnsafeProjection.create(dedupAttributes, child.output) - groupedIter.map { - case (_, groupedRowIter) => groupedRowIter.map(dedupProj) - } - } - val context = TaskContext.get() + val data = groupAndDedup(iter, groupingAttributes, child.output, dedupAttributes) - val columnarBatchIter = new ArrowPythonRunner( + val runner = new ArrowPythonRunner( chainedFunc, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF, argOffsets, dedupSchema, sessionLocalTimeZone, - pythonRunnerConf).compute(grouped, context.partitionId(), context) - - val unsafeProj = UnsafeProjection.create(output, output) + pythonRunnerConf) - columnarBatchIter.flatMap { batch => - // Grouped Map UDF returns a StructType column in ColumnarBatch, select the children here - val structVector = batch.column(0).asInstanceOf[ArrowColumnVector] - val outputVectors = output.indices.map(structVector.getChild) - val flattenedBatch = new ColumnarBatch(outputVectors.toArray) - flattenedBatch.setNumRows(batch.numRows()) - flattenedBatch.rowIterator.asScala - }.map(unsafeProj) + executePython(data, runner) }} } } From d4cf6d060efffc320fb46530371d8a79b9c74dae Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Thu, 27 Jun 2019 11:21:59 +0100 Subject: [PATCH 09/34] poc using arrow streams --- python/pyspark/serializers.py | 27 +++--- ...upExec.scala => BasePandasGroupExec.scala} | 29 +++---- .../python/FlatMapCoGroupsInPandasExec.scala | 17 ++-- .../python/FlatMapGroupsInPandasExec.scala | 5 +- .../python/InterleavedArrowPythonRunner.scala | 63 ++++++-------- .../python/InterleavedArrowWriter.scala | 85 ------------------- 6 files changed, 62 insertions(+), 164 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/python/{AbstractPandasGroupExec.scala => BasePandasGroupExec.scala} (86%) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowWriter.scala diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 9616450b1803c..81aac681f9790 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -359,19 +359,24 @@ def __repr__(self): class InterleavedArrowReader(object): def __init__(self, stream): - import pyarrow as pa - self._schema1 = pa.read_schema(stream) - self._schema2 = pa.read_schema(stream) - self._reader = pa.MessageReader.open_stream(stream) + self._stream = stream def __iter__(self): return self def __next__(self): + stream_status = read_int(self._stream) + if stream_status == SpecialLengths.START_ARROW_STREAM: + return self._read_df(), self._read_df() + elif stream_status == SpecialLengths.END_OF_DATA_SECTION: + raise StopIteration + else: + raise ValueError('Received invalid stream status {0}'.format(stream_status)) + + def _read_df(self): import pyarrow as pa - batch1 = pa.read_record_batch(self._reader.read_next_message(), self._schema1) - batch2 = pa.read_record_batch(self._reader.read_next_message(), self._schema2) - return batch1, batch2 + reader = pa.ipc.open_stream(self._stream) + return [b for b in reader] class ArrowStreamPandasUDFSerializer(ArrowStreamPandasSerializer): @@ -428,11 +433,11 @@ def load_stream(self, stream): """ Deserialize ArrowRecordBatches to an Arrow table and return as a list of pandas.Series. """ - import pyarrow as pa - reader = InterleavedArrowReader(pa.input_stream(stream)) + reader = InterleavedArrowReader(stream) for batch1, batch2 in reader: - yield ( [self.arrow_to_pandas(c) for c in pa.Table.from_batches([batch1]).itercolumns()], - [self.arrow_to_pandas(c) for c in pa.Table.from_batches([batch2]).itercolumns()]) + import pyarrow as pa + yield ([self.arrow_to_pandas(c) for c in pa.Table.from_batches(batch1).itercolumns()], + [self.arrow_to_pandas(c) for c in pa.Table.from_batches(batch2).itercolumns()]) class BatchedSerializer(Serializer): diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/AbstractPandasGroupExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala similarity index 86% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/python/AbstractPandasGroupExec.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala index 0305f79557781..4945a99482907 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/AbstractPandasGroupExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala @@ -20,7 +20,7 @@ import org.apache.spark.TaskContext import org.apache.spark.api.python.{BasePythonRunner, ChainedPythonFunctions} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, PythonUDF, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression, PythonUDF, UnsafeProjection} import org.apache.spark.sql.execution.{GroupedIterator, SparkPlan} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.ArrowUtils @@ -29,18 +29,19 @@ import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch} import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConverters._ -trait AbstractPandasGroupExec extends SparkPlan { +abstract class BasePandasGroupExec(func: Expression, + output: Seq[Attribute]) extends SparkPlan { protected val sessionLocalTimeZone = conf.sessionLocalTimeZone protected val pythonRunnerConf = ArrowUtils.getPythonRunnerConfMap(conf) - protected def chainedFunc = Seq( - ChainedPythonFunctions(Seq(func.asInstanceOf[PythonUDF].func))) + protected val pandasFunction = func.asInstanceOf[PythonUDF].func - def output: Seq[Attribute] + protected val chainedFunc = Seq(ChainedPythonFunctions(Seq(pandasFunction))) + + override def producedAttributes: AttributeSet = AttributeSet(output) - def func: Expression protected def executePython[T](data: Iterator[T], runner: BasePythonRunner[T, ColumnarBatch]): Iterator[InternalRow] = { @@ -62,16 +63,12 @@ trait AbstractPandasGroupExec extends SparkPlan { protected def groupAndDedup( input: Iterator[InternalRow], groupingAttributes: Seq[Attribute], - inputSchema: Seq[Attribute], dedupSchema: Seq[Attribute]): Iterator[Iterator[InternalRow]] = { - if (groupingAttributes.isEmpty) { - Iterator(input) - } else { - val groupedIter = GroupedIterator(input, groupingAttributes, inputSchema) - val dedupProj = UnsafeProjection.create(dedupSchema, inputSchema) - groupedIter.map { - case (_, groupedRowIter) => groupedRowIter.map(dedupProj) - } - } + inputSchema: Seq[Attribute], dedupSchema: Seq[Attribute]): Iterator[(InternalRow, Iterator[InternalRow])] = { + val groupedIter = GroupedIterator(input, groupingAttributes, inputSchema) + val dedupProj = UnsafeProjection.create(dedupSchema, inputSchema) + groupedIter.map { + case (k, groupedRowIter) => (k, groupedRowIter.map(dedupProj)) + } } protected def createSchema(child: SparkPlan, groupingAttributes: Seq[Attribute]) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala index a70b4763fc0f7..0edf5b87cff61 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala @@ -31,12 +31,10 @@ case class FlatMapCoGroupsInPandasExec( output: Seq[Attribute], left: SparkPlan, right: SparkPlan) - extends BinaryExecNode with AbstractPandasGroupExec { + extends BasePandasGroupExec(func, output) with BinaryExecNode{ override def outputPartitioning: Partitioning = left.outputPartitioning - override def producedAttributes: AttributeSet = AttributeSet(output) - override def requiredChildDistribution: Seq[Distribution] = { ClusteredDistribution(leftGroup) :: ClusteredDistribution(rightGroup) :: Nil } @@ -48,16 +46,15 @@ case class FlatMapCoGroupsInPandasExec( override protected def doExecute(): RDD[InternalRow] = { - val (schemaLeft, attrLeft, _) = createSchema(left, leftGroup) - val (schemaRight, attrRight, _) = createSchema(right, rightGroup) + val (schemaLeft, leftDedup, _) = createSchema(left, leftGroup) + val (schemaRight, rightDedup, _) = createSchema(right, rightGroup) left.execute().zipPartitions(right.execute()) { (leftData, rightData) => - val leftGrouped = GroupedIterator(leftData, leftGroup, left.output) - val rightGrouped = GroupedIterator(rightData, rightGroup, right.output) - val projLeft = UnsafeProjection.create(attrLeft, left.output) - val projRight = UnsafeProjection.create(attrRight, right.output) + + val leftGrouped = groupAndDedup(leftData, leftGroup, left.output, leftDedup) + val rightGrouped = groupAndDedup(rightData, rightGroup, right.output, rightDedup) val data = new CoGroupedIterator(leftGrouped, rightGrouped, leftGroup) - .map{case (k, l, r) => (l.map(projLeft), r.map(projRight))} + .map{case (k, l, r) => (l, r)} val runner = new InterleavedArrowPythonRunner( chainedFunc, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala index 474bbe04b8d62..eab1403619237 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala @@ -53,12 +53,10 @@ case class FlatMapGroupsInPandasExec( func: Expression, output: Seq[Attribute], child: SparkPlan) - extends UnaryExecNode with AbstractPandasGroupExec { + extends BasePandasGroupExec(func, output) with UnaryExecNode { override def outputPartitioning: Partitioning = child.outputPartitioning - override def producedAttributes: AttributeSet = AttributeSet(output) - override def requiredChildDistribution: Seq[Distribution] = { if (groupingAttributes.isEmpty) { AllTuples :: Nil @@ -79,6 +77,7 @@ case class FlatMapGroupsInPandasExec( inputRDD.mapPartitionsInternal { iter => if (iter.isEmpty) iter else { val data = groupAndDedup(iter, groupingAttributes, child.output, dedupAttributes) + .map{case(_, x) => x} val runner = new ArrowPythonRunner( chainedFunc, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala index b39885ee47a2d..52223e5103bba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala @@ -21,7 +21,8 @@ import java.io._ import java.net._ import org.apache.arrow.vector.VectorSchemaRoot - +import org.apache.arrow.vector.dictionary.DictionaryProvider +import org.apache.arrow.vector.ipc.ArrowStreamWriter import org.apache.spark._ import org.apache.spark.api.python._ import org.apache.spark.sql.catalyst.InternalRow @@ -64,55 +65,39 @@ class InterleavedArrowPythonRunner( } protected override def writeIteratorToStream(dataOut: DataOutputStream): Unit = { - val leftArrowSchema = ArrowUtils.toArrowSchema(leftSchema, timeZoneId) - val rightArrowSchema = ArrowUtils.toArrowSchema(rightSchema, timeZoneId) + while (inputIterator.hasNext) { + dataOut.writeInt(SpecialLengths.START_ARROW_STREAM) + val (nextLeft, nextRight) = inputIterator.next() + writeGroup(nextLeft, leftSchema, dataOut) + writeGroup(nextRight, rightSchema, dataOut) + } + dataOut.writeInt(SpecialLengths.END_OF_DATA_SECTION) + } + + def writeGroup(group: Iterator[InternalRow], schema: StructType, dataOut: DataOutputStream + ) = { + val arrowSchema = ArrowUtils.toArrowSchema(schema, timeZoneId) val allocator = ArrowUtils.rootAllocator.newChildAllocator( s"stdout writer for $pythonExec", 0, Long.MaxValue) - val leftRoot = VectorSchemaRoot.create(leftArrowSchema, allocator) - val rightRoot = VectorSchemaRoot.create(rightArrowSchema, allocator) + val root = VectorSchemaRoot.create(arrowSchema, allocator) Utils.tryWithSafeFinally { - val leftArrowWriter = ArrowWriter.create(leftRoot) - val rightArrowWriter = ArrowWriter.create(rightRoot) - val writer = InterleavedArrowWriter(leftRoot, rightRoot, dataOut) + val writer = new ArrowStreamWriter(root, null, dataOut) + val arrowWriter = ArrowWriter.create(root) writer.start() - while (inputIterator.hasNext) { - - val (nextLeft, nextRight) = inputIterator.next() - - while (nextLeft.hasNext) { - leftArrowWriter.write(nextLeft.next()) - } - while (nextRight.hasNext) { - rightArrowWriter.write(nextRight.next()) - } - leftArrowWriter.finish() - rightArrowWriter.finish() - writer.writeBatch() - leftArrowWriter.reset() - rightArrowWriter.reset() + while (group.hasNext) { + arrowWriter.write(group.next()) } - // end writes footer to the output stream and doesn't clean any resources. - // It could throw exception if the output stream is closed, so it should be - // in the try block. + arrowWriter.finish() + writer.writeBatch() writer.end() - } { - // If we close root and allocator in TaskCompletionListener, there could be a race - // condition where the writer thread keeps writing to the VectorSchemaRoot while - // it's being closed by the TaskCompletion listener. - // Closing root and allocator here is cleaner because root and allocator is owned - // by the writer thread and is only visible to the writer thread. - // - // If the writer thread is interrupted by TaskCompletionListener, it should either - // (1) in the try block, in which case it will get an InterruptedException when - // performing io, and goes into the finally block or (2) in the finally block, - // in which case it will ignore the interruption and close the resources. - leftRoot.close() - rightRoot.close() + }{ + root.close() allocator.close() } } } } } + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowWriter.scala deleted file mode 100644 index eb9f1d4494b91..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowWriter.scala +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.python - -import java.io.OutputStream -import java.nio.channels.Channels - -import org.apache.arrow.vector.{VectorSchemaRoot, VectorUnloader} -import org.apache.arrow.vector.ipc.WriteChannel -import org.apache.arrow.vector.ipc.message.{ArrowRecordBatch, MessageSerializer} - - -class InterleavedArrowWriter( leftRoot: VectorSchemaRoot, - rightRoot: VectorSchemaRoot, - out: WriteChannel) extends AutoCloseable{ - - - private var started = false - private val leftUnloader = new VectorUnloader(leftRoot) - private val rightUnloader = new VectorUnloader(rightRoot) - - def start(): Unit = { - this.ensureStarted() - } - - def writeBatch(): Unit = { - this.ensureStarted() - writeRecordBatch(leftUnloader.getRecordBatch) - writeRecordBatch(rightUnloader.getRecordBatch) - } - - private def writeRecordBatch(b: ArrowRecordBatch): Unit = { - try - MessageSerializer.serialize(out, b) - finally - b.close() - } - - private def ensureStarted(): Unit = { - if (!started) { - started = true - MessageSerializer.serialize(out, leftRoot.getSchema) - MessageSerializer.serialize(out, rightRoot.getSchema) - } - } - - def end(): Unit = { - ensureStarted() - ensureEnded() - } - - def ensureEnded(): Unit = { - out.writeIntLittleEndian(0) - } - - def close(): Unit = { - out.close() - } - -} - -object InterleavedArrowWriter{ - - def apply(leftRoot: VectorSchemaRoot, - rightRoot: VectorSchemaRoot, - out: OutputStream): InterleavedArrowWriter = { - new InterleavedArrowWriter(leftRoot, rightRoot, new WriteChannel(Channels.newChannel(out))) - } - -} From 87aeb92a4fbda8e1c90f90e025d984fff58f1d3f Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Thu, 27 Jun 2019 15:21:18 +0100 Subject: [PATCH 10/34] more unit tests fro cogroup --- .../tests/test_pandas_udf_cogrouped_map.py | 78 ++++++++++++++++--- .../python/FlatMapCoGroupsInPandasExec.scala | 2 +- .../python/InterleavedArrowPythonRunner.scala | 3 +- 3 files changed, 68 insertions(+), 15 deletions(-) diff --git a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py index 332bc260541a6..0acbd954e940c 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py +++ b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py @@ -69,24 +69,78 @@ def data2(self): .drop('ks') def test_simple(self): - import pandas as pd + self._test_merge(self.data1, self.data2) + + def test_left_group_empty(self): + left = self.data1.where(col("id") % 2 == 0) + self._test_merge(left, self.data2) + + def test_right_group_empty(self): + right = self.data2.where(col("id") % 2 == 0) + self._test_merge(self.data1, right) + + def test_different_schemas(self): + right = self.data2.withColumn('v3', lit('a')) + self._test_merge(self.data1, right, output_schema='id long, k int, v int, v2 int, v3 string') + + def test_complex_group_by(self): + left = pd.DataFrame.from_dict({ + 'id': [1, 2, 3], + 'k': [5, 6, 7], + 'v': [9, 10, 11] + }) + + right = pd.DataFrame.from_dict({ + 'id': [11, 12, 13], + 'k': [5, 6, 7], + 'v2': [90, 100, 110] + }) + + left_df = self.spark\ + .createDataFrame(left)\ + .groupby(col('id') % 2 == 0) + + right_df = self.spark \ + .createDataFrame(right) \ + .groupby(col('id') % 2 == 0) + + @pandas_udf('k long, v long, v2 long', PandasUDFType.COGROUPED_MAP) + def merge_pandas(l, r): + return pd.merge(l[['k', 'v']], r[['k', 'v2']], on=['k']) + + result = left_df \ + .cogroup(right_df) \ + .apply(merge_pandas) \ + .sort(['k']) \ + .toPandas() + + expected = pd.DataFrame.from_dict({ + 'k': [5, 6, 7], + 'v': [9, 10, 11], + 'v2': [90, 100, 110] + }) - l = self.data1 - r = self.data2 + assert_frame_equal(expected, result, check_column_type=_check_column_type) - @pandas_udf('id long, k int, v int, v2 int', PandasUDFType.COGROUPED_MAP) - def merge_pandas(left, right): - return pd.merge(left, right, how='outer', on=['k', 'id']) + def _test_merge(self, left, right, output_schema='id long, k int, v int, v2 int'): - result = l\ - .groupby('id')\ - .cogroup(r.groupby(r.id))\ + @pandas_udf(output_schema, PandasUDFType.COGROUPED_MAP) + def merge_pandas(l, r): + return pd.merge(l, r, on=['id', 'k']) + + result = left \ + .groupby('id') \ + .cogroup(right.groupby('id')) \ .apply(merge_pandas)\ - .sort(['id', 'k'])\ + .sort(['id', 'k']) \ .toPandas() - expected = pd\ - .merge(l.toPandas(), r.toPandas(), how='outer', on=['k', 'id']) + left = left.toPandas() + right = right.toPandas() + + expected = pd \ + .merge(left, right, on=['id', 'k']) \ + .sort_values(by=['id', 'k']) assert_frame_equal(expected, result, check_column_type=_check_column_type) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala index 0edf5b87cff61..84f2b1dd3f070 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala @@ -22,7 +22,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Distribution, Partitioning} -import org.apache.spark.sql.execution.{BinaryExecNode, CoGroupedIterator, GroupedIterator, SparkPlan} +import org.apache.spark.sql.execution.{BinaryExecNode, CoGroupedIterator, SparkPlan} case class FlatMapCoGroupsInPandasExec( leftGroup: Seq[Attribute], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala index 52223e5103bba..2a32a2a8bc64d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala @@ -21,15 +21,14 @@ import java.io._ import java.net._ import org.apache.arrow.vector.VectorSchemaRoot -import org.apache.arrow.vector.dictionary.DictionaryProvider import org.apache.arrow.vector.ipc.ArrowStreamWriter + import org.apache.spark._ import org.apache.spark.api.python._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.arrow.ArrowWriter import org.apache.spark.sql.types._ import org.apache.spark.sql.util.ArrowUtils -import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils From e7528d0f8ae705fb9d7e8741a391a21afbf74de6 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Tue, 2 Jul 2019 09:00:02 +0100 Subject: [PATCH 11/34] argspec includes grouping key --- python/pyspark/worker.py | 41 ++++++++++++++++--- .../python/BasePandasGroupExec.scala | 7 ++-- .../python/FlatMapCoGroupsInPandasExec.scala | 8 ++-- .../python/FlatMapGroupsInPandasExec.scala | 2 +- 4 files changed, 44 insertions(+), 14 deletions(-) diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index e42f4ac698c8a..023290f871959 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -111,11 +111,11 @@ def verify_result_length(result, length): map(verify_result_type, f(*iterator))) -def wrap_cogrouped_map_pandas_udf(f, return_type): +def wrap_cogrouped_map_pandas_udf(f, return_type, argspec): - def wrapped(left, right): + def wrapped(key_Series, left_value_series, right_value_series): import pandas as pd - result = f(pd.concat(left, axis=1), pd.concat(right, axis=1)) + result = f(pd.concat(left_value_series, axis=1), pd.concat(right_value_series, axis=1)) if not isinstance(result, pd.DataFrame): raise TypeError("Return type of the user-defined function should be " "pandas.DataFrame, but is {}".format(type(result))) @@ -126,7 +126,7 @@ def wrapped(left, right): "Expected: {} Actual: {}".format(len(return_type), len(result.columns))) return result - return lambda v: [(wrapped(v[0], v[1]), to_arrow_type(return_type))] + return lambda k, vl, vr: [(wrapped(k, vl, vr), to_arrow_type(return_type))] def wrap_grouped_map_pandas_udf(f, return_type, argspec): @@ -250,7 +250,8 @@ def read_single_udf(pickleSer, infile, eval_type, runner_conf, udf_index): argspec = _get_argspec(chained_func) # signature was lost when wrapping it return arg_offsets, wrap_grouped_map_pandas_udf(func, return_type, argspec) elif eval_type == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: - return arg_offsets, wrap_cogrouped_map_pandas_udf(func, return_type) + argspec = _get_argspec(chained_func) # signature was lost when wrapping it + return arg_offsets, wrap_cogrouped_map_pandas_udf(func, return_type, argspec) elif eval_type == PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF: return arg_offsets, wrap_grouped_agg_pandas_udf(func, return_type) elif eval_type == PythonEvalType.SQL_WINDOW_AGG_PANDAS_UDF: @@ -373,7 +374,12 @@ def map_batch(batch): arg_offsets, udf = read_single_udf( pickleSer, infile, eval_type, runner_conf, udf_index=0) udfs['f'] = udf - mapper_str = "lambda a: f(a)" + parsed_offsets = parse_grouped_arg_offsets(arg_offsets) + arg0 = ["a[0][%d]" % o for o in parsed_offsets[0][0]] + arg1 = ["a[0][%d]" % o for o in parsed_offsets[0][1]] + arg2 = ["a[1][%d]" % o for o in parsed_offsets[1][1]] + mapper_str = "lambda a: f([%s], [%s], [%s])" % (", ".join(arg0), ", ".join(arg1), ", ".join(arg2)) + print("mapper string is " + mapper_str) else: # Create function like this: # lambda a: (f0(a[0]), f1(a[1], a[2]), f2(a[3])) @@ -394,6 +400,29 @@ def map_batch(batch): return func, None, ser, ser +def parse_grouped_arg_offsets(arg_offsets): + print('arg offsets are ' + str(arg_offsets)) + parsed = [] + i = 0 + while i < len(arg_offsets): + offsets_len = arg_offsets[i] + print('i is ' + str(i)) + print('offsets_len is ' + str(offsets_len)) + i += 1 + offsets = arg_offsets[i: i + offsets_len] + print('offsets are ' + str(offsets)) + split_index = offsets[0] + 1 + print('split index is ' + str(split_index)) + keys = offsets[1: split_index] + values = offsets[split_index:] + print('keys are ' + str(keys)) + print('values are ' + str(values)) + parsed.append([keys, values]) + i += offsets_len + return parsed + + + def main(infile, outfile): try: boot_time = time.time() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala index 4945a99482907..f6ceaf3561370 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.python import org.apache.spark.TaskContext import org.apache.spark.api.python.{BasePythonRunner, ChainedPythonFunctions} -import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression, PythonUDF, UnsafeProjection} import org.apache.spark.sql.execution.{GroupedIterator, SparkPlan} @@ -72,7 +71,7 @@ abstract class BasePandasGroupExec(func: Expression, } protected def createSchema(child: SparkPlan, groupingAttributes: Seq[Attribute]) - : (StructType, Seq[Attribute], Array[Array[Int]]) = { + : (StructType, Seq[Attribute], Array[Int]) = { // Deduplicate the grouping attributes. // If a grouping attribute also appears in data attributes, then we don't need to send the @@ -114,7 +113,9 @@ abstract class BasePandasGroupExec(func: Expression, val dataArgOffsets = nonDupGroupingAttributes.length until (nonDupGroupingAttributes.length + dataAttributes.length) - val argOffsets = Array(Array(groupingAttributes.length) ++ groupingArgOffsets ++ dataArgOffsets) + val argOffsetsLength = groupingAttributes.length + dataArgOffsets.length + 1 + val argOffsets = Array(argOffsetsLength, + groupingAttributes.length) ++ groupingArgOffsets ++ dataArgOffsets // Attributes after deduplication val dedupAttributes = nonDupGroupingAttributes ++ dataAttributes diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala index 84f2b1dd3f070..d47111216a14b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala @@ -46,20 +46,20 @@ case class FlatMapCoGroupsInPandasExec( override protected def doExecute(): RDD[InternalRow] = { - val (schemaLeft, leftDedup, _) = createSchema(left, leftGroup) - val (schemaRight, rightDedup, _) = createSchema(right, rightGroup) + val (schemaLeft, leftDedup, leftArgOffsets) = createSchema(left, leftGroup) + val (schemaRight, rightDedup, rightArgOffsets) = createSchema(right, rightGroup) left.execute().zipPartitions(right.execute()) { (leftData, rightData) => val leftGrouped = groupAndDedup(leftData, leftGroup, left.output, leftDedup) val rightGrouped = groupAndDedup(rightData, rightGroup, right.output, rightDedup) val data = new CoGroupedIterator(leftGrouped, rightGrouped, leftGroup) - .map{case (k, l, r) => (l, r)} + .map{case (_, l, r) => (l, r)} val runner = new InterleavedArrowPythonRunner( chainedFunc, PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, - Array(Array.empty), + Array(leftArgOffsets ++ rightArgOffsets), schemaLeft, schemaRight, sessionLocalTimeZone, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala index eab1403619237..960f5131358c7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala @@ -82,7 +82,7 @@ case class FlatMapGroupsInPandasExec( val runner = new ArrowPythonRunner( chainedFunc, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF, - argOffsets, + Array(argOffsets), dedupSchema, sessionLocalTimeZone, pythonRunnerConf) From b85ec750e288eec48660e03c0f6db26f4efef67a Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Tue, 2 Jul 2019 10:33:11 +0100 Subject: [PATCH 12/34] fixed tests und --- python/pyspark/serializers.py | 3 +++ python/pyspark/sql/cogroup.py | 17 +++++++++++++++++ .../sql/tests/test_pandas_udf_cogrouped_map.py | 10 ++++++++++ 3 files changed, 30 insertions(+) diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 81aac681f9790..3c605a7a22262 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -373,6 +373,9 @@ def __next__(self): else: raise ValueError('Received invalid stream status {0}'.format(stream_status)) + def next(self): + return self.__next__() + def _read_df(self): import pyarrow as pa reader = pa.ipc.open_stream(self._stream) diff --git a/python/pyspark/sql/cogroup.py b/python/pyspark/sql/cogroup.py index 18dc397c8e348..b758d1b7d8c45 100644 --- a/python/pyspark/sql/cogroup.py +++ b/python/pyspark/sql/cogroup.py @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + from pyspark.sql.dataframe import DataFrame diff --git a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py index 0acbd954e940c..d6e09d797062c 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py +++ b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py @@ -144,3 +144,13 @@ def merge_pandas(l, r): assert_frame_equal(expected, result, check_column_type=_check_column_type) + +if __name__ == "__main__": + from pyspark.sql.tests.test_pandas_udf_cogrouped_map import * + + try: + import xmlrunner + testRunner = xmlrunner.XMLTestRunner(output='target/test-reports', verbosity=2) + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) \ No newline at end of file From 6a8ecff306814ec6a2a4599815f9a8c143382e55 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Tue, 2 Jul 2019 18:43:39 +0100 Subject: [PATCH 13/34] keys now handled properly. Validation of udf. More tests --- python/pyspark/sql/cogroup.py | 7 ++ .../tests/test_pandas_udf_cogrouped_map.py | 107 +++++++++++++++++- python/pyspark/sql/udf.py | 17 +++ python/pyspark/worker.py | 29 ++--- 4 files changed, 143 insertions(+), 17 deletions(-) diff --git a/python/pyspark/sql/cogroup.py b/python/pyspark/sql/cogroup.py index b758d1b7d8c45..09cd7b0c6fbfa 100644 --- a/python/pyspark/sql/cogroup.py +++ b/python/pyspark/sql/cogroup.py @@ -16,6 +16,8 @@ # from pyspark.sql.dataframe import DataFrame +from pyspark.rdd import PythonEvalType +from pyspark.sql.column import Column class CoGroupedData(object): @@ -26,6 +28,11 @@ def __init__(self, gd1, gd2): self.sql_ctx = gd1.sql_ctx def apply(self, udf): + # Columns are special because hasattr always return True + if isinstance(udf, Column) or not hasattr(udf, 'func') \ + or udf.evalType != PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: + raise ValueError("Invalid udf: the udf argument must be a pandas_udf of type " + "COGROUPED_MAP.") all_cols = self._extract_cols(self._gd1) + self._extract_cols(self._gd2) udf_column = udf(*all_cols) jdf = self._gd1._jgd.flatMapCoGroupsInPandas(self._gd2._jgd, udf_column._jc.expr()) diff --git a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py index d6e09d797062c..54397e04c40cc 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py +++ b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py @@ -31,7 +31,7 @@ if have_pandas: import pandas as pd - from pandas.util.testing import assert_frame_equal + from pandas.util.testing import assert_frame_equal, assert_series_equal if have_pyarrow: import pyarrow as pa @@ -96,7 +96,7 @@ def test_complex_group_by(self): 'v2': [90, 100, 110] }) - left_df = self.spark\ + left_df = self.spark\ .createDataFrame(left)\ .groupby(col('id') % 2 == 0) @@ -122,7 +122,108 @@ def merge_pandas(l, r): assert_frame_equal(expected, result, check_column_type=_check_column_type) - def _test_merge(self, left, right, output_schema='id long, k int, v int, v2 int'): + def test_with_key_left(self): + self._test_with_key_left(self.data1, self.data2) + + def test_with_key_right(self): + self._test_with_key_right(self.data1, self.data2) + + def test_with_key_left_group_empty(self): + left = self.data1.where(col("id") % 2 == 0) + self._test_with_key_right(left, self.data2) + + def test_with_key_right_group_empty(self): + right = self.data2.where(col("id") % 2 == 0) + self._test_with_key_left(self.data1, right) + + def test_with_key_complex(self): + + @pandas_udf('id long, k int, v int, key boolean', PandasUDFType.COGROUPED_MAP) + def left_assign_key(key, l, _): + return l.assign(key=key[0]) + + result = self.data1 \ + .groupby(col('id') % 2 == 0)\ + .cogroup(self.data2.groupby(col('id') % 2 == 0)) \ + .apply(left_assign_key) \ + .sort(['id', 'k']) \ + .toPandas() + + expected = self.data1.toPandas() + expected = expected.assign(key=expected.id % 2 == 0) + + assert_frame_equal(expected, result, check_column_type=_check_column_type) + + def test_wrong_return_type(self): + with QuietTest(self.sc): + with self.assertRaisesRegexp( + NotImplementedError, + 'Invalid returnType.*cogrouped map Pandas UDF.*MapType'): + pandas_udf( + lambda l, r: l, + 'id long, v map', + PandasUDFType.COGROUPED_MAP) + + def test_wrong_args(self): + left = self.data1 + right = self.data2 + + with QuietTest(self.sc): + with self.assertRaisesRegexp(ValueError, 'Invalid udf'): + left.groupby('id').cogroup(right.groupby('id')).apply(lambda l, r: l) + with self.assertRaisesRegexp(ValueError, 'Invalid udf'): + left.groupby('id').cogroup(right.groupby('id')).apply(udf(lambda l, r: l, DoubleType())) + with self.assertRaisesRegexp(ValueError, 'Invalid udf'): + left.groupby('id').cogroup(right.groupby('id')).apply(sum(left.v)) + with self.assertRaisesRegexp(ValueError, 'Invalid udf'): + left.groupby('id').cogroup(right.groupby('id')).apply(left.v + 1) + with self.assertRaisesRegexp(ValueError, 'Invalid function'): + left.groupby('id').cogroup(right.groupby('id')).apply( + pandas_udf(lambda: 1, StructType([StructField("d", DoubleType())]))) + with self.assertRaisesRegexp(ValueError, 'Invalid udf'): + left.groupby('id').cogroup(right.groupby('id')).apply(pandas_udf(lambda x, y: x, DoubleType())) + with self.assertRaisesRegexp(ValueError, 'Invalid udf.*COGROUPED_MAP'): + left.groupby('id').cogroup(right.groupby('id')).apply( + pandas_udf(lambda x, y: x, DoubleType(), PandasUDFType.SCALAR)) + + @staticmethod + def _test_with_key_left(left, right): + + @pandas_udf('id long, k int, v int, key long', PandasUDFType.COGROUPED_MAP) + def left_assign_key(key, l, _): + return l.assign(key=key[0]) + + result = left \ + .groupby('id') \ + .cogroup(right.groupby('id')) \ + .apply(left_assign_key) \ + .toPandas() + + expected = left.toPandas() + expected = expected.assign(key=expected.id) + + assert_frame_equal(expected, result, check_column_type=_check_column_type) + + @staticmethod + def _test_with_key_right(left, right): + + @pandas_udf('id long, k int, v2 int, key long', PandasUDFType.COGROUPED_MAP) + def right_assign_key(key, _, r): + return r.assign(key=key[0]) + + result = left \ + .groupby('id') \ + .cogroup(right.groupby('id')) \ + .apply(right_assign_key) \ + .toPandas() + + expected = right.toPandas() + expected = expected.assign(key=expected.id) + + assert_frame_equal(expected, result, check_column_type=_check_column_type) + + @staticmethod + def _test_merge(left, right, output_schema='id long, k int, v int, v2 int'): @pandas_udf(output_schema, PandasUDFType.COGROUPED_MAP) def merge_pandas(l, r): diff --git a/python/pyspark/sql/udf.py b/python/pyspark/sql/udf.py index 84be2d24d9fbc..4bc4c591c9cf9 100644 --- a/python/pyspark/sql/udf.py +++ b/python/pyspark/sql/udf.py @@ -64,6 +64,12 @@ def _create_udf(f, returnType, evalType): "Invalid function: pandas_udfs with function type GROUPED_MAP " "must take either one argument (data) or two arguments (key, data).") + if evalType == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF \ + and len(argspec.args) not in (2, 3): + raise ValueError( + "Invalid function: pandas_udfs with function type COGROUPED_MAP " + "must take either two arguments (left, right) or three arguments (key, left, right).") + # Set the name of the UserDefinedFunction object to be the name of function f udf_obj = UserDefinedFunction( f, returnType=returnType, name=None, evalType=evalType, deterministic=True) @@ -135,6 +141,17 @@ def returnType(self): else: raise TypeError("Invalid returnType for grouped map Pandas " "UDFs: returnType must be a StructType.") + elif self.evalType == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: + if isinstance(self._returnType_placeholder, StructType): + try: + to_arrow_type(self._returnType_placeholder) + except TypeError: + raise NotImplementedError( + "Invalid returnType with cogrouped map Pandas UDFs: " + "%s is not supported" % str(self._returnType_placeholder)) + else: + raise TypeError("Invalid returnType for grouped map Pandas " + "UDFs: returnType must be a StructType.") elif self.evalType == PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF: try: # StructType is not yet allowed as a return type, explicitly check here to fail fast diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 023290f871959..b6ead89393abd 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -113,9 +113,18 @@ def verify_result_length(result, length): def wrap_cogrouped_map_pandas_udf(f, return_type, argspec): - def wrapped(key_Series, left_value_series, right_value_series): + def wrapped(left_key_series, left_value_series, right_key_series, right_value_series): import pandas as pd - result = f(pd.concat(left_value_series, axis=1), pd.concat(right_value_series, axis=1)) + + left_df = pd.concat(left_value_series, axis=1) + right_df = pd.concat(right_value_series, axis=1) + + if len(argspec.args) == 2: + result = f(left_df, right_df) + elif len(argspec.args) == 3: + key_series = left_key_series if not left_df.empty else right_key_series + key = tuple(s[0] for s in key_series) + result = f(key, left_df, right_df) if not isinstance(result, pd.DataFrame): raise TypeError("Return type of the user-defined function should be " "pandas.DataFrame, but is {}".format(type(result))) @@ -126,7 +135,7 @@ def wrapped(key_Series, left_value_series, right_value_series): "Expected: {} Actual: {}".format(len(return_type), len(result.columns))) return result - return lambda k, vl, vr: [(wrapped(k, vl, vr), to_arrow_type(return_type))] + return lambda kl, vl, kr, vr: [(wrapped(kl, vl, kr, vr), to_arrow_type(return_type))] def wrap_grouped_map_pandas_udf(f, return_type, argspec): @@ -377,9 +386,9 @@ def map_batch(batch): parsed_offsets = parse_grouped_arg_offsets(arg_offsets) arg0 = ["a[0][%d]" % o for o in parsed_offsets[0][0]] arg1 = ["a[0][%d]" % o for o in parsed_offsets[0][1]] - arg2 = ["a[1][%d]" % o for o in parsed_offsets[1][1]] - mapper_str = "lambda a: f([%s], [%s], [%s])" % (", ".join(arg0), ", ".join(arg1), ", ".join(arg2)) - print("mapper string is " + mapper_str) + arg2 = ["a[1][%d]" % o for o in parsed_offsets[1][0]] + arg3 = ["a[1][%d]" % o for o in parsed_offsets[1][1]] + mapper_str = "lambda a: f([%s], [%s], [%s], [%s])" % (", ".join(arg0), ", ".join(arg1), ", ".join(arg2), ", ".join(arg3)) else: # Create function like this: # lambda a: (f0(a[0]), f1(a[1], a[2]), f2(a[3])) @@ -401,28 +410,20 @@ def map_batch(batch): def parse_grouped_arg_offsets(arg_offsets): - print('arg offsets are ' + str(arg_offsets)) parsed = [] i = 0 while i < len(arg_offsets): offsets_len = arg_offsets[i] - print('i is ' + str(i)) - print('offsets_len is ' + str(offsets_len)) i += 1 offsets = arg_offsets[i: i + offsets_len] - print('offsets are ' + str(offsets)) split_index = offsets[0] + 1 - print('split index is ' + str(split_index)) keys = offsets[1: split_index] values = offsets[split_index:] - print('keys are ' + str(keys)) - print('values are ' + str(values)) parsed.append([keys, values]) i += offsets_len return parsed - def main(infile, outfile): try: boot_time = time.time() From d2da787f4be194eaa8b881207f9bacc54314a74d Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Tue, 2 Jul 2019 18:50:15 +0100 Subject: [PATCH 14/34] formatting --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 9dad538686b30..725662d5e72e0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -971,9 +971,11 @@ class Analyzer( case j @ Join(left, right, _, _, _) if !j.duplicateResolved => j.copy(right = dedupRight(left, right)) case f @ FlatMapCoGroupsInPandas(leftAttributes, rightAttributes, _, _, left, right) => - val leftAttributes2 = leftAttributes.map(x => resolveExpressionBottomUp(x, left).asInstanceOf[Attribute]) - val rightAttributes2 = rightAttributes.map(x => resolveExpressionBottomUp(x, right).asInstanceOf[Attribute]) - f.copy(leftAttributes=leftAttributes2, rightAttributes=rightAttributes2) + val leftRes = leftAttributes + .map(x => resolveExpressionBottomUp(x, left).asInstanceOf[Attribute]) + val rightRes = rightAttributes + .map(x => resolveExpressionBottomUp(x, right).asInstanceOf[Attribute]) + f.copy(leftAttributes = leftRes, rightAttributes = rightRes) case i @ Intersect(left, right, _) if !i.duplicateResolved => i.copy(right = dedupRight(left, right)) case e @ Except(left, right, _) if !e.duplicateResolved => From 73211410b4aac97b6ad6fcdb6c6184abb2773236 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Tue, 2 Jul 2019 21:29:53 +0100 Subject: [PATCH 15/34] fixed scalastyle errors --- .../sql/execution/python/BasePandasGroupExec.scala | 13 ++++++++----- .../python/InterleavedArrowPythonRunner.scala | 12 ++++++------ 2 files changed, 14 insertions(+), 11 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala index f6ceaf3561370..d0aec94aecef0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala @@ -14,8 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.spark.sql.execution.python +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + import org.apache.spark.TaskContext import org.apache.spark.api.python.{BasePythonRunner, ChainedPythonFunctions} import org.apache.spark.sql.catalyst.InternalRow @@ -25,8 +29,6 @@ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.ArrowUtils import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch} -import scala.collection.mutable.ArrayBuffer -import scala.collection.JavaConverters._ abstract class BasePandasGroupExec(func: Expression, output: Seq[Attribute]) extends SparkPlan { @@ -42,8 +44,8 @@ abstract class BasePandasGroupExec(func: Expression, override def producedAttributes: AttributeSet = AttributeSet(output) - protected def executePython[T](data: Iterator[T], - runner: BasePythonRunner[T, ColumnarBatch]): Iterator[InternalRow] = { + protected def executePython[T] + (data: Iterator[T], runner: BasePythonRunner[T, ColumnarBatch]): Iterator[InternalRow] = { val context = TaskContext.get() val columnarBatchIter = runner.compute(data, context.partitionId(), context) @@ -62,7 +64,8 @@ abstract class BasePandasGroupExec(func: Expression, protected def groupAndDedup( input: Iterator[InternalRow], groupingAttributes: Seq[Attribute], - inputSchema: Seq[Attribute], dedupSchema: Seq[Attribute]): Iterator[(InternalRow, Iterator[InternalRow])] = { + inputSchema: Seq[Attribute], dedupSchema: Seq[Attribute] + ): Iterator[(InternalRow, Iterator[InternalRow])] = { val groupedIter = GroupedIterator(input, groupingAttributes, inputSchema) val dedupProj = UnsafeProjection.create(dedupSchema, inputSchema) groupedIter.map { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala index 2a32a2a8bc64d..198afdab6f3fc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala @@ -43,12 +43,12 @@ class InterleavedArrowPythonRunner( extends BaseArrowPythonRunner[(Iterator[InternalRow], Iterator[InternalRow])]( funcs, evalType, argOffsets) { - protected override def newWriterThread( - env: SparkEnv, - worker: Socket, - inputIterator: Iterator[(Iterator[InternalRow], Iterator[InternalRow])], - partitionIndex: Int, - context: TaskContext): WriterThread = { + protected def newWriterThread( + env: SparkEnv, + worker: Socket, + inputIterator: Iterator[(Iterator[InternalRow], Iterator[InternalRow])], + partitionIndex: Int, + context: TaskContext): WriterThread = { new WriterThread(env, worker, inputIterator, partitionIndex, context) { protected override def writeCommand(dataOut: DataOutputStream): Unit = { From 6bbe31c004b755eb0b86730c5b6002c14c351ec0 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Tue, 2 Jul 2019 22:06:49 +0100 Subject: [PATCH 16/34] updated grouped map to new args format --- python/pyspark/worker.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index b6ead89393abd..284b092a16e66 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -372,9 +372,9 @@ def map_batch(batch): arg_offsets, udf = read_single_udf( pickleSer, infile, eval_type, runner_conf, udf_index=0) udfs['f'] = udf - split_offset = arg_offsets[0] + 1 - arg0 = ["a[%d]" % o for o in arg_offsets[1: split_offset]] - arg1 = ["a[%d]" % o for o in arg_offsets[split_offset:]] + parsed_offsets = parse_grouped_arg_offsets(arg_offsets) + arg0 = ["a[%d]" % o for o in parsed_offsets[0][0]] + arg1 = ["a[%d]" % o for o in parsed_offsets[0][1]] mapper_str = "lambda a: f([%s], [%s])" % (", ".join(arg0), ", ".join(arg1)) elif eval_type == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: # We assume there is only one UDF here because cogrouped map doesn't From 94be57490824e1af4c66a63a11d3fdf10ad13c9f Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Thu, 11 Jul 2019 14:29:47 +0100 Subject: [PATCH 17/34] some code review fixes --- python/pyspark/serializers.py | 4 +- python/pyspark/sql/cogroup.py | 57 ++++++++++++++++++- python/pyspark/sql/group.py | 2 +- python/pyspark/sql/udf.py | 2 +- python/pyspark/worker.py | 4 +- .../python/InterleavedArrowPythonRunner.scala | 10 ++-- 6 files changed, 67 insertions(+), 12 deletions(-) diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 3c605a7a22262..579501f6f39e0 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -427,10 +427,10 @@ def __repr__(self): return "ArrowStreamPandasUDFSerializer" -class InterleavedArrowStreamPandasSerializer(ArrowStreamPandasUDFSerializer): +class PandasCogroupSerializer(ArrowStreamPandasUDFSerializer): def __init__(self, timezone, safecheck, assign_cols_by_name): - super(InterleavedArrowStreamPandasSerializer, self).__init__(timezone, safecheck, assign_cols_by_name) + super(PandasCogroupSerializer, self).__init__(timezone, safecheck, assign_cols_by_name) def load_stream(self, stream): """ diff --git a/python/pyspark/sql/cogroup.py b/python/pyspark/sql/cogroup.py index 09cd7b0c6fbfa..1a52a35e4e29d 100644 --- a/python/pyspark/sql/cogroup.py +++ b/python/pyspark/sql/cogroup.py @@ -15,19 +15,74 @@ # limitations under the License. # -from pyspark.sql.dataframe import DataFrame +from pyspark import since from pyspark.rdd import PythonEvalType from pyspark.sql.column import Column +from pyspark.sql.dataframe import DataFrame class CoGroupedData(object): + """ + A logical grouping of two :class:`GroupedData`, + created by :func:`GroupedData.cogroup`. + + .. note:: Experimental + + .. versionadded:: 3.0 + + """ def __init__(self, gd1, gd2): self._gd1 = gd1 self._gd2 = gd2 self.sql_ctx = gd1.sql_ctx + @since(3.0) def apply(self, udf): + """ + Applies a function to each cogroup using a pandas udf and returns the result + as a `DataFrame`. + + The user-defined function should take two `pandas.DataFrame` and return another + `pandas.DataFrame`. For each side of the cogroup, all columns are passed together + as a `pandas.DataFrame` to the user-function and the returned `pandas.DataFrame` + are combined as a :class:`DataFrame`. + + The returned `pandas.DataFrame` can be of arbitrary length and its schema must match the + returnType of the pandas udf. + + .. note:: This function requires a full shuffle. All the data of a cogroup will be loaded + into memory, so the user should be aware of the potential OOM risk if data is skewed + and certain goroups are too large to fit in memory. + + .. note:: Experimental + + :param udf: a cogrouped map user-defined function returned by + :func:`pyspark.sql.functions.pandas_udf`. + + >>> from pyspark.sql.functions import pandas_udf, PandasUDFType + >>> df1 = spark.createDataFrame( + ... [(20000101, 1, 1.0), (20000101, 2, 2.0), (20000102, 1, 3.0), (20000102, 2, 4.0)], + ... ("time", "id", "v1")) + >>> df2 = spark.createDataFrame( + ... [(20000101, 1, "x"), (20000101, 2, "y")], + ... ("time", "id", "v2")) + >>> @pandas_udf("time int, id int, v1 double, v2 string", PandasUDFType.COGROUPED_MAP) + ... def asof_join(l, r): + ... return pd.merge_asof(l, r, on="time", by="id") + >>> df1.groupby("id").cogroup(df2.groupby("id")).apply(asof_join).show() + +--------+---+---+---+ + | time| id| v1| v2| + +--------+---+---+---+ + |20000101| 1|1.0| x| + |20000102| 1|3.0| x| + |20000101| 2|2.0| y| + |20000102| 2|4.0| y| + +--------+---+---+---+ + + .. seealso:: :meth:`pyspark.sql.functions.pandas_udf` + + """ # Columns are special because hasattr always return True if isinstance(udf, Column) or not hasattr(udf, 'func') \ or udf.evalType != PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: diff --git a/python/pyspark/sql/group.py b/python/pyspark/sql/group.py index 04f42b1598376..292155d674300 100644 --- a/python/pyspark/sql/group.py +++ b/python/pyspark/sql/group.py @@ -238,7 +238,7 @@ def apply(self, udf): The returned `pandas.DataFrame` can be of arbitrary length and its schema must match the returnType of the pandas udf. - .. note:: This function requires a full shuffle. all the data of a group will be loaded + .. note:: This function requires a full shuffle. All the data of a group will be loaded into memory, so the user should be aware of the potential OOM risk if data is skewed and certain groups are too large to fit in memory. diff --git a/python/pyspark/sql/udf.py b/python/pyspark/sql/udf.py index 4bc4c591c9cf9..6ec269ee77538 100644 --- a/python/pyspark/sql/udf.py +++ b/python/pyspark/sql/udf.py @@ -150,7 +150,7 @@ def returnType(self): "Invalid returnType with cogrouped map Pandas UDFs: " "%s is not supported" % str(self._returnType_placeholder)) else: - raise TypeError("Invalid returnType for grouped map Pandas " + raise TypeError("Invalid returnType for cogrouped map Pandas " "UDFs: returnType must be a StructType.") elif self.evalType == PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF: try: diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index adaf936d5c3be..20fe1fb30476d 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -38,7 +38,7 @@ from pyspark.rdd import PythonEvalType from pyspark.serializers import write_with_length, write_int, read_long, read_bool, \ write_long, read_int, SpecialLengths, UTF8Deserializer, PickleSerializer, \ - BatchedSerializer, ArrowStreamPandasUDFSerializer, InterleavedArrowStreamPandasSerializer + BatchedSerializer, ArrowStreamPandasUDFSerializer, PandasCogroupSerializer from pyspark.sql.types import to_arrow_type, StructType from pyspark.util import _get_argspec, fail_on_stopiteration from pyspark import shuffle @@ -313,7 +313,7 @@ def read_udfs(pickleSer, infile, eval_type): # Scalar Pandas UDF handles struct type arguments as pandas DataFrames instead of # pandas Series. See SPARK-27240. if eval_type == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: - ser = InterleavedArrowStreamPandasSerializer(timezone, safecheck, assign_cols_by_name) + ser = PandasCogroupSerializer(timezone, safecheck, assign_cols_by_name) else: df_for_struct = (eval_type == PythonEvalType.SQL_SCALAR_PANDAS_UDF or eval_type == PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF or diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala index 198afdab6f3fc..daeb23c84b571 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala @@ -67,17 +67,17 @@ class InterleavedArrowPythonRunner( while (inputIterator.hasNext) { dataOut.writeInt(SpecialLengths.START_ARROW_STREAM) val (nextLeft, nextRight) = inputIterator.next() - writeGroup(nextLeft, leftSchema, dataOut) - writeGroup(nextRight, rightSchema, dataOut) + writeGroup(nextLeft, leftSchema, dataOut, "left") + writeGroup(nextRight, rightSchema, dataOut, "right") } dataOut.writeInt(SpecialLengths.END_OF_DATA_SECTION) } - def writeGroup(group: Iterator[InternalRow], schema: StructType, dataOut: DataOutputStream - ) = { + def writeGroup(group: Iterator[InternalRow], schema: StructType, dataOut: DataOutputStream, + name: String) = { val arrowSchema = ArrowUtils.toArrowSchema(schema, timeZoneId) val allocator = ArrowUtils.rootAllocator.newChildAllocator( - s"stdout writer for $pythonExec", 0, Long.MaxValue) + s"stdout writer for $pythonExec ($name)", 0, Long.MaxValue) val root = VectorSchemaRoot.create(arrowSchema, allocator) Utils.tryWithSafeFinally { From 3de551fd316c49b98a4be9e2b4e715e916059d6d Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Thu, 11 Jul 2019 21:40:44 +0100 Subject: [PATCH 18/34] more code review fixes --- python/pyspark/serializers.py | 8 +-- .../tests/test_pandas_udf_cogrouped_map.py | 51 +++++++--------- python/pyspark/worker.py | 59 +++++++++++-------- 3 files changed, 59 insertions(+), 59 deletions(-) diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index c96e3cf586468..42101249695a5 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -365,13 +365,13 @@ def __iter__(self): return self def __next__(self): - stream_status = read_int(self._stream) - if stream_status == SpecialLengths.START_ARROW_STREAM: + dataframes_in_group = read_int(self._stream) + if dataframes_in_group == 2: return self._read_df(), self._read_df() - elif stream_status == SpecialLengths.END_OF_DATA_SECTION: + elif dataframes_in_group == 0: raise StopIteration else: - raise ValueError('Received invalid stream status {0}'.format(stream_status)) + raise ValueError('Received Invalid number of ddataframes in group {0}'.format(dataframes_in_group)) def next(self): return self.__next__() diff --git a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py index 54397e04c40cc..f087add0a36e8 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py +++ b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py @@ -123,18 +123,18 @@ def merge_pandas(l, r): assert_frame_equal(expected, result, check_column_type=_check_column_type) def test_with_key_left(self): - self._test_with_key_left(self.data1, self.data2) + self._test_with_key(self.data1, self.data1, isLeft=True) def test_with_key_right(self): - self._test_with_key_right(self.data1, self.data2) + self._test_with_key(self.data1, self.data1, isLeft=False) def test_with_key_left_group_empty(self): left = self.data1.where(col("id") % 2 == 0) - self._test_with_key_right(left, self.data2) + self._test_with_key(left, self.data1, isLeft=True) def test_with_key_right_group_empty(self): - right = self.data2.where(col("id") % 2 == 0) - self._test_with_key_left(self.data1, right) + right = self.data1.where(col("id") % 2 == 0) + self._test_with_key(self.data1, right, isLeft=False) def test_with_key_complex(self): @@ -165,51 +165,42 @@ def test_wrong_return_type(self): PandasUDFType.COGROUPED_MAP) def test_wrong_args(self): + # Test that we get a sensible exception invalid values passed to apply left = self.data1 right = self.data2 - with QuietTest(self.sc): + # Function rather than a udf with self.assertRaisesRegexp(ValueError, 'Invalid udf'): left.groupby('id').cogroup(right.groupby('id')).apply(lambda l, r: l) + + # Udf missing return type and PandasUdfType with self.assertRaisesRegexp(ValueError, 'Invalid udf'): left.groupby('id').cogroup(right.groupby('id')).apply(udf(lambda l, r: l, DoubleType())) - with self.assertRaisesRegexp(ValueError, 'Invalid udf'): - left.groupby('id').cogroup(right.groupby('id')).apply(sum(left.v)) + + # Pass in expression rather than udf with self.assertRaisesRegexp(ValueError, 'Invalid udf'): left.groupby('id').cogroup(right.groupby('id')).apply(left.v + 1) + + # Zero arg function with self.assertRaisesRegexp(ValueError, 'Invalid function'): left.groupby('id').cogroup(right.groupby('id')).apply( pandas_udf(lambda: 1, StructType([StructField("d", DoubleType())]))) + + # Udf without PandasUDFType with self.assertRaisesRegexp(ValueError, 'Invalid udf'): left.groupby('id').cogroup(right.groupby('id')).apply(pandas_udf(lambda x, y: x, DoubleType())) + + # Udf with incorrect PandasUDFType with self.assertRaisesRegexp(ValueError, 'Invalid udf.*COGROUPED_MAP'): left.groupby('id').cogroup(right.groupby('id')).apply( pandas_udf(lambda x, y: x, DoubleType(), PandasUDFType.SCALAR)) @staticmethod - def _test_with_key_left(left, right): + def _test_with_key(left, right, isLeft): @pandas_udf('id long, k int, v int, key long', PandasUDFType.COGROUPED_MAP) - def left_assign_key(key, l, _): - return l.assign(key=key[0]) - - result = left \ - .groupby('id') \ - .cogroup(right.groupby('id')) \ - .apply(left_assign_key) \ - .toPandas() - - expected = left.toPandas() - expected = expected.assign(key=expected.id) - - assert_frame_equal(expected, result, check_column_type=_check_column_type) - - @staticmethod - def _test_with_key_right(left, right): - - @pandas_udf('id long, k int, v2 int, key long', PandasUDFType.COGROUPED_MAP) - def right_assign_key(key, _, r): - return r.assign(key=key[0]) + def right_assign_key(key, l, r): + return l.assign(key=key[0]) if isLeft else r.assign(key=key[0]) result = left \ .groupby('id') \ @@ -217,7 +208,7 @@ def right_assign_key(key, _, r): .apply(right_assign_key) \ .toPandas() - expected = right.toPandas() + expected = left.toPandas() if isLeft else right.toPandas() expected = expected.assign(key=expected.id) assert_frame_equal(expected, result, check_column_type=_check_column_type) diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 16b31d471d428..29282b4a6602b 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -377,6 +377,29 @@ def map_batch(batch): # profiling is not supported for UDF return func, None, ser, ser + # Helper function to extract the key and value indexs from arg_offsets + # arg_offsets is a List containing the key and value + # indexes of columns of the DataFrames to be passed to the udf. + # It consists of n repeating groups where n is the number of + # DataFrames. Each group has the following format. + # group[0]: length of group + # group[1]: length of key indexes + # group[2.. group[1] +2]: key attributes + # group[group[1] +3 group[0]]: value attributes + def extract_key_value_indexes(): + parsed = [] + i = 0 + while i < len(arg_offsets): + offsets_len = arg_offsets[i] + i += 1 + offsets = arg_offsets[i: i + offsets_len] + split_index = offsets[0] + 1 + keys = offsets[1: split_index] + values = offsets[split_index:] + parsed.append([keys, values]) + i += offsets_len + return parsed + udfs = {} call_udf = [] mapper_str = "" @@ -393,10 +416,10 @@ def map_batch(batch): arg_offsets, udf = read_single_udf( pickleSer, infile, eval_type, runner_conf, udf_index=0) udfs['f'] = udf - parsed_offsets = parse_grouped_arg_offsets(arg_offsets) - arg0 = ["a[%d]" % o for o in parsed_offsets[0][0]] - arg1 = ["a[%d]" % o for o in parsed_offsets[0][1]] - mapper_str = "lambda a: f([%s], [%s])" % (", ".join(arg0), ", ".join(arg1)) + parsed_offsets = extract_key_value_indexes() + keys = ["a[%d]" % o for o in parsed_offsets[0][0]] + vals = ["a[%d]" % o for o in parsed_offsets[0][1]] + mapper_str = "lambda a: f([%s], [%s])" % (", ".join(keys), ", ".join(vals)) elif eval_type == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: # We assume there is only one UDF here because cogrouped map doesn't # support combining multiple UDFs. @@ -404,12 +427,13 @@ def map_batch(batch): arg_offsets, udf = read_single_udf( pickleSer, infile, eval_type, runner_conf, udf_index=0) udfs['f'] = udf - parsed_offsets = parse_grouped_arg_offsets(arg_offsets) - arg0 = ["a[0][%d]" % o for o in parsed_offsets[0][0]] - arg1 = ["a[0][%d]" % o for o in parsed_offsets[0][1]] - arg2 = ["a[1][%d]" % o for o in parsed_offsets[1][0]] - arg3 = ["a[1][%d]" % o for o in parsed_offsets[1][1]] - mapper_str = "lambda a: f([%s], [%s], [%s], [%s])" % (", ".join(arg0), ", ".join(arg1), ", ".join(arg2), ", ".join(arg3)) + parsed_offsets = extract_key_value_indexes() + df1_keys = ["a[0][%d]" % o for o in parsed_offsets[0][0]] + df1_vals = ["a[0][%d]" % o for o in parsed_offsets[0][1]] + df2_keys = ["a[1][%d]" % o for o in parsed_offsets[1][0]] + df2_vals = ["a[1][%d]" % o for o in parsed_offsets[1][1]] + mapper_str = "lambda a: f([%s], [%s], [%s], [%s])" \ + % (", ".join(df1_keys), ", ".join(df1_vals), ", ".join(df2_keys), ", ".join(df2_vals)) else: # Create function like this: # lambda a: (f0(a[0]), f1(a[1], a[2]), f2(a[3])) @@ -430,21 +454,6 @@ def map_batch(batch): return func, None, ser, ser -def parse_grouped_arg_offsets(arg_offsets): - parsed = [] - i = 0 - while i < len(arg_offsets): - offsets_len = arg_offsets[i] - i += 1 - offsets = arg_offsets[i: i + offsets_len] - split_index = offsets[0] + 1 - keys = offsets[1: split_index] - values = offsets[split_index:] - parsed.append([keys, values]) - i += offsets_len - return parsed - - def main(infile, outfile): try: boot_time = time.time() From 300b53a9ef0757a0547c371da3207be114f12d75 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Thu, 11 Jul 2019 21:43:57 +0100 Subject: [PATCH 19/34] more code review fixes --- .../sql/catalyst/analysis/Analyzer.scala | 1 - .../python/BasePandasGroupExec.scala | 24 ++++++++------ .../python/InterleavedArrowPythonRunner.scala | 32 +++++++++++-------- 3 files changed, 32 insertions(+), 25 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index e52d66efd2474..2356bed3b5c50 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2284,7 +2284,6 @@ class Analyzer( } } - /** * Removes natural or using joins by calculating output columns based on output from two sides, * Then apply a Project on a normal Join to eliminate natural or using join. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala index d0aec94aecef0..404869231c348 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala @@ -44,8 +44,9 @@ abstract class BasePandasGroupExec(func: Expression, override def producedAttributes: AttributeSet = AttributeSet(output) - protected def executePython[T] - (data: Iterator[T], runner: BasePythonRunner[T, ColumnarBatch]): Iterator[InternalRow] = { + protected def executePython[T]( + data: Iterator[T], + runner: BasePythonRunner[T, ColumnarBatch]): Iterator[InternalRow] = { val context = TaskContext.get() val columnarBatchIter = runner.compute(data, context.partitionId(), context) @@ -63,9 +64,10 @@ abstract class BasePandasGroupExec(func: Expression, } protected def groupAndDedup( - input: Iterator[InternalRow], groupingAttributes: Seq[Attribute], - inputSchema: Seq[Attribute], dedupSchema: Seq[Attribute] - ): Iterator[(InternalRow, Iterator[InternalRow])] = { + input: Iterator[InternalRow], + groupingAttributes: Seq[Attribute], + inputSchema: Seq[Attribute], + dedupSchema: Seq[Attribute]): Iterator[(InternalRow, Iterator[InternalRow])] = { val groupedIter = GroupedIterator(input, groupingAttributes, inputSchema) val dedupProj = UnsafeProjection.create(dedupSchema, inputSchema) groupedIter.map { @@ -73,8 +75,9 @@ abstract class BasePandasGroupExec(func: Expression, } } - protected def createSchema(child: SparkPlan, groupingAttributes: Seq[Attribute]) - : (StructType, Seq[Attribute], Array[Int]) = { + protected def createSchema( + child: SparkPlan, + groupingAttributes: Seq[Attribute]): (StructType, Seq[Attribute], Array[Int]) = { // Deduplicate the grouping attributes. // If a grouping attribute also appears in data attributes, then we don't need to send the @@ -83,9 +86,10 @@ abstract class BasePandasGroupExec(func: Expression, // // We use argOffsets to distinguish grouping attributes and data attributes as following: // - // argOffsets[0] is the length of grouping attributes - // argOffsets[1 .. argOffsets[0]+1] is the arg offsets for grouping attributes - // argOffsets[argOffsets[0]+1 .. ] is the arg offsets for data attributes + // argOffsets[0] is the length of the argOffsets array + // argOffsets[1] is the length of grouping attributes + // argOffsets[2 .. argOffsets[0]+2] is the arg offsets for grouping attributes + // argOffsets[argOffsets[0]+2 .. ] is the arg offsets for data attributes val dataAttributes = child.output.drop(groupingAttributes.length) val groupingIndicesInData = groupingAttributes.map { attribute => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala index daeb23c84b571..578e19aa84b41 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala @@ -32,23 +32,25 @@ import org.apache.spark.sql.util.ArrowUtils import org.apache.spark.util.Utils + class InterleavedArrowPythonRunner( - funcs: Seq[ChainedPythonFunctions], - evalType: Int, - argOffsets: Array[Array[Int]], - leftSchema: StructType, - rightSchema: StructType, - timeZoneId: String, - conf: Map[String, String]) + funcs: Seq[ChainedPythonFunctions], + evalType: Int, + argOffsets: Array[Array[Int]], + leftSchema: StructType, + rightSchema: StructType, + timeZoneId: String, + conf: Map[String, String]) extends BaseArrowPythonRunner[(Iterator[InternalRow], Iterator[InternalRow])]( funcs, evalType, argOffsets) { protected def newWriterThread( - env: SparkEnv, - worker: Socket, - inputIterator: Iterator[(Iterator[InternalRow], Iterator[InternalRow])], - partitionIndex: Int, - context: TaskContext): WriterThread = { + env: SparkEnv, + worker: Socket, + inputIterator: Iterator[(Iterator[InternalRow], Iterator[InternalRow])], + partitionIndex: Int, + context: TaskContext): WriterThread = { + new WriterThread(env, worker, inputIterator, partitionIndex, context) { protected override def writeCommand(dataOut: DataOutputStream): Unit = { @@ -64,13 +66,15 @@ class InterleavedArrowPythonRunner( } protected override def writeIteratorToStream(dataOut: DataOutputStream): Unit = { + // For each we first send the number of dataframes in each group then send + // first df, then send second df. End of data is marked by sending 0. while (inputIterator.hasNext) { - dataOut.writeInt(SpecialLengths.START_ARROW_STREAM) + dataOut.writeInt(2) val (nextLeft, nextRight) = inputIterator.next() writeGroup(nextLeft, leftSchema, dataOut, "left") writeGroup(nextRight, rightSchema, dataOut, "right") } - dataOut.writeInt(SpecialLengths.END_OF_DATA_SECTION) + dataOut.writeInt(0) } def writeGroup(group: Iterator[InternalRow], schema: StructType, dataOut: DataOutputStream, From 7d161ba0f39bfc0b54e830ce188d595e0e89ed08 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Thu, 11 Jul 2019 21:47:02 +0100 Subject: [PATCH 20/34] fix comment on PandasCogroupSerializer --- python/pyspark/serializers.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 42101249695a5..01aff3f09ffbf 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -434,7 +434,8 @@ def __init__(self, timezone, safecheck, assign_cols_by_name): def load_stream(self, stream): """ - Deserialize ArrowRecordBatches to an Arrow table and return as a list of pandas.Series. + Deserialize Cogroupes ArrowRecordBatches to a tuple of Arrow tables and return as a two + lists of pandas.Series. """ reader = InterleavedArrowReader(stream) for batch1, batch2 in reader: From d1a6366c457ea8c7e4437186bfb8d480837d4e36 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Thu, 11 Jul 2019 22:19:28 +0100 Subject: [PATCH 21/34] formatting --- .../python/BaseArrowPythonRunner.scala | 23 +++++++++---------- 1 file changed, 11 insertions(+), 12 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BaseArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BaseArrowPythonRunner.scala index 3cba06dcf7d52..1d66f8adb874e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BaseArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BaseArrowPythonRunner.scala @@ -33,21 +33,20 @@ import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch, Column abstract class BaseArrowPythonRunner[T]( - funcs: Seq[ChainedPythonFunctions], - evalType: Int, - argOffsets: Array[Array[Int]]) - extends BasePythonRunner[T, ColumnarBatch]( - funcs, evalType, argOffsets) { + funcs: Seq[ChainedPythonFunctions], + evalType: Int, + argOffsets: Array[Array[Int]]) + extends BasePythonRunner[T, ColumnarBatch](funcs, evalType, argOffsets) { protected override def newReaderIterator( - stream: DataInputStream, - writerThread: WriterThread, - startTime: Long, - env: SparkEnv, - worker: Socket, - releasedOrClosed: AtomicBoolean, - context: TaskContext): Iterator[ColumnarBatch] = { + stream: DataInputStream, + writerThread: WriterThread, + startTime: Long, + env: SparkEnv, + worker: Socket, + releasedOrClosed: AtomicBoolean, + context: TaskContext): Iterator[ColumnarBatch] = { new ReaderIterator(stream, writerThread, startTime, env, worker, releasedOrClosed, context) { From 3e4bc955c3f0b044b92e8d2459266736da147932 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Fri, 19 Jul 2019 15:03:08 +0100 Subject: [PATCH 22/34] python style fixes --- python/pyspark/serializers.py | 3 ++- python/pyspark/sql/cogroup.py | 1 - .../tests/test_pandas_udf_cogrouped_map.py | 20 ++++++++++--------- python/pyspark/sql/udf.py | 3 ++- python/pyspark/worker.py | 4 ++-- 5 files changed, 17 insertions(+), 14 deletions(-) diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 01aff3f09ffbf..8f15659faa1b5 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -371,7 +371,8 @@ def __next__(self): elif dataframes_in_group == 0: raise StopIteration else: - raise ValueError('Received Invalid number of ddataframes in group {0}'.format(dataframes_in_group)) + raise ValueError( + 'Received Invalid number of dataframes in group {0}'.format(dataframes_in_group)) def next(self): return self.__next__() diff --git a/python/pyspark/sql/cogroup.py b/python/pyspark/sql/cogroup.py index 1a52a35e4e29d..b1ddd5b614ecf 100644 --- a/python/pyspark/sql/cogroup.py +++ b/python/pyspark/sql/cogroup.py @@ -97,4 +97,3 @@ def apply(self, udf): def _extract_cols(gd): df = gd._df return [df[col] for col in df.columns] - diff --git a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py index f087add0a36e8..11fd78944e733 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py +++ b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py @@ -81,7 +81,7 @@ def test_right_group_empty(self): def test_different_schemas(self): right = self.data2.withColumn('v3', lit('a')) - self._test_merge(self.data1, right, output_schema='id long, k int, v int, v2 int, v3 string') + self._test_merge(self.data1, right, 'id long, k int, v int, v2 int, v3 string') def test_complex_group_by(self): left = pd.DataFrame.from_dict({ @@ -173,9 +173,10 @@ def test_wrong_args(self): with self.assertRaisesRegexp(ValueError, 'Invalid udf'): left.groupby('id').cogroup(right.groupby('id')).apply(lambda l, r: l) - # Udf missing return type and PandasUdfType + # Udf missing return type with self.assertRaisesRegexp(ValueError, 'Invalid udf'): - left.groupby('id').cogroup(right.groupby('id')).apply(udf(lambda l, r: l, DoubleType())) + left.groupby('id').cogroup(right.groupby('id'))\ + .apply(udf(lambda l, r: l, DoubleType())) # Pass in expression rather than udf with self.assertRaisesRegexp(ValueError, 'Invalid udf'): @@ -183,17 +184,18 @@ def test_wrong_args(self): # Zero arg function with self.assertRaisesRegexp(ValueError, 'Invalid function'): - left.groupby('id').cogroup(right.groupby('id')).apply( - pandas_udf(lambda: 1, StructType([StructField("d", DoubleType())]))) + left.groupby('id').cogroup(right.groupby('id'))\ + .apply(pandas_udf(lambda: 1, StructType([StructField("d", DoubleType())]))) # Udf without PandasUDFType with self.assertRaisesRegexp(ValueError, 'Invalid udf'): - left.groupby('id').cogroup(right.groupby('id')).apply(pandas_udf(lambda x, y: x, DoubleType())) + left.groupby('id').cogroup(right.groupby('id'))\ + .apply(pandas_udf(lambda x, y: x, DoubleType())) # Udf with incorrect PandasUDFType with self.assertRaisesRegexp(ValueError, 'Invalid udf.*COGROUPED_MAP'): - left.groupby('id').cogroup(right.groupby('id')).apply( - pandas_udf(lambda x, y: x, DoubleType(), PandasUDFType.SCALAR)) + left.groupby('id').cogroup(right.groupby('id'))\ + .apply(pandas_udf(lambda x, y: x, DoubleType(), PandasUDFType.SCALAR)) @staticmethod def _test_with_key(left, right, isLeft): @@ -245,4 +247,4 @@ def merge_pandas(l, r): testRunner = xmlrunner.XMLTestRunner(output='target/test-reports', verbosity=2) except ImportError: testRunner = None - unittest.main(testRunner=testRunner, verbosity=2) \ No newline at end of file + unittest.main(testRunner=testRunner, verbosity=2) diff --git a/python/pyspark/sql/udf.py b/python/pyspark/sql/udf.py index feeb8714e5f9a..c4d7c1ed205f1 100644 --- a/python/pyspark/sql/udf.py +++ b/python/pyspark/sql/udf.py @@ -70,7 +70,8 @@ def _create_udf(f, returnType, evalType): and len(argspec.args) not in (2, 3): raise ValueError( "Invalid function: pandas_udfs with function type COGROUPED_MAP " - "must take either two arguments (left, right) or three arguments (key, left, right).") + "must take either two arguments (left, right) " + "or three arguments (key, left, right).") # Set the name of the UserDefinedFunction object to be the name of function f udf_obj = UserDefinedFunction( diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 29282b4a6602b..56e01fb9fa166 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -432,8 +432,8 @@ def extract_key_value_indexes(): df1_vals = ["a[0][%d]" % o for o in parsed_offsets[0][1]] df2_keys = ["a[1][%d]" % o for o in parsed_offsets[1][0]] df2_vals = ["a[1][%d]" % o for o in parsed_offsets[1][1]] - mapper_str = "lambda a: f([%s], [%s], [%s], [%s])" \ - % (", ".join(df1_keys), ", ".join(df1_vals), ", ".join(df2_keys), ", ".join(df2_vals)) + mapper_str = "lambda a: f([%s], [%s], [%s], [%s])" % ( + ", ".join(df1_keys), ", ".join(df1_vals), ", ".join(df2_keys), ", ".join(df2_vals)) else: # Create function like this: # lambda a: (f0(a[0]), f1(a[1], a[2]), f2(a[3])) From 307e6643ea825b2cd5ca404f72bc5ea1b6ea1a79 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Fri, 19 Jul 2019 17:31:16 +0100 Subject: [PATCH 23/34] added doc --- .../logical/pythonLogicalOperators.scala | 10 ++++++--- .../spark/sql/RelationalGroupedDataset.scala | 10 +++++++++ .../python/BaseArrowPythonRunner.scala | 4 +++- ...scala => CogroupedArrowPythonRunner.scala} | 8 +++++-- .../python/FlatMapCoGroupsInPandasExec.scala | 22 ++++++++++++++++++- 5 files changed, 47 insertions(+), 7 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/python/{InterleavedArrowPythonRunner.scala => CogroupedArrowPythonRunner.scala} (94%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala index f52a0730045eb..fee010c3c1111 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression, PythonUDF} /** - * FlatMap groups using an udf: pandas.Dataframe -> pandas.DataFrame. + * FlatMap groups using a udf: pandas.Dataframe -> pandas.DataFrame. * This is used by DataFrame.groupby().apply(). */ case class FlatMapGroupsInPandas( @@ -40,7 +40,7 @@ case class FlatMapGroupsInPandas( } /** - * Map partitions using an udf: iter(pandas.Dataframe) -> iter(pandas.DataFrame). + * Map partitions using a udf: iter(pandas.Dataframe) -> iter(pandas.DataFrame). * This is used by DataFrame.mapInPandas() */ case class MapInPandas( @@ -51,7 +51,10 @@ case class MapInPandas( override val producedAttributes = AttributeSet(output) } - +/** + * Flatmap cogroups using a udf: pandas.Dataframe, pandas.Dataframe -> pandas.Dataframe + * This is used by DataFrame.groupby().cogroup().apply(). + */ case class FlatMapCoGroupsInPandas( leftAttributes: Seq[Attribute], rightAttributes: Seq[Attribute], @@ -59,6 +62,7 @@ case class FlatMapCoGroupsInPandas( output: Seq[Attribute], left: LogicalPlan, right: LogicalPlan) extends BinaryNode { + override val producedAttributes = AttributeSet(output) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index 0018f6379e8fe..8f5db8a794c40 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -523,6 +523,16 @@ class RelationalGroupedDataset protected[sql]( Dataset.ofRows(df.sparkSession, plan) } + /** + * Applies a vectorized python user-defined function to each cogrouped data. + * The user-defined function defines a transformation: + * `pandas.DataFrame`, `pandas.DataFrame` -> `pandas.DataFrame`. + * For each group int he cogrouped data, all elements in the group are passed as a + * `pandas.DataFrame` and the results for all cogroups are combined into a new [[DataFrame]]. + * + * This function uses Apache Arrow as serialization format between Java executors and Python + * workers. + */ private[sql] def flatMapCoGroupsInPandas (r: RelationalGroupedDataset, expr: PythonUDF): DataFrame = { require(expr.evalType == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BaseArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BaseArrowPythonRunner.scala index 1d66f8adb874e..bc5d38e899cb6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BaseArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BaseArrowPythonRunner.scala @@ -31,7 +31,9 @@ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.ArrowUtils import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch, ColumnVector} - +/** + * Common functionality for a udf runner that exchanges data with Python worker via Arrow stream. + */ abstract class BaseArrowPythonRunner[T]( funcs: Seq[ChainedPythonFunctions], evalType: Int, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CogroupedArrowPythonRunner.scala similarity index 94% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/python/CogroupedArrowPythonRunner.scala index 578e19aa84b41..5c8751175b57d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/InterleavedArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CogroupedArrowPythonRunner.scala @@ -32,8 +32,12 @@ import org.apache.spark.sql.util.ArrowUtils import org.apache.spark.util.Utils - -class InterleavedArrowPythonRunner( +/** + * Python UDF Runner for cogrouped udfs. Although the data is exchanged with the python + * worker via arrow, we cannot use `ArrowPythonRunner` as we need to send more than one + * dataframe. + */ +class CogroupedArrowPythonRunner( funcs: Seq[ChainedPythonFunctions], evalType: Int, argOffsets: Array[Array[Int]], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala index d47111216a14b..dfd452710676d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala @@ -24,6 +24,26 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Distribution, Partitioning} import org.apache.spark.sql.execution.{BinaryExecNode, CoGroupedIterator, SparkPlan} + +/** + * Physical node for [[org.apache.spark.sql.catalyst.plans.logical.FlatMapCoGroupsInPandas]] + * + * The input dataframes are first Cogrouped. Rows from each side of the cogroup are passed to the + * Python worker via Arrow. As each side of the cogroup may have a different schema we send every + * group in it's own Arrow stream. + * The Python worker turns the resulting record batches to `pandas.DataFrame`s, invokes the + * user-defined function, and passes the resulting `pandas.DataFrame` + * as an Arrow record batch. Finally, each record batch is turned to + * Iterator[InternalRow] using ColumnarBatch. + * + * Note on memory usage: + * Both the Python worker and the Java executor need to have enough memory to + * hold the largest cogroup. The memory on the Java side is used to construct the + * record batches (off heap memory). The memory on the Python side is used for + * holding the `pandas.DataFrame`. It's possible to further split one group into + * multiple record batches to reduce the memory footprint on the Java side, this + * is left as future work. + */ case class FlatMapCoGroupsInPandasExec( leftGroup: Seq[Attribute], rightGroup: Seq[Attribute], @@ -56,7 +76,7 @@ case class FlatMapCoGroupsInPandasExec( val data = new CoGroupedIterator(leftGrouped, rightGrouped, leftGroup) .map{case (_, l, r) => (l, r)} - val runner = new InterleavedArrowPythonRunner( + val runner = new CogroupedArrowPythonRunner( chainedFunc, PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, Array(leftArgOffsets ++ rightArgOffsets), From 19360c4892bbf7919f511fcaf788ebe92ff64c2f Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Tue, 23 Jul 2019 09:34:05 +0100 Subject: [PATCH 24/34] minor formatting --- .../org/apache/spark/sql/RelationalGroupedDataset.scala | 2 +- .../spark/sql/execution/python/BasePandasGroupExec.scala | 6 +++--- .../sql/execution/python/FlatMapCoGroupsInPandasExec.scala | 3 --- 3 files changed, 4 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index 8f5db8a794c40..0b1c071937e22 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -527,7 +527,7 @@ class RelationalGroupedDataset protected[sql]( * Applies a vectorized python user-defined function to each cogrouped data. * The user-defined function defines a transformation: * `pandas.DataFrame`, `pandas.DataFrame` -> `pandas.DataFrame`. - * For each group int he cogrouped data, all elements in the group are passed as a + * For each group in the cogrouped data, all elements in the group are passed as a * `pandas.DataFrame` and the results for all cogroups are combined into a new [[DataFrame]]. * * This function uses Apache Arrow as serialization format between Java executors and Python diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala index 404869231c348..1df9a890a62e0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala @@ -29,7 +29,9 @@ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.ArrowUtils import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch} - +/** + * Base functionality for plans which execute grouped python udfs. + */ abstract class BasePandasGroupExec(func: Expression, output: Seq[Attribute]) extends SparkPlan { @@ -60,7 +62,6 @@ abstract class BasePandasGroupExec(func: Expression, flattenedBatch.setNumRows(batch.numRows()) flattenedBatch.rowIterator.asScala }.map(unsafeProj) - } protected def groupAndDedup( @@ -129,5 +130,4 @@ abstract class BasePandasGroupExec(func: Expression, val dedupSchema = StructType.fromAttributes(dedupAttributes) (dedupSchema, dedupAttributes, argOffsets) } - } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala index dfd452710676d..ba29c27116a8e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala @@ -86,9 +86,6 @@ case class FlatMapCoGroupsInPandasExec( pythonRunnerConf) executePython(data, runner) - } - } - } From 28493b43f78180b46d3701044171cb825fc92ef4 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Tue, 23 Jul 2019 10:52:45 +0100 Subject: [PATCH 25/34] a couple more usnit tests --- .../tests/test_pandas_udf_cogrouped_map.py | 35 +++++++++++++++++++ 1 file changed, 35 insertions(+) diff --git a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py index 11fd78944e733..cb004bf381902 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py +++ b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py @@ -122,6 +122,41 @@ def merge_pandas(l, r): assert_frame_equal(expected, result, check_column_type=_check_column_type) + def test_empty_group_by(self): + left = self.data1 + right = self.data2 + + @pandas_udf('id long, k int, v int, v2 int', PandasUDFType.COGROUPED_MAP) + def merge_pandas(l, r): + return pd.merge(l, r, on=['id', 'k']) + + result = left.groupby().cogroup(right.groupby())\ + .apply(merge_pandas) \ + .sort(['id', 'k']) \ + .toPandas() + + left = left.toPandas() + right = right.toPandas() + + expected = pd \ + .merge(left, right, on=['id', 'k']) \ + .sort_values(by=['id', 'k']) + + assert_frame_equal(expected, result, check_column_type=_check_column_type) + + def test_mixed_scalar_udfs_followed_by_cogrouby_apply(self): + df = self.spark.range(0, 10).toDF('v1') + df = df.withColumn('v2', udf(lambda x: x + 1, 'int')(df['v1'])) \ + .withColumn('v3', pandas_udf(lambda x: x + 2, 'int')(df['v1'])) + + result = df.groupby().cogroup(df.groupby())\ + .apply(pandas_udf(lambda x, y: pd.DataFrame([(x.sum().sum(), y.sum().sum())]), + 'sum1 int, sum2 int', + PandasUDFType.COGROUPED_MAP)).collect() + + self.assertEquals(result[0]['sum1'], 165) + self.assertEquals(result[0]['sum2'], 165) + def test_with_key_left(self): self._test_with_key(self.data1, self.data1, isLeft=True) From d6d11e456002e5dc5386ffb0b0aab8c3ca702dd5 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Tue, 23 Jul 2019 10:53:11 +0100 Subject: [PATCH 26/34] minor formatting --- .../sql/execution/python/FlatMapCoGroupsInPandasExec.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala index ba29c27116a8e..812de0e313215 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala @@ -21,7 +21,7 @@ import org.apache.spark.api.python.PythonEvalType import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Distribution, Partitioning} +import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, Partitioning} import org.apache.spark.sql.execution.{BinaryExecNode, CoGroupedIterator, SparkPlan} @@ -56,7 +56,9 @@ case class FlatMapCoGroupsInPandasExec( override def outputPartitioning: Partitioning = left.outputPartitioning override def requiredChildDistribution: Seq[Distribution] = { - ClusteredDistribution(leftGroup) :: ClusteredDistribution(rightGroup) :: Nil + val leftDist = if (leftGroup.isEmpty) AllTuples else ClusteredDistribution(leftGroup) + val rightDist = if (rightGroup.isEmpty) AllTuples else ClusteredDistribution(rightGroup) + leftDist :: rightDist :: Nil } override def requiredChildOrdering: Seq[Seq[SortOrder]] = { From a62a1e32d19400d3ba2c63bb55ee1e746904f8f1 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Thu, 25 Jul 2019 10:54:39 +0100 Subject: [PATCH 27/34] more doc --- python/pyspark/serializers.py | 2 +- python/pyspark/worker.py | 1 + .../python/BasePandasGroupExec.scala | 55 ++++++++++--------- .../python/FlatMapCoGroupsInPandasExec.scala | 13 +++-- .../python/FlatMapGroupsInPandasExec.scala | 6 +- 5 files changed, 41 insertions(+), 36 deletions(-) diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 8f15659faa1b5..ef2f5998f6b39 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -435,7 +435,7 @@ def __init__(self, timezone, safecheck, assign_cols_by_name): def load_stream(self, stream): """ - Deserialize Cogroupes ArrowRecordBatches to a tuple of Arrow tables and return as a two + Deserialize Cogrouped ArrowRecordBatches to a tuple of Arrow tables and return as a two lists of pandas.Series. """ reader = InterleavedArrowReader(stream) diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 56e01fb9fa166..cb41b0943b17d 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -386,6 +386,7 @@ def map_batch(batch): # group[1]: length of key indexes # group[2.. group[1] +2]: key attributes # group[group[1] +3 group[0]]: value attributes + # See BasePandasGroupExec.resolveArgOffsets for equivalent scala code def extract_key_value_indexes(): parsed = [] i = 0 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala index 1df9a890a62e0..0db7cf5605728 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala @@ -45,7 +45,10 @@ abstract class BasePandasGroupExec(func: Expression, override def producedAttributes: AttributeSet = AttributeSet(output) - + /** + * passes the data to the python runner and coverts the resulting + * columnarbatch into internal rows. + */ protected def executePython[T]( data: Iterator[T], runner: BasePythonRunner[T, ColumnarBatch]): Iterator[InternalRow] = { @@ -64,7 +67,10 @@ abstract class BasePandasGroupExec(func: Expression, }.map(unsafeProj) } - protected def groupAndDedup( + /** + * groups according to grouping attributes and then projects into the deduplicated schema + */ + protected def groupAndProject( input: Iterator[InternalRow], groupingAttributes: Seq[Attribute], inputSchema: Seq[Attribute], @@ -76,21 +82,26 @@ abstract class BasePandasGroupExec(func: Expression, } } - protected def createSchema( - child: SparkPlan, - groupingAttributes: Seq[Attribute]): (StructType, Seq[Attribute], Array[Int]) = { - - // Deduplicate the grouping attributes. - // If a grouping attribute also appears in data attributes, then we don't need to send the - // grouping attribute to Python worker. If a grouping attribute is not in data attributes, - // then we need to send this grouping attribute to python worker. - // - // We use argOffsets to distinguish grouping attributes and data attributes as following: - // - // argOffsets[0] is the length of the argOffsets array - // argOffsets[1] is the length of grouping attributes - // argOffsets[2 .. argOffsets[0]+2] is the arg offsets for grouping attributes - // argOffsets[argOffsets[0]+2 .. ] is the arg offsets for data attributes + /** + * Returns a the deduplicated attributes of the spark plan and the arg offsets of the + * keys and values. + * + * The deduplicated attributes are needed because the spark plan may contain an attribute + * twice; once in the key and once in the value. For any such attribute we need to + * deduplicate. + * + * The arg offsets are used to distinguish grouping grouping attributes and data attributes + * as following: + * + * argOffsets[0] is the length of the argOffsets array + * + * argOffsets[1] is the length of grouping attribute + * argOffsets[2 .. argOffsets[0]+2] is the arg offsets for grouping attributes + * + * argOffsets[argOffsets[0]+2 .. ] is the arg offsets for data attributes + */ + protected def resolveArgOffsets( + child: SparkPlan, groupingAttributes: Seq[Attribute]): (Seq[Attribute], Array[Int]) = { val dataAttributes = child.output.drop(groupingAttributes.length) val groupingIndicesInData = groupingAttributes.map { attribute => @@ -101,13 +112,6 @@ abstract class BasePandasGroupExec(func: Expression, val nonDupGroupingAttributes = new ArrayBuffer[Attribute] val nonDupGroupingSize = groupingIndicesInData.count(_ == -1) - // Non duplicate grouping attributes are added to nonDupGroupingAttributes and - // their offsets are 0, 1, 2 ... - // Duplicate grouping attributes are NOT added to nonDupGroupingAttributes and - // their offsets are n + index, where n is the total number of non duplicate grouping - // attributes and index is the index in the data attributes that the grouping attribute - // is a duplicate of. - groupingAttributes.zip(groupingIndicesInData).foreach { case (attribute, index) => if (index == -1) { @@ -127,7 +131,6 @@ abstract class BasePandasGroupExec(func: Expression, // Attributes after deduplication val dedupAttributes = nonDupGroupingAttributes ++ dataAttributes - val dedupSchema = StructType.fromAttributes(dedupAttributes) - (dedupSchema, dedupAttributes, argOffsets) + (dedupAttributes, argOffsets) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala index 812de0e313215..0f7d871303db4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, Partitioning} import org.apache.spark.sql.execution.{BinaryExecNode, CoGroupedIterator, SparkPlan} +import org.apache.spark.sql.types.StructType /** @@ -68,13 +69,13 @@ case class FlatMapCoGroupsInPandasExec( override protected def doExecute(): RDD[InternalRow] = { - val (schemaLeft, leftDedup, leftArgOffsets) = createSchema(left, leftGroup) - val (schemaRight, rightDedup, rightArgOffsets) = createSchema(right, rightGroup) + val (leftDedup, leftArgOffsets) = resolveArgOffsets(left, leftGroup) + val (rightDedup, rightArgOffsets) = resolveArgOffsets(right, rightGroup) left.execute().zipPartitions(right.execute()) { (leftData, rightData) => - val leftGrouped = groupAndDedup(leftData, leftGroup, left.output, leftDedup) - val rightGrouped = groupAndDedup(rightData, rightGroup, right.output, rightDedup) + val leftGrouped = groupAndProject(leftData, leftGroup, left.output, leftDedup) + val rightGrouped = groupAndProject(rightData, rightGroup, right.output, rightDedup) val data = new CoGroupedIterator(leftGrouped, rightGrouped, leftGroup) .map{case (_, l, r) => (l, r)} @@ -82,8 +83,8 @@ case class FlatMapCoGroupsInPandasExec( chainedFunc, PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, Array(leftArgOffsets ++ rightArgOffsets), - schemaLeft, - schemaRight, + StructType.fromAttributes(leftDedup), + StructType.fromAttributes(rightDedup), sessionLocalTimeZone, pythonRunnerConf) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala index 960f5131358c7..1fb0df09394f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala @@ -71,19 +71,19 @@ case class FlatMapGroupsInPandasExec( override protected def doExecute(): RDD[InternalRow] = { val inputRDD = child.execute() - val (dedupSchema, dedupAttributes, argOffsets) = createSchema(child, groupingAttributes) + val (dedupAttributes, argOffsets) = resolveArgOffsets(child, groupingAttributes) // Map grouped rows to ArrowPythonRunner results, Only execute if partition is not empty inputRDD.mapPartitionsInternal { iter => if (iter.isEmpty) iter else { - val data = groupAndDedup(iter, groupingAttributes, child.output, dedupAttributes) + val data = groupAndProject(iter, groupingAttributes, child.output, dedupAttributes) .map{case(_, x) => x} val runner = new ArrowPythonRunner( chainedFunc, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF, Array(argOffsets), - dedupSchema, + StructType.fromAttributes(dedupAttributes), sessionLocalTimeZone, pythonRunnerConf) From ec782848391665b46577055c89223d50689d3fea Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Thu, 25 Jul 2019 14:30:47 +0100 Subject: [PATCH 28/34] added comment to cogroup func --- python/pyspark/sql/group.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/python/pyspark/sql/group.py b/python/pyspark/sql/group.py index 292155d674300..cce5cdb8ae429 100644 --- a/python/pyspark/sql/group.py +++ b/python/pyspark/sql/group.py @@ -221,9 +221,16 @@ def pivot(self, pivot_col, values=None): jgd = self._jgd.pivot(pivot_col, values) return GroupedData(jgd, self._df) + @since(3.0) def cogroup(self, other): + """ + Cogroups this group with another group so that we can run cogrouped operations. + + See :class:`CoGroupedData` for the operations that can be run. + """ return CoGroupedData(self, other) + @since(2.3) def apply(self, udf): """ From 1a9ff58aec0f2af59349372a21612f4908bcc385 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Thu, 25 Jul 2019 14:36:55 +0100 Subject: [PATCH 29/34] fixed python style --- python/pyspark/sql/group.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/pyspark/sql/group.py b/python/pyspark/sql/group.py index cce5cdb8ae429..4788aaf96f7bc 100644 --- a/python/pyspark/sql/group.py +++ b/python/pyspark/sql/group.py @@ -230,7 +230,6 @@ def cogroup(self, other): """ return CoGroupedData(self, other) - @since(2.3) def apply(self, udf): """ From c0d291961316950d0390ab588222d18160fc9b12 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Tue, 20 Aug 2019 16:49:18 +0100 Subject: [PATCH 30/34] review comments --- python/pyspark/serializers.py | 51 +++++++------------ python/pyspark/sql/cogroup.py | 2 +- .../tests/test_pandas_udf_cogrouped_map.py | 8 +-- python/pyspark/worker.py | 16 +++--- 4 files changed, 30 insertions(+), 47 deletions(-) diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index ef2f5998f6b39..487bf370c5c32 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -356,33 +356,6 @@ def __repr__(self): return "ArrowStreamPandasSerializer" -class InterleavedArrowReader(object): - - def __init__(self, stream): - self._stream = stream - - def __iter__(self): - return self - - def __next__(self): - dataframes_in_group = read_int(self._stream) - if dataframes_in_group == 2: - return self._read_df(), self._read_df() - elif dataframes_in_group == 0: - raise StopIteration - else: - raise ValueError( - 'Received Invalid number of dataframes in group {0}'.format(dataframes_in_group)) - - def next(self): - return self.__next__() - - def _read_df(self): - import pyarrow as pa - reader = pa.ipc.open_stream(self._stream) - return [b for b in reader] - - class ArrowStreamPandasUDFSerializer(ArrowStreamPandasSerializer): """ Serializer used by Python worker to evaluate Pandas UDFs @@ -428,21 +401,31 @@ def __repr__(self): return "ArrowStreamPandasUDFSerializer" -class PandasCogroupSerializer(ArrowStreamPandasUDFSerializer): +class CogroupUDFSerializer(ArrowStreamPandasUDFSerializer): def __init__(self, timezone, safecheck, assign_cols_by_name): - super(PandasCogroupSerializer, self).__init__(timezone, safecheck, assign_cols_by_name) + super(CogroupUDFSerializer, self).__init__(timezone, safecheck, assign_cols_by_name) def load_stream(self, stream): """ Deserialize Cogrouped ArrowRecordBatches to a tuple of Arrow tables and return as a two lists of pandas.Series. """ - reader = InterleavedArrowReader(stream) - for batch1, batch2 in reader: - import pyarrow as pa - yield ([self.arrow_to_pandas(c) for c in pa.Table.from_batches(batch1).itercolumns()], - [self.arrow_to_pandas(c) for c in pa.Table.from_batches(batch2).itercolumns()]) + import pyarrow as pa + dataframes_in_group = None + + while dataframes_in_group is None or dataframes_in_group > 0: + dataframes_in_group = read_int(stream) + + if dataframes_in_group == 2: + batch1 = [batch for batch in ArrowStreamSerializer.load_stream(self, stream)] + batch2 = [batch for batch in ArrowStreamSerializer.load_stream(self, stream)] + yield ([self.arrow_to_pandas(c) for c in pa.Table.from_batches(batch1).itercolumns()], + [self.arrow_to_pandas(c) for c in pa.Table.from_batches(batch2).itercolumns()]) + + elif dataframes_in_group != 0: + raise ValueError( + 'Received Invalid number of dataframes in group {0}'.format(dataframes_in_group)) class BatchedSerializer(Serializer): diff --git a/python/pyspark/sql/cogroup.py b/python/pyspark/sql/cogroup.py index b1ddd5b614ecf..7438f8c9f93f9 100644 --- a/python/pyspark/sql/cogroup.py +++ b/python/pyspark/sql/cogroup.py @@ -44,7 +44,7 @@ def apply(self, udf): as a `DataFrame`. The user-defined function should take two `pandas.DataFrame` and return another - `pandas.DataFrame`. For each side of the cogroup, all columns are passed together + ``pandas.DataFrame``. For each side of the cogroup, all columns are passed together as a `pandas.DataFrame` to the user-function and the returned `pandas.DataFrame` are combined as a :class:`DataFrame`. diff --git a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py index cb004bf381902..27099fcd6c974 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py +++ b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py @@ -96,11 +96,11 @@ def test_complex_group_by(self): 'v2': [90, 100, 110] }) - left_df = self.spark\ + left_gdf = self.spark\ .createDataFrame(left)\ .groupby(col('id') % 2 == 0) - right_df = self.spark \ + right_gdf = self.spark \ .createDataFrame(right) \ .groupby(col('id') % 2 == 0) @@ -108,8 +108,8 @@ def test_complex_group_by(self): def merge_pandas(l, r): return pd.merge(l[['k', 'v']], r[['k', 'v2']], on=['k']) - result = left_df \ - .cogroup(right_df) \ + result = left_gdf \ + .cogroup(right_gdf) \ .apply(merge_pandas) \ .sort(['k']) \ .toPandas() diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index cb41b0943b17d..cc51bd5397988 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -39,7 +39,7 @@ from pyspark.rdd import PythonEvalType from pyspark.serializers import write_with_length, write_int, read_long, read_bool, \ write_long, read_int, SpecialLengths, UTF8Deserializer, PickleSerializer, \ - BatchedSerializer, ArrowStreamPandasUDFSerializer, PandasCogroupSerializer + BatchedSerializer, ArrowStreamPandasUDFSerializer, CogroupUDFSerializer from pyspark.sql.types import to_arrow_type, StructType from pyspark.util import _get_argspec, fail_on_stopiteration from pyspark import shuffle @@ -314,7 +314,7 @@ def read_udfs(pickleSer, infile, eval_type): # Scalar Pandas UDF handles struct type arguments as pandas DataFrames instead of # pandas Series. See SPARK-27240. if eval_type == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: - ser = PandasCogroupSerializer(timezone, safecheck, assign_cols_by_name) + ser = CogroupUDFSerializer(timezone, safecheck, assign_cols_by_name) else: df_for_struct = (eval_type == PythonEvalType.SQL_SCALAR_PANDAS_UDF or eval_type == PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF or @@ -418,8 +418,8 @@ def extract_key_value_indexes(): pickleSer, infile, eval_type, runner_conf, udf_index=0) udfs['f'] = udf parsed_offsets = extract_key_value_indexes() - keys = ["a[%d]" % o for o in parsed_offsets[0][0]] - vals = ["a[%d]" % o for o in parsed_offsets[0][1]] + keys = ["a[%d]" % (o,) for o in parsed_offsets[0][0]] + vals = ["a[%d]" % (o, ) for o in parsed_offsets[0][1]] mapper_str = "lambda a: f([%s], [%s])" % (", ".join(keys), ", ".join(vals)) elif eval_type == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: # We assume there is only one UDF here because cogrouped map doesn't @@ -429,10 +429,10 @@ def extract_key_value_indexes(): pickleSer, infile, eval_type, runner_conf, udf_index=0) udfs['f'] = udf parsed_offsets = extract_key_value_indexes() - df1_keys = ["a[0][%d]" % o for o in parsed_offsets[0][0]] - df1_vals = ["a[0][%d]" % o for o in parsed_offsets[0][1]] - df2_keys = ["a[1][%d]" % o for o in parsed_offsets[1][0]] - df2_vals = ["a[1][%d]" % o for o in parsed_offsets[1][1]] + df1_keys = ["a[0][%d]" % (o, ) for o in parsed_offsets[0][0]] + df1_vals = ["a[0][%d]" % (o, )for o in parsed_offsets[0][1]] + df2_keys = ["a[1][%d]" % (o, ) for o in parsed_offsets[1][0]] + df2_vals = ["a[1][%d]" % (o, ) for o in parsed_offsets[1][1]] mapper_str = "lambda a: f([%s], [%s], [%s], [%s])" % ( ", ".join(df1_keys), ", ".join(df1_vals), ", ".join(df2_keys), ", ".join(df2_vals)) else: From 4cd5c70656391c4637d576db79719e79e944002a Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Tue, 20 Aug 2019 16:50:27 +0100 Subject: [PATCH 31/34] review comments scala --- .../plans/logical/pythonLogicalOperators.scala | 12 ++++++------ .../sql/execution/python/BasePandasGroupExec.scala | 1 - .../python/FlatMapCoGroupsInPandasExec.scala | 2 +- 3 files changed, 7 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala index fee010c3c1111..66122b3a52cd0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala @@ -56,12 +56,12 @@ case class MapInPandas( * This is used by DataFrame.groupby().cogroup().apply(). */ case class FlatMapCoGroupsInPandas( - leftAttributes: Seq[Attribute], - rightAttributes: Seq[Attribute], - functionExpr: Expression, - output: Seq[Attribute], - left: LogicalPlan, - right: LogicalPlan) extends BinaryNode { + leftAttributes: Seq[Attribute], + rightAttributes: Seq[Attribute], + functionExpr: Expression, + output: Seq[Attribute], + left: LogicalPlan, + right: LogicalPlan) extends BinaryNode { override val producedAttributes = AttributeSet(output) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala index 0db7cf5605728..eb5bafbbe54cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala @@ -25,7 +25,6 @@ import org.apache.spark.api.python.{BasePythonRunner, ChainedPythonFunctions} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression, PythonUDF, UnsafeProjection} import org.apache.spark.sql.execution.{GroupedIterator, SparkPlan} -import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.ArrowUtils import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala index 0f7d871303db4..18d493437c840 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.types.StructType * * The input dataframes are first Cogrouped. Rows from each side of the cogroup are passed to the * Python worker via Arrow. As each side of the cogroup may have a different schema we send every - * group in it's own Arrow stream. + * group in its own Arrow stream. * The Python worker turns the resulting record batches to `pandas.DataFrame`s, invokes the * user-defined function, and passes the resulting `pandas.DataFrame` * as an Arrow record batch. Finally, each record batch is turned to From dd1ffaf86f5d1a44fd812540b405e2eb8a21e6cf Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Tue, 20 Aug 2019 17:39:43 +0100 Subject: [PATCH 32/34] python formatting --- python/pyspark/serializers.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index bd7637fba9bcc..17d270e6649c2 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -427,8 +427,7 @@ def load_stream(self, stream): elif dataframes_in_group != 0: raise ValueError( - 'Received Invalid number of dataframes in group {0}' - .format(dataframes_in_group)) + 'Invalid number of dataframes in group {0}'.format(dataframes_in_group)) class BatchedSerializer(Serializer): From 733b59277b51d36e4640d3ea0ff3e097aa301294 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Sun, 8 Sep 2019 09:12:51 +0100 Subject: [PATCH 33/34] review comments (mainly formatting) --- python/pyspark/serializers.py | 7 +-- python/pyspark/sql/cogroup.py | 13 +++-- .../tests/test_pandas_udf_cogrouped_map.py | 7 +-- python/pyspark/worker.py | 52 ++++++++++--------- .../logical/pythonLogicalOperators.scala | 3 +- .../execution/python/ArrowPythonRunner.scala | 6 +-- .../python/BaseArrowPythonRunner.scala | 3 -- .../python/BasePandasGroupExec.scala | 6 ++- .../python/CogroupedArrowPythonRunner.scala | 7 ++- .../python/FlatMapCoGroupsInPandasExec.scala | 31 ++++++----- .../python/FlatMapGroupsInPandasExec.scala | 13 ++--- 11 files changed, 68 insertions(+), 80 deletions(-) diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 17d270e6649c2..bceb92cb274ae 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -403,12 +403,9 @@ def __repr__(self): class CogroupUDFSerializer(ArrowStreamPandasUDFSerializer): - def __init__(self, timezone, safecheck, assign_cols_by_name): - super(CogroupUDFSerializer, self).__init__(timezone, safecheck, assign_cols_by_name) - def load_stream(self, stream): """ - Deserialize Cogrouped ArrowRecordBatches to a tuple of Arrow tables and return as a two + Deserialize Cogrouped ArrowRecordBatches to a tuple of Arrow tables and yield as two lists of pandas.Series. """ import pyarrow as pa @@ -427,7 +424,7 @@ def load_stream(self, stream): elif dataframes_in_group != 0: raise ValueError( - 'Invalid number of dataframes in group {0}'.format(dataframes_in_group)) + 'Invalid number of pandas.DataFrames in group {0}'.format(dataframes_in_group)) class BatchedSerializer(Serializer): diff --git a/python/pyspark/sql/cogroup.py b/python/pyspark/sql/cogroup.py index 7438f8c9f93f9..ddcff8824e2ee 100644 --- a/python/pyspark/sql/cogroup.py +++ b/python/pyspark/sql/cogroup.py @@ -23,14 +23,13 @@ class CoGroupedData(object): """ - A logical grouping of two :class:`GroupedData`, - created by :func:`GroupedData.cogroup`. + A logical grouping of two :class:`GroupedData`, + created by :func:`GroupedData.cogroup`. - .. note:: Experimental + .. note:: Experimental - .. versionadded:: 3.0 - - """ + .. versionadded:: 3.0 + """ def __init__(self, gd1, gd2): self._gd1 = gd1 @@ -53,7 +52,7 @@ def apply(self, udf): .. note:: This function requires a full shuffle. All the data of a cogroup will be loaded into memory, so the user should be aware of the potential OOM risk if data is skewed - and certain goroups are too large to fit in memory. + and certain groups are too large to fit in memory. .. note:: Experimental diff --git a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py index 27099fcd6c974..7f3f7fa3168a7 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py +++ b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py @@ -15,16 +15,11 @@ # limitations under the License. # -import datetime import unittest import sys -from collections import OrderedDict -from decimal import Decimal - -from pyspark.sql import Row from pyspark.sql.functions import array, explode, col, lit, udf, sum, pandas_udf, PandasUDFType -from pyspark.sql.types import * +from pyspark.sql.types import DoubleType, StructType, StructField from pyspark.testing.sqlutils import ReusedSQLTestCase, have_pandas, have_pyarrow, \ pandas_requirement_message, pyarrow_requirement_message from pyspark.testing.utils import QuietTest diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index cc51bd5397988..086202de2c68b 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -311,11 +311,11 @@ def read_udfs(pickleSer, infile, eval_type): "spark.sql.legacy.execution.pandas.groupedMap.assignColumnsByName", "true")\ .lower() == "true" - # Scalar Pandas UDF handles struct type arguments as pandas DataFrames instead of - # pandas Series. See SPARK-27240. if eval_type == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: ser = CogroupUDFSerializer(timezone, safecheck, assign_cols_by_name) else: + # Scalar Pandas UDF handles struct type arguments as pandas DataFrames instead of + # pandas Series. See SPARK-27240. df_for_struct = (eval_type == PythonEvalType.SQL_SCALAR_PANDAS_UDF or eval_type == PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF or eval_type == PythonEvalType.SQL_MAP_PANDAS_ITER_UDF) @@ -377,28 +377,30 @@ def map_batch(batch): # profiling is not supported for UDF return func, None, ser, ser - # Helper function to extract the key and value indexs from arg_offsets - # arg_offsets is a List containing the key and value - # indexes of columns of the DataFrames to be passed to the udf. - # It consists of n repeating groups where n is the number of - # DataFrames. Each group has the following format. - # group[0]: length of group - # group[1]: length of key indexes - # group[2.. group[1] +2]: key attributes - # group[group[1] +3 group[0]]: value attributes - # See BasePandasGroupExec.resolveArgOffsets for equivalent scala code - def extract_key_value_indexes(): + def extract_key_value_indexes(grouped_arg_offsets): + """ + Helper function to extract the key and value indexes from arg_offsets for the grouped and + cogrouped pandas udfs. See BasePandasGroupExec.resolveArgOffsets for equivalent scala code. + + :param grouped_arg_offsets: List containing the key and value indexes of columns of the + DataFrames to be passed to the udf. It consists of n repeating groups where n is the + number of DataFrames. Each group has the following format: + group[0]: length of group + group[1]: length of key indexes + group[2.. group[1] +2]: key attributes + group[group[1] +3 group[0]]: value attributes + """ parsed = [] - i = 0 - while i < len(arg_offsets): - offsets_len = arg_offsets[i] - i += 1 - offsets = arg_offsets[i: i + offsets_len] + idx = 0 + while idx < len(grouped_arg_offsets): + offsets_len = grouped_arg_offsets[idx] + idx += 1 + offsets = grouped_arg_offsets[idx: idx + offsets_len] split_index = offsets[0] + 1 - keys = offsets[1: split_index] - values = offsets[split_index:] - parsed.append([keys, values]) - i += offsets_len + offset_keys = offsets[1: split_index] + offset_values = offsets[split_index:] + parsed.append([offset_keys, offset_values]) + idx += offsets_len return parsed udfs = {} @@ -417,7 +419,7 @@ def extract_key_value_indexes(): arg_offsets, udf = read_single_udf( pickleSer, infile, eval_type, runner_conf, udf_index=0) udfs['f'] = udf - parsed_offsets = extract_key_value_indexes() + parsed_offsets = extract_key_value_indexes(arg_offsets) keys = ["a[%d]" % (o,) for o in parsed_offsets[0][0]] vals = ["a[%d]" % (o, ) for o in parsed_offsets[0][1]] mapper_str = "lambda a: f([%s], [%s])" % (", ".join(keys), ", ".join(vals)) @@ -428,9 +430,9 @@ def extract_key_value_indexes(): arg_offsets, udf = read_single_udf( pickleSer, infile, eval_type, runner_conf, udf_index=0) udfs['f'] = udf - parsed_offsets = extract_key_value_indexes() + parsed_offsets = extract_key_value_indexes(arg_offsets) df1_keys = ["a[0][%d]" % (o, ) for o in parsed_offsets[0][0]] - df1_vals = ["a[0][%d]" % (o, )for o in parsed_offsets[0][1]] + df1_vals = ["a[0][%d]" % (o, ) for o in parsed_offsets[0][1]] df2_keys = ["a[1][%d]" % (o, ) for o in parsed_offsets[1][0]] df2_vals = ["a[1][%d]" % (o, ) for o in parsed_offsets[1][1]] mapper_str = "lambda a: f([%s], [%s], [%s], [%s])" % ( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala index 66122b3a52cd0..c4f741cd2cec8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala @@ -52,7 +52,7 @@ case class MapInPandas( } /** - * Flatmap cogroups using a udf: pandas.Dataframe, pandas.Dataframe -> pandas.Dataframe + * Flatmap cogroups using a udf: pandas.Dataframe, pandas.Dataframe -> pandas.Dataframe * This is used by DataFrame.groupby().cogroup().apply(). */ case class FlatMapCoGroupsInPandas( @@ -66,7 +66,6 @@ case class FlatMapCoGroupsInPandas( override val producedAttributes = AttributeSet(output) } - trait BaseEvalPython extends UnaryNode { def udfs: Seq[PythonUDF] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala index 2ff2517a90da3..fcf68467460bb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala @@ -19,12 +19,9 @@ package org.apache.spark.sql.execution.python import java.io._ import java.net._ -import java.util.concurrent.atomic.AtomicBoolean - -import scala.collection.JavaConverters._ import org.apache.arrow.vector.VectorSchemaRoot -import org.apache.arrow.vector.ipc.{ArrowStreamReader, ArrowStreamWriter} +import org.apache.arrow.vector.ipc.ArrowStreamWriter import org.apache.spark._ import org.apache.spark.api.python._ @@ -33,7 +30,6 @@ import org.apache.spark.sql.execution.arrow.ArrowWriter import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.sql.util.ArrowUtils -import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch, ColumnVector} import org.apache.spark.util.Utils /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BaseArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BaseArrowPythonRunner.scala index bc5d38e899cb6..0cee7d2f96c22 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BaseArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BaseArrowPythonRunner.scala @@ -40,7 +40,6 @@ abstract class BaseArrowPythonRunner[T]( argOffsets: Array[Array[Int]]) extends BasePythonRunner[T, ColumnarBatch](funcs, evalType, argOffsets) { - protected override def newReaderIterator( stream: DataInputStream, writerThread: WriterThread, @@ -111,5 +110,3 @@ abstract class BaseArrowPythonRunner[T]( } } } - - diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala index eb5bafbbe54cd..477c288ad1211 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala @@ -31,8 +31,10 @@ import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch} /** * Base functionality for plans which execute grouped python udfs. */ -abstract class BasePandasGroupExec(func: Expression, - output: Seq[Attribute]) extends SparkPlan { +abstract class BasePandasGroupExec( + func: Expression, + output: Seq[Attribute]) + extends SparkPlan { protected val sessionLocalTimeZone = conf.sessionLocalTimeZone diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CogroupedArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CogroupedArrowPythonRunner.scala index 5c8751175b57d..8ea9881c575a1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CogroupedArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CogroupedArrowPythonRunner.scala @@ -81,8 +81,11 @@ class CogroupedArrowPythonRunner( dataOut.writeInt(0) } - def writeGroup(group: Iterator[InternalRow], schema: StructType, dataOut: DataOutputStream, - name: String) = { + def writeGroup( + group: Iterator[InternalRow], + schema: StructType, + dataOut: DataOutputStream, + name: String) = { val arrowSchema = ArrowUtils.toArrowSchema(schema, timeZoneId) val allocator = ArrowUtils.rootAllocator.newChildAllocator( s"stdout writer for $pythonExec ($name)", 0, Long.MaxValue) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala index 18d493437c840..cc83e0cecdc33 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala @@ -52,7 +52,7 @@ case class FlatMapCoGroupsInPandasExec( output: Seq[Attribute], left: SparkPlan, right: SparkPlan) - extends BasePandasGroupExec(func, output) with BinaryExecNode{ + extends BasePandasGroupExec(func, output) with BinaryExecNode { override def outputPartitioning: Partitioning = left.outputPartitioning @@ -72,23 +72,26 @@ case class FlatMapCoGroupsInPandasExec( val (leftDedup, leftArgOffsets) = resolveArgOffsets(left, leftGroup) val (rightDedup, rightArgOffsets) = resolveArgOffsets(right, rightGroup) + // Map cogrouped rows to ArrowPythonRunner results, Only execute if partition is not empty left.execute().zipPartitions(right.execute()) { (leftData, rightData) => + if (leftData.isEmpty && rightData.isEmpty) Iterator.empty else { - val leftGrouped = groupAndProject(leftData, leftGroup, left.output, leftDedup) - val rightGrouped = groupAndProject(rightData, rightGroup, right.output, rightDedup) - val data = new CoGroupedIterator(leftGrouped, rightGrouped, leftGroup) - .map{case (_, l, r) => (l, r)} + val leftGrouped = groupAndProject(leftData, leftGroup, left.output, leftDedup) + val rightGrouped = groupAndProject(rightData, rightGroup, right.output, rightDedup) + val data = new CoGroupedIterator(leftGrouped, rightGrouped, leftGroup) + .map { case (_, l, r) => (l, r) } - val runner = new CogroupedArrowPythonRunner( - chainedFunc, - PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, - Array(leftArgOffsets ++ rightArgOffsets), - StructType.fromAttributes(leftDedup), - StructType.fromAttributes(rightDedup), - sessionLocalTimeZone, - pythonRunnerConf) + val runner = new CogroupedArrowPythonRunner( + chainedFunc, + PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, + Array(leftArgOffsets ++ rightArgOffsets), + StructType.fromAttributes(leftDedup), + StructType.fromAttributes(rightDedup), + sessionLocalTimeZone, + pythonRunnerConf) - executePython(data, runner) + executePython(data, runner) + } } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala index 1fb0df09394f6..22a0d1e09b12e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala @@ -17,19 +17,14 @@ package org.apache.spark.sql.execution.python -import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.TaskContext -import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType} +import org.apache.spark.api.python.PythonEvalType import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, Partitioning} -import org.apache.spark.sql.execution.{GroupedIterator, SparkPlan, UnaryExecNode} +import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.util.ArrowUtils -import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch} + /** * Physical node for [[org.apache.spark.sql.catalyst.plans.logical.FlatMapGroupsInPandas]] @@ -53,7 +48,7 @@ case class FlatMapGroupsInPandasExec( func: Expression, output: Seq[Attribute], child: SparkPlan) - extends BasePandasGroupExec(func, output) with UnaryExecNode { + extends BasePandasGroupExec(func, output) with UnaryExecNode { override def outputPartitioning: Partitioning = child.outputPartitioning From 1b966fda46c5334cf7963bae0bece159c9568622 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Sun, 15 Sep 2019 09:33:23 +0100 Subject: [PATCH 34/34] couple more format changes --- python/pyspark/sql/cogroup.py | 2 +- .../org/apache/spark/sql/RelationalGroupedDataset.scala | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/python/pyspark/sql/cogroup.py b/python/pyspark/sql/cogroup.py index ddcff8824e2ee..9b725e4bafe79 100644 --- a/python/pyspark/sql/cogroup.py +++ b/python/pyspark/sql/cogroup.py @@ -43,7 +43,7 @@ def apply(self, udf): as a `DataFrame`. The user-defined function should take two `pandas.DataFrame` and return another - ``pandas.DataFrame``. For each side of the cogroup, all columns are passed together + `pandas.DataFrame`. For each side of the cogroup, all columns are passed together as a `pandas.DataFrame` to the user-function and the returned `pandas.DataFrame` are combined as a :class:`DataFrame`. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index 0b1c071937e22..f6d13be0e89be 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -533,8 +533,9 @@ class RelationalGroupedDataset protected[sql]( * This function uses Apache Arrow as serialization format between Java executors and Python * workers. */ - private[sql] def flatMapCoGroupsInPandas - (r: RelationalGroupedDataset, expr: PythonUDF): DataFrame = { + private[sql] def flatMapCoGroupsInPandas( + r: RelationalGroupedDataset, + expr: PythonUDF): DataFrame = { require(expr.evalType == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, "Must pass a cogrouped map udf") require(expr.dataType.isInstanceOf[StructType],