|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | +package org.apache.spark.sql.execution.python |
| 18 | + |
| 19 | +import org.apache.spark.TaskContext |
| 20 | +import org.apache.spark.api.python.{BasePythonRunner, ChainedPythonFunctions} |
| 21 | +import org.apache.spark.rdd.RDD |
| 22 | +import org.apache.spark.sql.catalyst.InternalRow |
| 23 | +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, PythonUDF, UnsafeProjection} |
| 24 | +import org.apache.spark.sql.execution.{GroupedIterator, SparkPlan} |
| 25 | +import org.apache.spark.sql.types.StructType |
| 26 | +import org.apache.spark.sql.util.ArrowUtils |
| 27 | +import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch} |
| 28 | + |
| 29 | +import scala.collection.mutable.ArrayBuffer |
| 30 | +import scala.collection.JavaConverters._ |
| 31 | + |
| 32 | +trait AbstractPandasGroupExec extends SparkPlan { |
| 33 | + |
| 34 | + protected val sessionLocalTimeZone = conf.sessionLocalTimeZone |
| 35 | + |
| 36 | + protected val pythonRunnerConf = ArrowUtils.getPythonRunnerConfMap(conf) |
| 37 | + |
| 38 | + protected def chainedFunc = Seq( |
| 39 | + ChainedPythonFunctions(Seq(func.asInstanceOf[PythonUDF].func))) |
| 40 | + |
| 41 | + def output: Seq[Attribute] |
| 42 | + |
| 43 | + def func: Expression |
| 44 | + |
| 45 | + protected def executePython[T](data: Iterator[T], |
| 46 | + runner: BasePythonRunner[T, ColumnarBatch]): Iterator[InternalRow] = { |
| 47 | + |
| 48 | + val context = TaskContext.get() |
| 49 | + val columnarBatchIter = runner.compute(data, context.partitionId(), context) |
| 50 | + val unsafeProj = UnsafeProjection.create(output, output) |
| 51 | + |
| 52 | + columnarBatchIter.flatMap { batch => |
| 53 | + // UDF returns a StructType column in ColumnarBatch, select the children here |
| 54 | + val structVector = batch.column(0).asInstanceOf[ArrowColumnVector] |
| 55 | + val outputVectors = output.indices.map(structVector.getChild) |
| 56 | + val flattenedBatch = new ColumnarBatch(outputVectors.toArray) |
| 57 | + flattenedBatch.setNumRows(batch.numRows()) |
| 58 | + flattenedBatch.rowIterator.asScala |
| 59 | + }.map(unsafeProj) |
| 60 | + |
| 61 | + } |
| 62 | + |
| 63 | + protected def groupAndDedup( |
| 64 | + input: Iterator[InternalRow], groupingAttributes: Seq[Attribute], |
| 65 | + inputSchema: Seq[Attribute], dedupSchema: Seq[Attribute]): Iterator[Iterator[InternalRow]] = { |
| 66 | + if (groupingAttributes.isEmpty) { |
| 67 | + Iterator(input) |
| 68 | + } else { |
| 69 | + val groupedIter = GroupedIterator(input, groupingAttributes, inputSchema) |
| 70 | + val dedupProj = UnsafeProjection.create(dedupSchema, inputSchema) |
| 71 | + groupedIter.map { |
| 72 | + case (_, groupedRowIter) => groupedRowIter.map(dedupProj) |
| 73 | + } |
| 74 | + } |
| 75 | + } |
| 76 | + |
| 77 | + protected def createSchema(child: SparkPlan, groupingAttributes: Seq[Attribute]) |
| 78 | + : (StructType, Seq[Attribute], Array[Array[Int]]) = { |
| 79 | + |
| 80 | + // Deduplicate the grouping attributes. |
| 81 | + // If a grouping attribute also appears in data attributes, then we don't need to send the |
| 82 | + // grouping attribute to Python worker. If a grouping attribute is not in data attributes, |
| 83 | + // then we need to send this grouping attribute to python worker. |
| 84 | + // |
| 85 | + // We use argOffsets to distinguish grouping attributes and data attributes as following: |
| 86 | + // |
| 87 | + // argOffsets[0] is the length of grouping attributes |
| 88 | + // argOffsets[1 .. argOffsets[0]+1] is the arg offsets for grouping attributes |
| 89 | + // argOffsets[argOffsets[0]+1 .. ] is the arg offsets for data attributes |
| 90 | + |
| 91 | + val dataAttributes = child.output.drop(groupingAttributes.length) |
| 92 | + val groupingIndicesInData = groupingAttributes.map { attribute => |
| 93 | + dataAttributes.indexWhere(attribute.semanticEquals) |
| 94 | + } |
| 95 | + |
| 96 | + val groupingArgOffsets = new ArrayBuffer[Int] |
| 97 | + val nonDupGroupingAttributes = new ArrayBuffer[Attribute] |
| 98 | + val nonDupGroupingSize = groupingIndicesInData.count(_ == -1) |
| 99 | + |
| 100 | + // Non duplicate grouping attributes are added to nonDupGroupingAttributes and |
| 101 | + // their offsets are 0, 1, 2 ... |
| 102 | + // Duplicate grouping attributes are NOT added to nonDupGroupingAttributes and |
| 103 | + // their offsets are n + index, where n is the total number of non duplicate grouping |
| 104 | + // attributes and index is the index in the data attributes that the grouping attribute |
| 105 | + // is a duplicate of. |
| 106 | + |
| 107 | + groupingAttributes.zip(groupingIndicesInData).foreach { |
| 108 | + case (attribute, index) => |
| 109 | + if (index == -1) { |
| 110 | + groupingArgOffsets += nonDupGroupingAttributes.length |
| 111 | + nonDupGroupingAttributes += attribute |
| 112 | + } else { |
| 113 | + groupingArgOffsets += index + nonDupGroupingSize |
| 114 | + } |
| 115 | + } |
| 116 | + |
| 117 | + val dataArgOffsets = nonDupGroupingAttributes.length until |
| 118 | + (nonDupGroupingAttributes.length + dataAttributes.length) |
| 119 | + |
| 120 | + val argOffsets = Array(Array(groupingAttributes.length) ++ groupingArgOffsets ++ dataArgOffsets) |
| 121 | + |
| 122 | + // Attributes after deduplication |
| 123 | + val dedupAttributes = nonDupGroupingAttributes ++ dataAttributes |
| 124 | + val dedupSchema = StructType.fromAttributes(dedupAttributes) |
| 125 | + (dedupSchema, dedupAttributes, argOffsets) |
| 126 | + } |
| 127 | + |
| 128 | +} |
0 commit comments