diff --git a/core/src/main/scala/com/intel/oap/execution/ColumnarHashAggregateExec.scala b/core/src/main/scala/com/intel/oap/execution/ColumnarHashAggregateExec.scala index 68468d45d..61f71b859 100644 --- a/core/src/main/scala/com/intel/oap/execution/ColumnarHashAggregateExec.scala +++ b/core/src/main/scala/com/intel/oap/execution/ColumnarHashAggregateExec.scala @@ -49,8 +49,10 @@ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.execution.vectorized.MutableColumnarRow import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} -import org.apache.spark.sql.types.{DecimalType, StringType, StructType} +import org.apache.spark.sql.util.ArrowUtils +import org.apache.spark.sql.types._ import org.apache.spark.unsafe.KVIterator +import scala.collection.JavaConverters._ import scala.collection.Iterator @@ -106,34 +108,115 @@ case class ColumnarHashAggregateExec( buildCheck() override def doExecuteColumnar(): RDD[ColumnarBatch] = { - child.executeColumnar().mapPartitionsWithIndex { (partIndex, iter) => + var eval_elapse: Long = 0 + child.executeColumnar().mapPartitions { iter => ExecutorManager.tryTaskSet(numaBindingInfo) - val hasInput = iter.hasNext - val res = if (!hasInput) { - // This is a grouped aggregate and the input iterator is empty, - // so return an empty iterator. - Iterator.empty - } else { - var aggregation = ColumnarAggregation.create( - partIndex, - groupingExpressions, - child.output, - aggregateExpressions, - aggregateAttributes, - resultExpressions, - output, - numInputBatches, - numOutputBatches, - numOutputRows, - aggTime, - totalTime, - sparkConf) - SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit](_ => { - aggregation.close() - }) - new CloseableColumnBatchIterator(aggregation.createIterator(iter)) + val native_function = TreeBuilder.makeFunction( + s"standalone", + Lists.newArrayList(getKernelFunction), + new ArrowType.Int(32, true)) + val hash_aggr_expr = + TreeBuilder + .makeExpression(native_function, Field.nullable("result", new ArrowType.Int(32, true))) + val hash_aggr_input_schema = ConverterUtils.toArrowSchema(child.output) + val hash_aggr_out_schema = ConverterUtils.toArrowSchema(output) + val resultStructType = ArrowUtils.fromArrowSchema(hash_aggr_out_schema) + val nativeKernel = new ExpressionEvaluator() + nativeKernel + .build( + hash_aggr_input_schema, + Lists.newArrayList(hash_aggr_expr), + hash_aggr_out_schema, + true) + val nativeIterator = nativeKernel.finishByIterator() + + def close = { + aggTime += (eval_elapse / 1000000) + totalTime += (eval_elapse / 1000000) + nativeKernel.close + nativeIterator.close + } + + var numRowsInput = 0 + // now we can return this wholestagecodegen iter + val res = new Iterator[ColumnarBatch] { + var processed = false + var skip_native = false + var count_num_row = 0 + def process: Unit = { + while (iter.hasNext) { + val cb = iter.next() + numInputBatches += 1 + if (cb.numRows != 0) { + numRowsInput += cb.numRows + val beforeEval = System.nanoTime() + if (hash_aggr_input_schema.getFields.size == 0) { + // This is a special case used by only do count literal + count_num_row += cb.numRows + skip_native = true + } else { + val input_rb = + ConverterUtils.createArrowRecordBatch(cb) + nativeIterator.processAndCacheOne(hash_aggr_input_schema, input_rb) + ConverterUtils.releaseArrowRecordBatch(input_rb) + } + eval_elapse += System.nanoTime() - beforeEval + } + } + processed = true + } + override def hasNext: Boolean = { + if (!processed) process + if (skip_native) { + count_num_row > 0 + } else { + nativeIterator.hasNext + } + } + + override def next(): ColumnarBatch = { + if (!processed) process + val beforeEval = System.nanoTime() + if (skip_native) { + // special handling for only count literal in this operator + val out_res = count_num_row + count_num_row = 0 + val resultColumnVectors = + ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray + resultColumnVectors.foreach { v => + { + val numRows = v.dataType match { + case t: IntegerType => + out_res.asInstanceOf[Number].intValue + case t: LongType => + out_res.asInstanceOf[Number].longValue + } + v.put(0, numRows) + } + } + return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 1) + } else { + val output_rb = nativeIterator.next + if (output_rb == null) { + eval_elapse += System.nanoTime() - beforeEval + val resultColumnVectors = + ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray + return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0) + } + val outputNumRows = output_rb.getLength + val output = ConverterUtils.fromArrowRecordBatch(hash_aggr_out_schema, output_rb) + ConverterUtils.releaseArrowRecordBatch(output_rb) + eval_elapse += System.nanoTime() - beforeEval + numOutputRows += outputNumRows + numOutputBatches += 1 + new ColumnarBatch(output.map(v => v.asInstanceOf[ColumnVector]), outputNumRows) + } + } } - res + SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit](_ => { + close + }) + new CloseableColumnBatchIterator(res) } } @@ -153,7 +236,7 @@ case class ColumnarHashAggregateExec( try { ConverterUtils.checkIfTypeSupported(expr.dataType) } catch { - case e : UnsupportedOperationException => + case e: UnsupportedOperationException => throw new UnsupportedOperationException( s"${expr.dataType} is not supported in ColumnarAggregation") } @@ -228,7 +311,7 @@ case class ColumnarHashAggregateExec( // override def canEqual(that: Any): Boolean = false def getKernelFunction: TreeNode = { - ColumnarHashAggregationWithCodegen.prepareKernelFunction( + ColumnarHashAggregation.prepareKernelFunction( groupingExpressions, child.output, aggregateExpressions, diff --git a/core/src/main/scala/com/intel/oap/execution/ColumnarWholeStageCodegenExec.scala b/core/src/main/scala/com/intel/oap/execution/ColumnarWholeStageCodegenExec.scala index adb3df2f6..9d619a153 100644 --- a/core/src/main/scala/com/intel/oap/execution/ColumnarWholeStageCodegenExec.scala +++ b/core/src/main/scala/com/intel/oap/execution/ColumnarWholeStageCodegenExec.scala @@ -474,19 +474,14 @@ case class ColumnarWholeStageCodegenExec(child: SparkPlan)(val codegenStageId: I def process: Unit = { while (iter.hasNext) { val cb = iter.next() - if (cb.numRows == 0) { - val resultColumnVectors = - ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray - return new ColumnarBatch( - resultColumnVectors.map(_.asInstanceOf[ColumnVector]), - 0) + if (cb.numRows != 0) { + val beforeEval = System.nanoTime() + val input_rb = + ConverterUtils.createArrowRecordBatch(cb) + nativeIterator.processAndCacheOne(resCtx.inputSchema, input_rb) + ConverterUtils.releaseArrowRecordBatch(input_rb) + eval_elapse += System.nanoTime() - beforeEval } - val beforeEval = System.nanoTime() - val input_rb = - ConverterUtils.createArrowRecordBatch(cb) - nativeIterator.processAndCacheOne(resCtx.inputSchema, input_rb) - ConverterUtils.releaseArrowRecordBatch(input_rb) - eval_elapse += System.nanoTime() - beforeEval } processed = true } diff --git a/core/src/main/scala/com/intel/oap/expression/ColumnarAggregateExpression.scala b/core/src/main/scala/com/intel/oap/expression/ColumnarAggregateExpression.scala deleted file mode 100644 index bfc9171a4..000000000 --- a/core/src/main/scala/com/intel/oap/expression/ColumnarAggregateExpression.scala +++ /dev/null @@ -1,162 +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 com.intel.oap.expression - -import scala.collection.immutable.List -import scala.collection.JavaConverters._ -import org.apache.arrow.gandiva.expression._ -import org.apache.arrow.vector.types.pojo.ArrowType -import org.apache.arrow.vector.types.pojo.Field - -import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.aggregate._ -import org.apache.spark.sql.types._ - -trait ColumnarAggregateExpressionBase extends ColumnarExpression with Logging { - def requiredColNum: Int - def expectedResColNum: Int - def setInputFields(fieldList: List[Field]): Unit = {} - def doColumnarCodeGen_ext(args: Object): TreeNode = { - throw new UnsupportedOperationException(s"ColumnarAggregateExpressionBase doColumnarCodeGen_ext is a abstract function.") - } -} - -class ColumnarUniqueAggregateExpression(aggrFieldList: List[Field], hashCollisionCheck: Int = 0) extends ColumnarAggregateExpressionBase with Logging { - - override def requiredColNum: Int = 1 - override def expectedResColNum: Int = 1 - override def doColumnarCodeGen_ext(args: Object): TreeNode = { - val (keyFieldList, inputFieldList, resultType, resultField) = - args.asInstanceOf[(List[Field], List[Field], ArrowType, Field)] - val funcName = "action_unique" - val inputNode = { - // if keyList has keys, we need to do groupby by these keys. - var inputFieldNode = - keyFieldList.map({field => TreeBuilder.makeField(field)}).asJava - val encodeArrayFuncName = if (hashCollisionCheck == 1) "encodeArraySafe" else "encodeArray" - val encodeNode = TreeBuilder.makeFunction( - encodeArrayFuncName, - inputFieldNode, - resultType/*this arg won't be used*/) - inputFieldNode = - (encodeNode :: inputFieldList.map({field => TreeBuilder.makeField(field)})).asJava - val groupByFuncNode = TreeBuilder.makeFunction( - "splitArrayListWithAction", - inputFieldNode, - resultType/*this arg won't be used*/) - List(groupByFuncNode) ::: aggrFieldList.map(field => TreeBuilder.makeField(field)) - } - logInfo(s"${funcName}(${inputNode})") - TreeBuilder.makeFunction( - funcName, - inputNode.asJava, - resultType) - } -} - -class ColumnarAggregateExpression( - aggregateFunction: AggregateFunction, - mode: AggregateMode, - isDistinct: Boolean, - resultId: ExprId, hashCollisionCheck: Int = 0) - extends AggregateExpression(aggregateFunction, mode, isDistinct, None, resultId) - with ColumnarAggregateExpressionBase - with Logging { - - var aggrFieldList: List[Field] = _ - val (funcName, argSize, resSize) = mode match { - case Partial => - aggregateFunction.prettyName match { - case "avg" => ("sum_count", 1, 2) - case "count" => { - if (aggregateFunction.children(0).isInstanceOf[Literal]) { - (s"countLiteral_${aggregateFunction.children(0)}", 0, 1) - } else { - ("count", 1, 1) - } - } - case "stddev_samp" => ("stddev_samp_partial", 1, 3) - case other => (aggregateFunction.prettyName, 1, 1) - } - case PartialMerge => - aggregateFunction.prettyName match { - case "avg" => ("sum_count_merge", 2, 2) - case "count" => ("sum", 1, 1) - case other => (aggregateFunction.prettyName, 1, 1) - } - case Final => - aggregateFunction.prettyName match { - case "count" => ("sum", 1, 1) - case "avg" => ("avgByCount", 2, 1) - case "stddev_samp" => ("stddev_samp_final", 3, 1) - case other => (aggregateFunction.prettyName, 1, 1) - } - case _ => - throw new UnsupportedOperationException("doesn't support this mode") - } - - val finalFuncName = funcName match { - case "count" => "sum" - case other => other - } - logInfo(s"funcName is $funcName, mode is $mode, argSize is $argSize, resSize is ${resSize}") - - override def requiredColNum: Int = argSize - override def expectedResColNum: Int = resSize - override def setInputFields(fieldList: List[Field]): Unit = { - aggrFieldList = fieldList - } - - override def doColumnarCodeGen_ext(args: Object): TreeNode = { - val (keyFieldList, inputFieldList, resultType, resultField) = - args.asInstanceOf[(List[Field], List[Field], ArrowType, Field)] - if (keyFieldList.isEmpty != true) { - // if keyList has keys, we need to do groupby by these keys. - var inputFieldNode = - keyFieldList.map({field => TreeBuilder.makeField(field)}).asJava - val encodeArrayFuncName = if (hashCollisionCheck == 1) "encodeArraySafe" else "encodeArray" - val encodeNode = TreeBuilder.makeFunction( - encodeArrayFuncName, - inputFieldNode, - resultType/*this arg won't be used*/) - inputFieldNode = - (encodeNode :: inputFieldList.map({field => TreeBuilder.makeField(field)})).asJava - val groupByFuncNode = TreeBuilder.makeFunction( - "splitArrayListWithAction", - inputFieldNode, - resultType/*this arg won't be used*/) - val inputAggrFieldNode = - List(groupByFuncNode) ::: aggrFieldList.map(field => TreeBuilder.makeField(field)) - val aggregateFuncName = "action_" + funcName - logInfo(s"${aggregateFuncName}(${inputAggrFieldNode})") - TreeBuilder.makeFunction( - aggregateFuncName, - inputAggrFieldNode.asJava, - resultType) - } else { - val inputFieldNode = - aggrFieldList.map(field => TreeBuilder.makeField(field)) - logInfo(s"${funcName}(${inputFieldNode})") - TreeBuilder.makeFunction( - funcName, - inputFieldNode.asJava, - resultType) - } - } -} diff --git a/core/src/main/scala/com/intel/oap/expression/ColumnarAggregation.scala b/core/src/main/scala/com/intel/oap/expression/ColumnarAggregation.scala deleted file mode 100644 index c27b70403..000000000 --- a/core/src/main/scala/com/intel/oap/expression/ColumnarAggregation.scala +++ /dev/null @@ -1,601 +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 com.intel.oap.expression - -import io.netty.buffer.ArrowBuf -import java.util.ArrayList -import java.util.Collections -import java.util.concurrent.TimeUnit._ -import util.control.Breaks._ - -import com.intel.oap.ColumnarPluginConfig -import com.intel.oap.vectorized.ArrowWritableColumnVector -import org.apache.spark.sql.util.ArrowUtils -import com.intel.oap.vectorized.ExpressionEvaluator -import com.intel.oap.vectorized.BatchIterator - -import com.google.common.collect.Lists -import org.apache.hadoop.mapreduce.TaskAttemptID -import org.apache.spark.SparkConf -import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.aggregate._ -import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences -import org.apache.spark.sql.types._ -import org.apache.spark.sql.vectorized._ -import org.apache.spark.sql.execution.aggregate.HashAggregateExec -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.TaskContext - -import org.apache.arrow.gandiva.evaluator._ -import org.apache.arrow.gandiva.exceptions.GandivaException -import org.apache.arrow.gandiva.expression._ -import org.apache.arrow.vector.ValueVector -import org.apache.arrow.vector.ipc.message.ArrowFieldNode -import org.apache.arrow.vector.ipc.message.ArrowRecordBatch -import org.apache.arrow.vector.types.pojo.Schema -import org.apache.arrow.vector.types.pojo.Field -import org.apache.arrow.vector.types.pojo.ArrowType - -import scala.collection.JavaConverters._ -import scala.collection.mutable.ListBuffer -import scala.collection.immutable.List -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.Map - -class ColumnarAggregation( - partIndex: Int, - groupingExpressions: Seq[NamedExpression], - originalInputAttributes: Seq[Attribute], - aggregateExpressions: Seq[AggregateExpression], - aggregateAttributes: Seq[Attribute], - resultExpressions: Seq[NamedExpression], - output: Seq[Attribute], - numInputBatches: SQLMetric, - numOutputBatches: SQLMetric, - numOutputRows: SQLMetric, - aggrTime: SQLMetric, - elapseTime: SQLMetric, - sparkConf: SparkConf) - extends Logging { - // build gandiva projection here. - ColumnarPluginConfig.getConf - var elapseTime_make: Long = 0 - var rowId: Int = 0 - var processedNumRows: Int = 0 - var result_iterator: BatchIterator = _ - val hashCollisionCheck: Int = if (ColumnarPluginConfig.getConf.hashCompare) 1 else 0 - - logInfo( - s"\ngroupingExpressions: $groupingExpressions,\noriginalInputAttributes: $originalInputAttributes,\naggregateExpressions: $aggregateExpressions,\naggregateAttributes: $aggregateAttributes,\nresultExpressions: $resultExpressions, \noutput: $output") - - var resultTotalRows: Int = 0 - val mode = if (aggregateExpressions.size > 0) { - aggregateExpressions(0).mode - } else { - null - } - - //////////////// Project original input to aggregateExpression input ////////////////// - // 1. map original input to grouping input - var groupingExpressionsProjector = ColumnarProjection.create(originalInputAttributes, groupingExpressions) - val groupingOrdinalList = groupingExpressionsProjector.getOrdinalList - val groupingAttributes = groupingExpressionsProjector.output - logInfo(s"groupingAttributes is ${groupingAttributes},\ngroupingOrdinalList is ${groupingOrdinalList}") - - // 2. create grouping native Expression - val groupingFieldList = groupingAttributes.map(attr => { - Field.nullable(s"${attr.name}#${attr.exprId.id}", CodeGeneration.getResultType(attr.dataType)) - }) - val groupingNativeExpression: List[ColumnarAggregateExpressionBase] = groupingFieldList.map(field => { - new ColumnarUniqueAggregateExpression(List(field), hashCollisionCheck).asInstanceOf[ColumnarAggregateExpressionBase] - }) - - // 3. map original input to aggregate input - val aggregateAttributeList : Seq[Attribute] = aggregateAttributes - val beforeAggregateExprListBuffer = ListBuffer[Expression]() - val projectOrdinalListBuffer = ListBuffer[Int]() - val aggregateFieldListBuffer = ListBuffer[Field]() - var aggregateResultAttributesBuffer = getAttrForAggregateExpr(aggregateExpressions) - // we need to remove ordinal used in partial mode expression - val nonPartialProjectOrdinalList = (0 until originalInputAttributes.size).toList.filter(i => !groupingOrdinalList.contains(i)).to[ListBuffer] - aggregateExpressions.zipWithIndex.foreach{case(expr, index) => expr.mode match { - case Partial => { - val internalExpressionList = expr.aggregateFunction.children - val ordinalList = ColumnarProjection.binding(originalInputAttributes, internalExpressionList, index, skipLiteral = true) - ordinalList.foreach{i => { - nonPartialProjectOrdinalList -= i - }} - } - case _ => {} - }} - var non_partial_field_id = 0 - var res_field_id = 0 - val (aggregateNativeExpressions, beforeAggregateProjector) = if (mode == null) { - (List[ColumnarAggregateExpressionBase](), null) - } else { - // we need to filter all Partial mode aggregation - val columnarExprList = aggregateExpressions.toList.zipWithIndex.map{case(expr, index) => expr.mode match { - case Partial => { - val res = new ColumnarAggregateExpression( - expr.aggregateFunction, - expr.mode, - expr.isDistinct, - expr.resultId, - hashCollisionCheck) - val arg_size = res.requiredColNum - val internalExpressionList = expr.aggregateFunction.children - val ordinalList = ColumnarProjection.binding(originalInputAttributes, internalExpressionList, index, skipLiteral = true) - val fieldList = if (arg_size > 0) { - internalExpressionList.map(projectExpr => { - val attr = ConverterUtils.getResultAttrFromExpr(projectExpr, s"res_$index") - if (attr.dataType.isInstanceOf[DecimalType]) - throw new UnsupportedOperationException(s"Decimal type is not supported in ColumnarAggregation.") - Field.nullable(s"${attr.name}#${attr.exprId.id}", CodeGeneration.getResultType(attr.dataType)) - }) - } else { - List[Field]() - } - ordinalList.foreach{i => { - nonPartialProjectOrdinalList -= i - projectOrdinalListBuffer += i - }} - fieldList.foreach{field => { - aggregateFieldListBuffer += field - }} - expr.aggregateFunction.children.foreach(e => beforeAggregateExprListBuffer += e) - res.setInputFields(fieldList.toList) - res - } - case _ => { - val res = new ColumnarAggregateExpression( - expr.aggregateFunction, - expr.mode, - expr.isDistinct, - expr.resultId, - hashCollisionCheck) - val arg_size = res.requiredColNum - val ordinalList = (non_partial_field_id until (non_partial_field_id + arg_size)).map(i => nonPartialProjectOrdinalList(i)) - non_partial_field_id += arg_size - val fieldList = ordinalList.map(i => { - val attr = originalInputAttributes(i) - if (attr.dataType.isInstanceOf[DecimalType]) - throw new UnsupportedOperationException(s"Decimal type is not supported in ColumnarAggregation.") - Field.nullable(s"${attr.name}#${attr.exprId.id}", CodeGeneration.getResultType(attr.dataType)) - }) - ordinalList.foreach{i => { - beforeAggregateExprListBuffer += originalInputAttributes(i) - projectOrdinalListBuffer += i - }} - fieldList.foreach{field => { - aggregateFieldListBuffer += field - }} - res.setInputFields(fieldList.toList) - res - } - }} - val projector = ColumnarProjection.create( - originalInputAttributes, beforeAggregateExprListBuffer.toList, skipLiteral = true, renameResult = true) - (columnarExprList, projector) - } - - // 4. create aggregate native Expression - - val aggregateOrdinalList = projectOrdinalListBuffer.toList - var projectOrdinalList = aggregateOrdinalList.distinct - val aggregateFieldList = aggregateFieldListBuffer.toList - - // 5. create nativeAggregate evaluator - val allNativeExpressions = groupingNativeExpression ::: aggregateNativeExpressions - val allAggregateInputFieldList = groupingFieldList ::: aggregateFieldList - val allAggregateResultAttributes = groupingAttributes ::: aggregateResultAttributesBuffer.toList - val aggregateResultFieldList = allAggregateResultAttributes.map(attr => { - Field.nullable(s"${attr.name}#${attr.exprId.id}", CodeGeneration.getResultType(attr.dataType)) - }) - /* declare dummy resultType and resultField to generate Gandiva expression - * Both won't be actually used.*/ - val resultType = CodeGeneration.getResultType() - val resultField = Field.nullable(s"dummy_res", resultType) - - val expressionTree: List[ExpressionTree] = allNativeExpressions.map(expr => { - val node = - expr.doColumnarCodeGen_ext((groupingFieldList, allAggregateInputFieldList, resultType, resultField)) - TreeBuilder.makeExpression(node, resultField) - }) - val aggregateInputSchema = new Schema(allAggregateInputFieldList.asJava) - val aggregateResultSchema = new Schema(aggregateResultFieldList.asJava) - - var aggregator: ExpressionEvaluator = _ - if (allAggregateInputFieldList.size > 0) { - aggregator = new ExpressionEvaluator() - aggregator.build(aggregateInputSchema, expressionTree.asJava, true) - aggregator.setReturnFields(aggregateResultSchema) - logInfo(s"native aggregate input is $aggregateInputSchema,\noutput is $aggregateResultSchema") - } else { - null - } - // 6. map grouping and aggregate result to FinalResult - var aggregateToResultProjector = ColumnarProjection.create(allAggregateResultAttributes, resultExpressions, skipLiteral = false, renameResult = true) - val aggregateToResultOrdinalList = aggregateToResultProjector.getOrdinalList - val resultAttributes = aggregateToResultProjector.output - val resultArrowSchema = new Schema(resultAttributes.map(attr => { - Field.nullable(s"${attr.name}#${attr.exprId.id}", CodeGeneration.getResultType(attr.dataType)) - }).asJava) - logInfo(s"resultAttributes is ${resultAttributes},\naggregateToResultOrdinalList is ${aggregateToResultOrdinalList}") -////////////////////////////////////////////////////////////////////////////////////////////////// - def close(): Unit = { - if (aggregator != null) { - aggregator.close() - aggregator = null - } - if (result_iterator != null) { - result_iterator.close() - result_iterator = null - } - - elapseTime.merge(aggrTime) - } - - def getAttrForAggregateExpr(aggregateExpressions: Seq[AggregateExpression]): ListBuffer[Attribute] = { - var aggregateAttr = new ListBuffer[Attribute]() - val size = aggregateExpressions.size - var res_index = 0 - for (expIdx <- 0 until size) { - val exp: AggregateExpression = aggregateExpressions(expIdx) - val mode = exp.mode - val aggregateFunc = exp.aggregateFunction - aggregateFunc match { - case Average(_) => mode match { - case Partial => { - val avg = aggregateFunc.asInstanceOf[Average] - val aggBufferAttr = avg.inputAggBufferAttributes - for (index <- 0 until aggBufferAttr.size) { - val attr = ConverterUtils.getAttrFromExpr(aggBufferAttr(index)) - aggregateAttr += attr - } - res_index += 2 - } - case PartialMerge => { - val avg = aggregateFunc.asInstanceOf[Average] - val aggBufferAttr = avg.inputAggBufferAttributes - for (index <- 0 until aggBufferAttr.size) { - val attr = ConverterUtils.getAttrFromExpr(aggBufferAttr(index)) - aggregateAttr += attr - } - res_index += 1 - } - case Final => { - aggregateAttr += aggregateAttributeList(res_index) - res_index += 1 - } - case other => - throw new UnsupportedOperationException(s"not currently supported: $other.") - } - case Sum(_) => mode match { - case Partial | PartialMerge => { - val sum = aggregateFunc.asInstanceOf[Sum] - val aggBufferAttr = sum.inputAggBufferAttributes - val attr = ConverterUtils.getAttrFromExpr(aggBufferAttr(0)) - aggregateAttr += attr - res_index += 1 - } - case Final => { - aggregateAttr += aggregateAttributeList(res_index) - res_index += 1 - } - case other => - throw new UnsupportedOperationException(s"not currently supported: $other.") - } - case Count(_) => mode match { - case Partial | PartialMerge => { - val count = aggregateFunc.asInstanceOf[Count] - val aggBufferAttr = count.inputAggBufferAttributes - val attr = ConverterUtils.getAttrFromExpr(aggBufferAttr(0)) - aggregateAttr += attr - res_index += 1 - } - case Final => { - aggregateAttr += aggregateAttributeList(res_index) - res_index += 1 - } - case other => - throw new UnsupportedOperationException(s"not currently supported: $other.") - } - case Max(_) => mode match { - case Partial | PartialMerge => { - val max = aggregateFunc.asInstanceOf[Max] - val aggBufferAttr = max.inputAggBufferAttributes - val attr = ConverterUtils.getAttrFromExpr(aggBufferAttr(0)) - aggregateAttr += attr - res_index += 1 - } - case Final => { - aggregateAttr += aggregateAttributeList(res_index) - res_index += 1 - } - case other => - throw new UnsupportedOperationException(s"not currently supported: $other.") - } - case Min(_) => mode match { - case Partial | PartialMerge => { - val min = aggregateFunc.asInstanceOf[Min] - val aggBufferAttr = min.inputAggBufferAttributes - val attr = ConverterUtils.getAttrFromExpr(aggBufferAttr(0)) - aggregateAttr += attr - res_index += 1 - } - case Final => { - aggregateAttr += aggregateAttributeList(res_index) - res_index += 1 - } - case other => - throw new UnsupportedOperationException(s"not currently supported: $other.") - } - case StddevSamp(_) => mode match { - case Partial => { - val stddevSamp = aggregateFunc.asInstanceOf[StddevSamp] - val aggBufferAttr = stddevSamp.inputAggBufferAttributes - for (index <- 0 until aggBufferAttr.size) { - val attr = ConverterUtils.getAttrFromExpr(aggBufferAttr(index)) - aggregateAttr += attr - } - res_index += 3 - } - case PartialMerge => { - throw new UnsupportedOperationException("not currently supported: PartialMerge.") - } - case Final => { - aggregateAttr += aggregateAttributeList(res_index) - res_index += 1 - } - case other => - throw new UnsupportedOperationException(s"not currently supported: $other.") - } - case other => - throw new UnsupportedOperationException(s"not currently supported: $other.") - } - } - aggregateAttr - } - - def updateAggregationResult(columnarBatch: ColumnarBatch): Unit = { - val numRows = columnarBatch.numRows - val groupingProjectCols = groupingOrdinalList.map(i => { - columnarBatch.column(i).asInstanceOf[ArrowWritableColumnVector].retain() - columnarBatch.column(i).asInstanceOf[ArrowWritableColumnVector] - }) - val aggregateProjectCols = projectOrdinalList.map(i => { - columnarBatch.column(i).asInstanceOf[ArrowWritableColumnVector].retain() - columnarBatch.column(i).asInstanceOf[ArrowWritableColumnVector] - }) - val aggregateFullCols = aggregateOrdinalList.map(i => { - columnarBatch.column(i).asInstanceOf[ArrowWritableColumnVector].retain() - columnarBatch.column(i).asInstanceOf[ArrowWritableColumnVector] - }) - - val groupingAggregateCols : List[ArrowWritableColumnVector] = if (groupingExpressionsProjector.needEvaluate) { - val res = groupingExpressionsProjector.evaluate(numRows, groupingProjectCols.map(_.getValueVector())) - groupingProjectCols.foreach(_.close()) - res - } else { - groupingProjectCols - } - - val aggregateCols : List[ArrowWritableColumnVector]= if (beforeAggregateProjector != null && beforeAggregateProjector.needEvaluate) { - val res = beforeAggregateProjector.evaluate(numRows, aggregateProjectCols.map(_.getValueVector())) - aggregateProjectCols.foreach(_.close()) - aggregateFullCols.foreach(_.close()) - res - } else { - aggregateProjectCols.foreach(_.close()) - aggregateFullCols - } - - val combinedAggregateCols = groupingAggregateCols ::: aggregateCols - val inputAggrRecordBatch: ArrowRecordBatch = - ConverterUtils.createArrowRecordBatch(numRows, combinedAggregateCols.map(_.getValueVector())) - aggregator.evaluate(inputAggrRecordBatch) - ConverterUtils.releaseArrowRecordBatch(inputAggrRecordBatch) - groupingAggregateCols.foreach(_.close()) - aggregateCols.foreach(_.close()) - } - - def getAggregationResult(resultIter: BatchIterator): ColumnarBatch = { - val resultStructType = ArrowUtils.fromArrowSchema(resultArrowSchema) - if (processedNumRows == 0) { - val resultColumnVectors = - ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray - return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0) - } else if (aggregator == null){ - //TODO: add an special case when this hash aggr is doing countLiteral only - val doCountLiteral: Boolean = expressionTree.map(expr => s"${expr.toProtobuf}").filter(_.contains("countLiteral")).size == 1 - if (doCountLiteral) { - val resultColumnVectors = - ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray - (resultColumnVectors zip resultAttributes).foreach{case(v, attr) => { - val numRows = attr.dataType match { - case t: IntegerType => - processedNumRows.asInstanceOf[Number].intValue - case t: LongType => - processedNumRows.asInstanceOf[Number].longValue - } - v.put(0, numRows) - }} - return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 1) - } else { - val resultColumnVectors = - ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray - return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0) - } - } else { - val finalResultRecordBatch = if (resultIter != null) { - resultIter.next() - } else { - aggregator.finish()(0) - } - - if (finalResultRecordBatch == null) { - val resultColumnVectors = - ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray - return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0) - } - val resultLength = finalResultRecordBatch.getLength - - val aggrExprResultColumnVectorList = ConverterUtils.fromArrowRecordBatch(aggregateResultSchema, finalResultRecordBatch) - ConverterUtils.releaseArrowRecordBatch(finalResultRecordBatch) - val resultInputCols = aggregateToResultOrdinalList.map(i => { - aggrExprResultColumnVectorList(i).asInstanceOf[ArrowWritableColumnVector].retain() - aggrExprResultColumnVectorList(i) - }) - val resultColumnVectorList = if (aggregateToResultProjector.needEvaluate) { - val res = aggregateToResultProjector.evaluate(resultLength, resultInputCols.map(_.getValueVector())) - //for (i <- 0 until resultLength) - // logInfo(s"aggregateToResultProjector, input is ${resultInputCols.map(v => v.getUTF8String(i))}, output is ${res.map(v => v.getUTF8String(i))}") - resultInputCols.foreach(_.close()) - res - } else { - resultInputCols - } - aggrExprResultColumnVectorList.foreach(_.close()) - new ColumnarBatch(resultColumnVectorList.map(v => v.asInstanceOf[ColumnVector]).toArray, resultLength) - } - } - - def createIterator(cbIterator: Iterator[ColumnarBatch]): Iterator[ColumnarBatch] = { - new Iterator[ColumnarBatch] { - var cb: ColumnarBatch = null - var nextCalled = false - var resultColumnarBatch: ColumnarBatch = null - var data_loaded = false - var nextBatch = true - var eval_elapse: Long = 0 - var noNext: Boolean = false - - override def hasNext: Boolean = { - if (noNext) return false - if (nextCalled == false && resultColumnarBatch != null) { - return true - } - if (!nextBatch) { - noNext = true - return false - } - - nextCalled = false - if (data_loaded == false) { - while (cbIterator.hasNext) { - cb = cbIterator.next() - - if (cb.numRows > 0) { - val beforeEval = System.nanoTime() - if (aggregator != null) { - updateAggregationResult(cb) - } - eval_elapse += System.nanoTime() - beforeEval - processedNumRows += cb.numRows - } - numInputBatches += 1 - } - if (processedNumRows == 0) { - data_loaded = true - aggrTime += NANOSECONDS.toMillis(eval_elapse) - nextBatch = false - noNext = true - System.out.println(s"ColumnarHashAggregate input is empty") - return false - } - if (groupingFieldList.size > 0) { - val beforeFinish = System.nanoTime() - result_iterator = aggregator.finishByIterator() - eval_elapse += System.nanoTime() - beforeFinish - } - data_loaded = true - aggrTime += NANOSECONDS.toMillis(eval_elapse) - } - val beforeResultFetch = System.nanoTime() - resultColumnarBatch = getAggregationResult(result_iterator) - aggrTime += NANOSECONDS.toMillis(System.nanoTime() - beforeResultFetch) - if (resultColumnarBatch.numRows == 0) { - resultColumnarBatch.close() - logInfo(s"Aggregation completed, total output ${numOutputRows} rows, ${numOutputBatches} batches") - noNext = true - return false - } - numOutputBatches += 1 - numOutputRows += resultColumnarBatch.numRows - if (result_iterator == null) { - nextBatch = false - } - true - } - - override def next(): ColumnarBatch = { - if (resultColumnarBatch == null) { - throw new UnsupportedOperationException(s"next() called, while there is no next") - } - nextCalled = true - val numCols = resultColumnarBatch.numCols - //logInfo(s"result has ${resultColumnarBatch.numRows}, first row is ${(0 until numCols).map(resultColumnarBatch.column(_).getUTF8String(0))}") - resultColumnarBatch - } - }// iterator - } - -} - -object ColumnarAggregation { - var columnarAggregation: ColumnarAggregation = _ - - def create( - partIndex: Int, - groupingExpressions: Seq[NamedExpression], - originalInputAttributes: Seq[Attribute], - aggregateExpressions: Seq[AggregateExpression], - aggregateAttributes: Seq[Attribute], - resultExpressions: Seq[NamedExpression], - output: Seq[Attribute], - numInputBatches: SQLMetric, - numOutputBatches: SQLMetric, - numOutputRows: SQLMetric, - aggrTime: SQLMetric, - elapseTime: SQLMetric, - sparkConf: SparkConf): ColumnarAggregation = synchronized { - columnarAggregation = new ColumnarAggregation( - partIndex, - groupingExpressions, - originalInputAttributes, - aggregateExpressions, - aggregateAttributes, - resultExpressions, - output, - numInputBatches, - numOutputBatches, - numOutputRows, - aggrTime, - elapseTime, - sparkConf) - columnarAggregation - } - - def close(): Unit = { - if (columnarAggregation != null) { - columnarAggregation.close() - columnarAggregation = null - } - } -} diff --git a/core/src/main/scala/com/intel/oap/expression/ColumnarHashAggregationWithCodegen.scala b/core/src/main/scala/com/intel/oap/expression/ColumnarHashAggregation.scala similarity index 56% rename from core/src/main/scala/com/intel/oap/expression/ColumnarHashAggregationWithCodegen.scala rename to core/src/main/scala/com/intel/oap/expression/ColumnarHashAggregation.scala index 81dd4ac88..8f43983e0 100644 --- a/core/src/main/scala/com/intel/oap/expression/ColumnarHashAggregationWithCodegen.scala +++ b/core/src/main/scala/com/intel/oap/expression/ColumnarHashAggregation.scala @@ -58,279 +58,162 @@ import scala.collection.immutable.List import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.Map -class ColumnarHashAggregationWithCodegen( - aggregator: ExpressionEvaluator, - aggregateAttributeArrowSchema: Schema, - resultArrowSchema: Schema, - aggregateToResultProjector: ColumnarProjection, - aggregateToResultOrdinalList: List[Int], - numInputBatches: SQLMetric, - numOutputBatches: SQLMetric, - numOutputRows: SQLMetric, - aggrTime: SQLMetric, - totalTime: SQLMetric, +class ColumnarHashAggregation( + groupingExpressions: Seq[NamedExpression], + originalInputAttributes: Seq[Attribute], + aggregateExpressions: Seq[AggregateExpression], + aggregateAttributes: Seq[Attribute], + resultExpressions: Seq[NamedExpression], + output: Seq[Attribute], sparkConf: SparkConf) extends Logging { - var processedNumRows: Int = 0 - var resultTotalRows: Int = 0 - var aggregator_iterator: BatchIterator = _ - def updateAggregationResult(columnarBatch: ColumnarBatch): Unit = { - val numRows = columnarBatch.numRows - - val inputAggrRecordBatch = ConverterUtils.createArrowRecordBatch(columnarBatch) - aggregator.evaluate(inputAggrRecordBatch) - ConverterUtils.releaseArrowRecordBatch(inputAggrRecordBatch) - } - def getAggregationResult(resultIter: BatchIterator): ColumnarBatch = { - val resultStructType = ArrowUtils.fromArrowSchema(resultArrowSchema) - if (processedNumRows == 0) { - val resultColumnVectors = - ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray - return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0) - } else { - val finalResultRecordBatch = resultIter.next() - - if (finalResultRecordBatch == null) { - val resultColumnVectors = - ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray - return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0) - } - val resultLength = finalResultRecordBatch.getLength - - val aggrExprResultColumnVectorList = - ConverterUtils.fromArrowRecordBatch(aggregateAttributeArrowSchema, finalResultRecordBatch) - ConverterUtils.releaseArrowRecordBatch(finalResultRecordBatch) - val resultInputCols = aggregateToResultOrdinalList.map(i => { - aggrExprResultColumnVectorList(i).asInstanceOf[ArrowWritableColumnVector].retain() - aggrExprResultColumnVectorList(i) - }) - val resultColumnVectorList = if (aggregateToResultProjector.needEvaluate) { - val res = aggregateToResultProjector.evaluate( - resultLength, - resultInputCols.map(_.getValueVector())) - //for (i <- 0 until resultLength) - // logInfo(s"aggregateToResultProjector, input is ${resultInputCols.map(v => v.getUTF8String(i))}, output is ${res.map(v => v.getUTF8String(i))}") - resultInputCols.foreach(_.close()) - res - } else { - resultInputCols - } - aggrExprResultColumnVectorList.foreach(_.close()) - //logInfo(s"AggregationResult first row is ${resultColumnVectorList.map(v => v.getUTF8String(0))}") - new ColumnarBatch( - resultColumnVectorList.map(v => v.asInstanceOf[ColumnVector]).toArray, - resultLength) - } - } - - def close(): Unit = { - if (aggregator != null) { - aggregator.close() - } - if (aggregator_iterator != null) { - aggregator_iterator.close() - aggregator_iterator = null - } - totalTime.merge(aggrTime) - } - - def createIterator(cbIterator: Iterator[ColumnarBatch]): Iterator[ColumnarBatch] = { - new Iterator[ColumnarBatch] { - var cb: ColumnarBatch = null - var nextCalled = false - var resultColumnarBatch: ColumnarBatch = null - var data_loaded = false - var nextBatch = true - var eval_elapse: Long = 0 - - override def hasNext: Boolean = { - if (nextCalled == false && resultColumnarBatch != null) { - return true - } - if (!nextBatch) { - return false - } - - nextCalled = false - if (data_loaded == false) { - while (cbIterator.hasNext) { - cb = cbIterator.next() - - if (cb.numRows > 0) { - val beforeEval = System.nanoTime() - updateAggregationResult(cb) - eval_elapse += System.nanoTime() - beforeEval - processedNumRows += cb.numRows - } - numInputBatches += 1 - } - val beforeFinish = System.nanoTime() - aggregator_iterator = aggregator.finishByIterator() - eval_elapse += System.nanoTime() - beforeFinish - data_loaded = true - aggrTime += NANOSECONDS.toMillis(eval_elapse) - } - val beforeResultFetch = System.nanoTime() - resultColumnarBatch = getAggregationResult(aggregator_iterator) - aggrTime += NANOSECONDS.toMillis(System.nanoTime() - beforeResultFetch) - if (resultColumnarBatch.numRows == 0) { - resultColumnarBatch.close() - logInfo( - s"Aggregation completed, total output ${numOutputRows} rows, ${numOutputBatches} batches") - return false - } - numOutputBatches += 1 - numOutputRows += resultColumnarBatch.numRows - true - } - - override def next(): ColumnarBatch = { - if (resultColumnarBatch == null) { - throw new UnsupportedOperationException(s"next() called, while there is no next") - } - nextCalled = true - val numCols = resultColumnarBatch.numCols - //logInfo(s"result has ${resultColumnarBatch.numRows}, first row is ${(0 until numCols).map(resultColumnarBatch.column(_).getUTF8String(0))}") - resultColumnarBatch - } - } // iterator - } - -} - -object ColumnarHashAggregationWithCodegen extends Logging { var inputAttrQueue: scala.collection.mutable.Queue[Attribute] = _ val resultType = CodeGeneration.getResultType() def getColumnarFuncNode(expr: Expression): TreeNode = { - if (expr.isInstanceOf[AttributeReference] && expr - .asInstanceOf[AttributeReference] - .name == "none") { - throw new UnsupportedOperationException( - s"Unsupport to generate native expression from replaceable expression.") + try { + if (expr.isInstanceOf[AttributeReference] && expr + .asInstanceOf[AttributeReference] + .name == "none") { + throw new UnsupportedOperationException( + s"Unsupport to generate native expression from replaceable expression.") + } + var columnarExpr: Expression = + ColumnarExpressionConverter.replaceWithColumnarExpression(expr) + if (columnarExpr.dataType.isInstanceOf[DecimalType]) + throw new UnsupportedOperationException( + s"Decimal type is not supported in ColumnarHashAggregation.") + var inputList: java.util.List[Field] = Lists.newArrayList() + val (node, _resultType) = + columnarExpr.asInstanceOf[ColumnarExpression].doColumnarCodeGen(inputList) + node + } catch { + case e: Throwable => + val errmsg = e.getStackTrace.mkString("\n") + logError(s"getColumnarFuncNode failed, expr is ${expr}, forward throw this exception") + throw e } - var columnarExpr: Expression = - ColumnarExpressionConverter.replaceWithColumnarExpression(expr) - if (columnarExpr.dataType.isInstanceOf[DecimalType]) - throw new UnsupportedOperationException( - s"Decimal type is not supported in ColumnarHashAggregationWithCodegen.") - var inputList: java.util.List[Field] = Lists.newArrayList() - val (node, _resultType) = - columnarExpr.asInstanceOf[ColumnarExpression].doColumnarCodeGen(inputList) - node } def getColumnarFuncNode(aggregateExpression: AggregateExpression): TreeNode = { val aggregateFunc = aggregateExpression.aggregateFunction val mode = aggregateExpression.mode - aggregateFunc match { - case Average(_) => - mode match { - case Partial => - val childrenColumnarFuncNodeList = - aggregateFunc.children.toList.map(expr => getColumnarFuncNode(expr)) - TreeBuilder - .makeFunction("action_sum_count", childrenColumnarFuncNodeList.asJava, resultType) - case PartialMerge => - val childrenColumnarFuncNodeList = - List(inputAttrQueue.dequeue, inputAttrQueue.dequeue).map(attr => - getColumnarFuncNode(attr)) - TreeBuilder - .makeFunction( - "action_sum_count_merge", - childrenColumnarFuncNodeList.asJava, - resultType) - case Final => - val childrenColumnarFuncNodeList = - List(inputAttrQueue.dequeue, inputAttrQueue.dequeue).map(attr => - getColumnarFuncNode(attr)) - TreeBuilder.makeFunction( - "action_avgByCount", - childrenColumnarFuncNodeList.asJava, - resultType) - case other => - throw new UnsupportedOperationException(s"not currently supported: $other.") - } - case Sum(_) => - val childrenColumnarFuncNodeList = + try { + aggregateFunc match { + case Average(_) => mode match { case Partial => - aggregateFunc.children.toList.map(expr => getColumnarFuncNode(expr)) - case Final | PartialMerge => - List(inputAttrQueue.dequeue).map(attr => getColumnarFuncNode(attr)) + val childrenColumnarFuncNodeList = + aggregateFunc.children.toList.map(expr => getColumnarFuncNode(expr)) + TreeBuilder + .makeFunction("action_sum_count", childrenColumnarFuncNodeList.asJava, resultType) + case PartialMerge => + val childrenColumnarFuncNodeList = + List(inputAttrQueue.dequeue, inputAttrQueue.dequeue).map(attr => + getColumnarFuncNode(attr)) + TreeBuilder + .makeFunction( + "action_sum_count_merge", + childrenColumnarFuncNodeList.asJava, + resultType) + case Final => + val childrenColumnarFuncNodeList = + List(inputAttrQueue.dequeue, inputAttrQueue.dequeue).map(attr => + getColumnarFuncNode(attr)) + TreeBuilder.makeFunction( + "action_avgByCount", + childrenColumnarFuncNodeList.asJava, + resultType) case other => throw new UnsupportedOperationException(s"not currently supported: $other.") } - TreeBuilder.makeFunction("action_sum", childrenColumnarFuncNodeList.asJava, resultType) - case Count(_) => - mode match { - case Partial => - val childrenColumnarFuncNodeList = - aggregateFunc.children.toList.map(expr => getColumnarFuncNode(expr)) - if (aggregateFunc.children(0).isInstanceOf[Literal]) { - TreeBuilder.makeFunction( - s"action_countLiteral_${aggregateFunc.children(0)}", - Lists.newArrayList(), - resultType) - } else { - TreeBuilder - .makeFunction("action_count", childrenColumnarFuncNodeList.asJava, resultType) + case Sum(_) => + val childrenColumnarFuncNodeList = + mode match { + case Partial => + aggregateFunc.children.toList.map(expr => getColumnarFuncNode(expr)) + case Final | PartialMerge => + List(inputAttrQueue.dequeue).map(attr => getColumnarFuncNode(attr)) + case other => + throw new UnsupportedOperationException(s"not currently supported: $other.") } - case Final | PartialMerge => - val childrenColumnarFuncNodeList = - List(inputAttrQueue.dequeue).map(attr => getColumnarFuncNode(attr)) - TreeBuilder - .makeFunction("action_sum", childrenColumnarFuncNodeList.asJava, resultType) - case other => - throw new UnsupportedOperationException(s"not currently supported: $other.") - } - case Max(_) => - val childrenColumnarFuncNodeList = + TreeBuilder.makeFunction("action_sum", childrenColumnarFuncNodeList.asJava, resultType) + case Count(_) => mode match { case Partial => - aggregateFunc.children.toList.map(expr => getColumnarFuncNode(expr)) + val childrenColumnarFuncNodeList = + aggregateFunc.children.toList.map(expr => getColumnarFuncNode(expr)) + if (aggregateFunc.children(0).isInstanceOf[Literal]) { + TreeBuilder.makeFunction( + s"action_countLiteral_${aggregateFunc.children(0)}", + Lists.newArrayList(), + resultType) + } else { + TreeBuilder + .makeFunction("action_count", childrenColumnarFuncNodeList.asJava, resultType) + } case Final | PartialMerge => - List(inputAttrQueue.dequeue).map(attr => getColumnarFuncNode(attr)) + val childrenColumnarFuncNodeList = + List(inputAttrQueue.dequeue).map(attr => getColumnarFuncNode(attr)) + TreeBuilder + .makeFunction("action_sum", childrenColumnarFuncNodeList.asJava, resultType) case other => throw new UnsupportedOperationException(s"not currently supported: $other.") } - TreeBuilder.makeFunction("action_max", childrenColumnarFuncNodeList.asJava, resultType) - case Min(_) => - val childrenColumnarFuncNodeList = + case Max(_) => + val childrenColumnarFuncNodeList = + mode match { + case Partial => + aggregateFunc.children.toList.map(expr => getColumnarFuncNode(expr)) + case Final | PartialMerge => + List(inputAttrQueue.dequeue).map(attr => getColumnarFuncNode(attr)) + case other => + throw new UnsupportedOperationException(s"not currently supported: $other.") + } + TreeBuilder.makeFunction("action_max", childrenColumnarFuncNodeList.asJava, resultType) + case Min(_) => + val childrenColumnarFuncNodeList = + mode match { + case Partial => + aggregateFunc.children.toList.map(expr => getColumnarFuncNode(expr)) + case Final | PartialMerge => + List(inputAttrQueue.dequeue).map(attr => getColumnarFuncNode(attr)) + case other => + throw new UnsupportedOperationException(s"not currently supported: $other.") + } + TreeBuilder.makeFunction("action_min", childrenColumnarFuncNodeList.asJava, resultType) + case StddevSamp(_) => mode match { case Partial => - aggregateFunc.children.toList.map(expr => getColumnarFuncNode(expr)) - case Final | PartialMerge => - List(inputAttrQueue.dequeue).map(attr => getColumnarFuncNode(attr)) + val childrenColumnarFuncNodeList = + aggregateFunc.children.toList.map(expr => getColumnarFuncNode(expr)) + TreeBuilder.makeFunction( + "action_stddev_samp_partial", + childrenColumnarFuncNodeList.asJava, + resultType) + case PartialMerge => + throw new UnsupportedOperationException("not currently supported: PartialMerge.") + case Final => + val childrenColumnarFuncNodeList = + List(inputAttrQueue.dequeue, inputAttrQueue.dequeue, inputAttrQueue.dequeue) + .map(attr => getColumnarFuncNode(attr)) + TreeBuilder.makeFunction( + "action_stddev_samp_final", + childrenColumnarFuncNodeList.asJava, + resultType) case other => throw new UnsupportedOperationException(s"not currently supported: $other.") } - TreeBuilder.makeFunction("action_min", childrenColumnarFuncNodeList.asJava, resultType) - case StddevSamp(_) => - mode match { - case Partial => - val childrenColumnarFuncNodeList = - aggregateFunc.children.toList.map(expr => getColumnarFuncNode(expr)) - TreeBuilder.makeFunction( - "action_stddev_samp_partial", - childrenColumnarFuncNodeList.asJava, - resultType) - case PartialMerge => - throw new UnsupportedOperationException("not currently supported: PartialMerge.") - case Final => - val childrenColumnarFuncNodeList = - List(inputAttrQueue.dequeue, inputAttrQueue.dequeue, inputAttrQueue.dequeue) - .map(attr => getColumnarFuncNode(attr)) - TreeBuilder.makeFunction( - "action_stddev_samp_final", - childrenColumnarFuncNodeList.asJava, - resultType) - case other => - throw new UnsupportedOperationException(s"not currently supported: $other.") - } - case other => - throw new UnsupportedOperationException(s"not currently supported: $other.") + case other => + throw new UnsupportedOperationException(s"not currently supported: $other.") + } + } catch { + case e: Throwable => + val errmsg = e.getStackTrace.mkString("\n") + logError( + s"getColumnarFuncNode failed, expr is ${aggregateExpression}, forward throw this exception") + throw e } } @@ -464,14 +347,7 @@ object ColumnarHashAggregationWithCodegen extends Logging { aggregateAttr.toList } - def prepareKernelFunction( - groupingExpressions: Seq[NamedExpression], - originalInputAttributes: Seq[Attribute], - aggregateExpressions: Seq[AggregateExpression], - aggregateAttributes: Seq[Attribute], - resultExpressions: Seq[NamedExpression], - output: Seq[Attribute], - sparkConf: SparkConf): TreeNode = { + def prepareKernelFunction: TreeNode = { // build gandiva projection here. ColumnarPluginConfig.getConf @@ -484,7 +360,7 @@ object ColumnarHashAggregationWithCodegen extends Logging { val originalInputFieldList = originalInputAttributes.toList.map(attr => { if (attr.dataType.isInstanceOf[DecimalType]) throw new UnsupportedOperationException( - s"Decimal type is not supported in ColumnarHashAggregationWithCodegen.") + s"Decimal type is not supported in ColumnarHashAggregation.") Field .nullable(s"${attr.name}#${attr.exprId.id}", CodeGeneration.getResultType(attr.dataType)) }) @@ -586,4 +462,28 @@ object ColumnarHashAggregationWithCodegen extends Logging { resultType /*dummy ret type, won't be used*/ ) } + +} + +object ColumnarHashAggregation extends Logging { + + def prepareKernelFunction( + groupingExpressions: Seq[NamedExpression], + originalInputAttributes: Seq[Attribute], + aggregateExpressions: Seq[AggregateExpression], + aggregateAttributes: Seq[Attribute], + resultExpressions: Seq[NamedExpression], + output: Seq[Attribute], + sparkConf: SparkConf): TreeNode = { + val ins = new ColumnarHashAggregation( + groupingExpressions, + originalInputAttributes, + aggregateExpressions, + aggregateAttributes, + resultExpressions, + output, + sparkConf) + ins.prepareKernelFunction + + } } diff --git a/cpp/src/benchmarks/arrow_compute_benchmark.cc b/cpp/src/benchmarks/arrow_compute_benchmark.cc deleted file mode 100644 index d2f3ea5b0..000000000 --- a/cpp/src/benchmarks/arrow_compute_benchmark.cc +++ /dev/null @@ -1,259 +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. - */ - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -#include "codegen/code_generator.h" -#include "codegen/code_generator_factory.h" -#include "codegen/common/result_iterator.h" -#include "tests/test_utils.h" - -namespace sparkcolumnarplugin { -namespace codegen { - -class BenchmarkArrowCompute : public ::testing::Test { - public: - void SetUp() override { - // read input from parquet file -#ifdef BENCHMARK_FILE_PATH - std::string dir_path = BENCHMARK_FILE_PATH; -#else - std::string dir_path = ""; -#endif - std::string path = dir_path + "tpcds_websales_sort_big.parquet"; - std::cout << "This Benchmark used file " << path - << ", please download from server " - "vsr200://home/zhouyuan/sparkColumnarPlugin/source_files" - << std::endl; - std::shared_ptr fs; - std::string file_name; - ASSERT_OK_AND_ASSIGN(fs, arrow::fs::FileSystemFromUri(path, &file_name)); - - ARROW_ASSIGN_OR_THROW(file, fs->OpenInputFile(file_name)); - - parquet::ArrowReaderProperties properties(true); - properties.set_batch_size(4096); - - auto pool = arrow::default_memory_pool(); - ASSERT_NOT_OK(::parquet::arrow::FileReader::Make( - pool, ::parquet::ParquetFileReader::Open(file), properties, &parquet_reader)); - - ASSERT_NOT_OK( - parquet_reader->GetRecordBatchReader({0}, {0, 1, 2}, &record_batch_reader)); - - schema = record_batch_reader->schema(); - - ////////////////// expr prepration //////////////// - field_list = schema->fields(); - } - - void Start() { - std::shared_ptr expr; - std::vector> result_batch; - std::shared_ptr record_batch; - uint64_t elapse_gen = 0; - uint64_t elapse_read = 0; - uint64_t elapse_eval = 0; - uint64_t num_batches = 0; - - TIME_MICRO_OR_THROW(elapse_gen, CreateCodeGenerator(schema, expr_vector, - ret_field_list, &expr, true)); - - do { - TIME_MICRO_OR_THROW(elapse_read, record_batch_reader->ReadNext(&record_batch)); - if (record_batch) { - TIME_MICRO_OR_THROW(elapse_eval, expr->evaluate(record_batch, &result_batch)); - num_batches += 1; - } - } while (record_batch); - std::cout << "Readed " << num_batches << " batches." << std::endl; - - TIME_MICRO_OR_THROW(elapse_eval, expr->finish(&result_batch)); - /*auto batch = result_batch[result_batch.size() - 1]; - arrow::PrettyPrint(*batch.get(), 2, &std::cout);*/ - - std::cout << "BenchmarkArrowComputeBigScale processed " << num_batches - << " batches,\n output " << result_batch.size() << " batches,\n took " - << TIME_TO_STRING(elapse_gen) << " doing codegen,\n took " - << TIME_TO_STRING(elapse_read) << " doing BatchRead,\n took " - << TIME_TO_STRING(elapse_eval) << " doing Batch Evaluation." << std::endl; - } - - void StartWithIterator() { - std::shared_ptr expr; - std::vector> result_batch; - std::shared_ptr record_batch; - std::shared_ptr out; - uint64_t elapse_gen = 0; - uint64_t elapse_read = 0; - uint64_t elapse_eval = 0; - uint64_t num_batches = 0; - uint64_t num_rows = 0; - - TIME_MICRO_OR_THROW(elapse_gen, CreateCodeGenerator(schema, expr_vector, - ret_field_list, &expr, true)); - - do { - TIME_MICRO_OR_THROW(elapse_read, record_batch_reader->ReadNext(&record_batch)); - if (record_batch) { - TIME_MICRO_OR_THROW(elapse_eval, expr->evaluate(record_batch, &result_batch)); - num_batches += 1; - } - } while (record_batch); - std::cout << "Readed " << num_batches << " batches." << std::endl; - - std::shared_ptr> it; - uint64_t num_output_batches = 0; - TIME_MICRO_OR_THROW(elapse_eval, expr->finish(&it)); - while (it->HasNext()) { - TIME_MICRO_OR_THROW(elapse_eval, it->Next(&out)); - num_output_batches++; - num_rows += out->num_rows(); - } - - std::cout << "BenchmarkArrowCompute processed " << num_batches - << " batches, then output " << num_output_batches << " batches with " - << num_rows << " rows, to complete, it took " << TIME_TO_STRING(elapse_gen) - << " doing codegen, took " << TIME_TO_STRING(elapse_read) - << " doing BatchRead, took " << TIME_TO_STRING(elapse_eval) - << " doing Batch Evaluation." << std::endl; - } - - protected: - std::shared_ptr file; - std::unique_ptr<::parquet::arrow::FileReader> parquet_reader; - std::shared_ptr record_batch_reader; - std::shared_ptr schema; - - std::vector> field_list; - std::vector> expr_vector; - std::vector> ret_field_list; -}; - -TEST_F(BenchmarkArrowCompute, AggregateBenchmark) { - for (auto field : field_list) { - auto n_sum = TreeExprBuilder::MakeFunction("sum", {TreeExprBuilder::MakeField(field)}, - field->type()); - auto sum_expr = TreeExprBuilder::MakeExpression(n_sum, field); - expr_vector.push_back(sum_expr); - ret_field_list.push_back(field); - } - - ///////////////////// Calculation ////////////////// - Start(); -} - -TEST_F(BenchmarkArrowCompute, GroupByAggregateBenchmark) { - // prepare expression - std::vector> field_node_list; - for (auto field : field_list) { - field_node_list.push_back(TreeExprBuilder::MakeField(field)); - } - - auto n_encode = TreeExprBuilder::MakeFunction( - "encodeArray", {TreeExprBuilder::MakeField(field_list[0])}, field_list[0]->type()); - - std::vector> arg_for_aggr = {n_encode}; - arg_for_aggr.insert(arg_for_aggr.end(), field_node_list.begin(), field_node_list.end()); - - auto n_aggr = TreeExprBuilder::MakeFunction("splitArrayListWithAction", arg_for_aggr, - uint32() /*won't be used*/); - - for (auto field : field_list) { - auto action = TreeExprBuilder::MakeFunction( - "action_sum", {n_aggr, TreeExprBuilder::MakeField(field)}, field->type()); - auto aggr_expr = TreeExprBuilder::MakeExpression(action, field); - expr_vector.push_back(aggr_expr); - ret_field_list.push_back(field); - } - - ///////////////////// Calculation ////////////////// - Start(); -} - -TEST_F(BenchmarkArrowCompute, GroupByWithTwoAggregateBenchmark) { - // prepare expression - std::vector> field_node_list; - for (auto field : field_list) { - field_node_list.push_back(TreeExprBuilder::MakeField(field)); - } - - auto n_encode = - TreeExprBuilder::MakeFunction("encodeArray", - {TreeExprBuilder::MakeField(field_list[0]), - TreeExprBuilder::MakeField(field_list[1])}, - uint32()); - - std::vector> arg_for_aggr = {n_encode}; - arg_for_aggr.insert(arg_for_aggr.end(), field_node_list.begin(), field_node_list.end()); - - auto n_aggr = TreeExprBuilder::MakeFunction("splitArrayListWithAction", arg_for_aggr, - uint32() /*won't be used*/); - - for (auto field : field_list) { - auto action = TreeExprBuilder::MakeFunction( - "action_sum", {n_aggr, TreeExprBuilder::MakeField(field)}, field->type()); - auto aggr_expr = TreeExprBuilder::MakeExpression(action, field); - expr_vector.push_back(aggr_expr); - ret_field_list.push_back(field); - } - - ///////////////////// Calculation ////////////////// - Start(); -} - -/*TEST_F(BenchmarkArrowCompute, SortBenchmark) { - // prepare expression - std::vector> field_node_list; - for (auto field : field_list) { - field_node_list.push_back(TreeExprBuilder::MakeField(field)); - } - - auto n_sort_to_indices = TreeExprBuilder::MakeFunction( - "sortArraysToIndicesNullsFirstAsc", {TreeExprBuilder::MakeField(field_list[0])}, - field_list[0]->type()); - std::vector> arg_for_shuffle = {n_sort_to_indices}; - arg_for_shuffle.insert(arg_for_shuffle.end(), field_node_list.begin(), - field_node_list.end()); - auto n_sort = - TreeExprBuilder::MakeFunction("shuffleArrayList", arg_for_shuffle, uint32()); - for (auto field : field_list) { - auto action = TreeExprBuilder::MakeFunction( - "action_dono", {n_sort, TreeExprBuilder::MakeField(field)}, field->type()); - auto sort_expr = TreeExprBuilder::MakeExpression(action, field); - expr_vector.push_back(sort_expr); - ret_field_list.push_back(field); - } - - ///////////////////// Calculation ////////////////// - StartWithIterator(); -}*/ - -} // namespace codegen -} // namespace sparkcolumnarplugin diff --git a/cpp/src/benchmarks/arrow_compute_benchmark_big_scale.cc b/cpp/src/benchmarks/arrow_compute_benchmark_big_scale.cc deleted file mode 100644 index e7cc65489..000000000 --- a/cpp/src/benchmarks/arrow_compute_benchmark_big_scale.cc +++ /dev/null @@ -1,251 +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. - */ - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include "codegen/code_generator.h" -#include "codegen/code_generator_factory.h" -#include "tests/test_utils.h" - -namespace sparkcolumnarplugin { -namespace codegen { - -class BenchmarkArrowComputeBigScale : public ::testing::Test { - public: - void SetUp() override { - // read input from parquet file -#ifdef BENCHMARK_FILE_PATH - std::string dir_path = BENCHMARK_FILE_PATH; -#else - std::string dir_path = ""; -#endif - std::string path = dir_path + "tpcds_websales_sort_big.parquet"; - std::cout << "This Benchmark used file " << path - << ", please download from server " - "vsr200://home/zhouyuan/sparkColumnarPlugin/source_files" - << std::endl; - std::shared_ptr fs; - std::string file_name; - ASSERT_OK_AND_ASSIGN(fs, arrow::fs::FileSystemFromUri(path, &file_name)); - - ARROW_ASSIGN_OR_THROW(file, fs->OpenInputFile(file_name)); - - parquet::ArrowReaderProperties properties(true); - properties.set_batch_size(4096); - - auto pool = arrow::default_memory_pool(); - ASSERT_NOT_OK(::parquet::arrow::FileReader::Make( - pool, ::parquet::ParquetFileReader::Open(file), properties, &parquet_reader)); - - ASSERT_NOT_OK( - parquet_reader->GetRecordBatchReader({0}, {0, 1, 2}, &record_batch_reader)); - - schema = record_batch_reader->schema(); - - ////////////////// expr prepration //////////////// - field_list = schema->fields(); - } - - void Start() { - std::shared_ptr expr; - std::vector> result_batch; - std::shared_ptr record_batch; - uint64_t elapse_gen = 0; - uint64_t elapse_read = 0; - uint64_t elapse_eval = 0; - uint64_t num_batches = 0; - - TIME_MICRO_OR_THROW(elapse_gen, CreateCodeGenerator(schema, expr_vector, - ret_field_list, &expr, true)); - - do { - TIME_MICRO_OR_THROW(elapse_read, record_batch_reader->ReadNext(&record_batch)); - if (record_batch) { - TIME_MICRO_OR_THROW(elapse_eval, expr->evaluate(record_batch, &result_batch)); - num_batches += 1; - } - } while (record_batch); - std::cout << "Readed " << num_batches << " batches." << std::endl; - - TIME_MICRO_OR_THROW(elapse_eval, expr->finish(&result_batch)); - - std::cout << "BenchmarkArrowComputeBigScale processed " << num_batches - << " batches, took " << TIME_TO_STRING(elapse_gen) - << " doing codegen, took " << TIME_TO_STRING(elapse_read) - << " doing BatchRead, took " << TIME_TO_STRING(elapse_eval) - << " doing Batch Evaluation." << std::endl; - } - - void StartWithIterator() { - std::shared_ptr expr; - std::vector> result_batch; - std::shared_ptr record_batch; - std::shared_ptr out; - uint64_t elapse_gen = 0; - uint64_t elapse_read = 0; - uint64_t elapse_eval = 0; - uint64_t num_batches = 0; - - TIME_MICRO_OR_THROW(elapse_gen, CreateCodeGenerator(schema, expr_vector, - ret_field_list, &expr, true)); - - do { - TIME_MICRO_OR_THROW(elapse_read, record_batch_reader->ReadNext(&record_batch)); - if (record_batch) { - TIME_MICRO_OR_THROW(elapse_eval, expr->evaluate(record_batch, &result_batch)); - num_batches += 1; - } - } while (record_batch); - std::cout << "Readed " << num_batches << " batches." << std::endl; - - std::shared_ptr> it; - uint64_t num_output_batches = 0; - TIME_MICRO_OR_THROW(elapse_eval, expr->finish(&it)); - while (it->HasNext()) { - TIME_MICRO_OR_THROW(elapse_eval, it->Next(&out)); - num_output_batches++; - } - - std::cout << "BenchmarkArrowCompute processed " << num_batches - << " batches, then output " << num_output_batches - << " batches, to complete, it took " << TIME_TO_STRING(elapse_gen) - << " doing codegen, took " << TIME_TO_STRING(elapse_read) - << " doing BatchRead, took " << TIME_TO_STRING(elapse_eval) - << " doing Batch Evaluation." << std::endl; - } - - protected: - std::shared_ptr file; - std::unique_ptr<::parquet::arrow::FileReader> parquet_reader; - std::shared_ptr record_batch_reader; - std::shared_ptr schema; - - std::vector> field_list; - std::vector> expr_vector; - std::vector> ret_field_list; -}; - -TEST_F(BenchmarkArrowComputeBigScale, AggregateBenchmark) { - for (auto field : field_list) { - auto n_sum = TreeExprBuilder::MakeFunction("sum", {TreeExprBuilder::MakeField(field)}, - field->type()); - auto sum_expr = TreeExprBuilder::MakeExpression(n_sum, field); - expr_vector.push_back(sum_expr); - ret_field_list.push_back(field); - } - - ///////////////////// Calculation ////////////////// - Start(); -} - -TEST_F(BenchmarkArrowComputeBigScale, GroupByAggregateBenchmark) { - // prepare expression - std::vector> field_node_list; - for (auto field : field_list) { - field_node_list.push_back(TreeExprBuilder::MakeField(field)); - } - - auto n_encode = TreeExprBuilder::MakeFunction( - "encodeArray", {TreeExprBuilder::MakeField(field_list[0])}, field_list[0]->type()); - - std::vector> arg_for_aggr = {n_encode}; - arg_for_aggr.insert(arg_for_aggr.end(), field_node_list.begin(), field_node_list.end()); - - auto n_aggr = TreeExprBuilder::MakeFunction("splitArrayListWithAction", arg_for_aggr, - uint32() /*won't be used*/); - - for (auto field : field_list) { - auto action = TreeExprBuilder::MakeFunction( - "action_sum", {n_aggr, TreeExprBuilder::MakeField(field)}, field->type()); - auto aggr_expr = TreeExprBuilder::MakeExpression(action, field); - expr_vector.push_back(aggr_expr); - ret_field_list.push_back(field); - } - - ///////////////////// Calculation ////////////////// - Start(); -} - -TEST_F(BenchmarkArrowComputeBigScale, GroupByWithTwoAggregateBenchmark) { - // prepare expression - std::vector> field_node_list; - for (auto field : field_list) { - field_node_list.push_back(TreeExprBuilder::MakeField(field)); - } - - auto n_encode = - TreeExprBuilder::MakeFunction("encodeArray", - {TreeExprBuilder::MakeField(field_list[0]), - TreeExprBuilder::MakeField(field_list[1])}, - uint32()); - - std::vector> arg_for_aggr = {n_encode}; - arg_for_aggr.insert(arg_for_aggr.end(), field_node_list.begin(), field_node_list.end()); - - auto n_aggr = TreeExprBuilder::MakeFunction("splitArrayListWithAction", arg_for_aggr, - uint32() /*won't be used*/); - - for (auto field : field_list) { - auto action = TreeExprBuilder::MakeFunction( - "action_sum", {n_aggr, TreeExprBuilder::MakeField(field)}, field->type()); - auto aggr_expr = TreeExprBuilder::MakeExpression(action, field); - expr_vector.push_back(aggr_expr); - ret_field_list.push_back(field); - } - - ///////////////////// Calculation ////////////////// - Start(); -} - -/*TEST_F(BenchmarkArrowComputeBigScale, SortBenchmark) { - // prepare expression - std::vector> field_node_list; - for (auto field : field_list) { - field_node_list.push_back(TreeExprBuilder::MakeField(field)); - } - - auto n_sort_to_indices = TreeExprBuilder::MakeFunction( - "sortArraysToIndicesNullsFirstAsc", {TreeExprBuilder::MakeField(field_list[0])}, - field_list[0]->type()); - std::vector> arg_for_shuffle = {n_sort_to_indices}; - arg_for_shuffle.insert(arg_for_shuffle.end(), field_node_list.begin(), - field_node_list.end()); - auto n_sort = - TreeExprBuilder::MakeFunction("shuffleArrayList", arg_for_shuffle, uint32()); - for (auto field : field_list) { - auto action = TreeExprBuilder::MakeFunction( - "action_dono", {n_sort, TreeExprBuilder::MakeField(field)}, field->type()); - auto sort_expr = TreeExprBuilder::MakeExpression(action, field); - expr_vector.push_back(sort_expr); - ret_field_list.push_back(field); - } - - ///////////////////// Calculation ////////////////// - StartWithIterator(); -}*/ - -} // namespace codegen -} // namespace sparkcolumnarplugin diff --git a/cpp/src/codegen/arrow_compute/expr_visitor.cc b/cpp/src/codegen/arrow_compute/expr_visitor.cc index 03aa9407b..605d74500 100644 --- a/cpp/src/codegen/arrow_compute/expr_visitor.cc +++ b/cpp/src/codegen/arrow_compute/expr_visitor.cc @@ -348,6 +348,9 @@ arrow::Status ExprVisitor::MakeExprVisitorImpl( } else if (child_func_name.compare("ConcatArrayList") == 0) { RETURN_NOT_OK( ConcatArrayListVisitorImpl::Make(field_list, func_node, ret_fields, p, &impl_)); + } else if (child_func_name.compare("hashAggregateArrays") == 0) { + RETURN_NOT_OK( + HashAggregateArraysImpl::Make(field_list, func_node, ret_fields, p, &impl_)); } goto finish; } @@ -463,20 +466,6 @@ arrow::Status ExprVisitor::MakeExprVisitorImpl( arrow::Status ExprVisitor::MakeExprVisitorImpl(const std::string& func_name, ExprVisitor* p) { - if (func_name.compare("splitArrayListWithAction") == 0) { - RETURN_NOT_OK(SplitArrayListWithActionVisitorImpl::Make(p, &impl_)); - goto finish; - } - if (func_name.compare("sum") == 0 || func_name.compare("count") == 0 || - func_name.compare("unique") == 0 || func_name.compare("append") == 0 || - func_name.compare("sum_count") == 0 || func_name.compare("avgByCount") == 0 || - func_name.compare("min") == 0 || func_name.compare("max") == 0 || - func_name.compare("stddev_samp_partial") == 0 || - func_name.compare("stddev_samp_final") == 0 || - func_name.compare("sum_count_merge") == 0) { - RETURN_NOT_OK(AggregateVisitorImpl::Make(p, func_name, &impl_)); - goto finish; - } if (func_name.compare("encodeArray") == 0) { RETURN_NOT_OK(EncodeVisitorImpl::Make(p, 0, &impl_)); goto finish; diff --git a/cpp/src/codegen/arrow_compute/expr_visitor_impl.h b/cpp/src/codegen/arrow_compute/expr_visitor_impl.h index 74f63acdb..fd2d30a36 100644 --- a/cpp/src/codegen/arrow_compute/expr_visitor_impl.h +++ b/cpp/src/codegen/arrow_compute/expr_visitor_impl.h @@ -100,225 +100,6 @@ class ExprVisitorImpl { } }; -//////////////////////// SplitArrayListWithActionVisitorImpl ////////////////////// -class SplitArrayListWithActionVisitorImpl : public ExprVisitorImpl { - public: - SplitArrayListWithActionVisitorImpl(ExprVisitor* p) : ExprVisitorImpl(p) {} - static arrow::Status Make(ExprVisitor* p, std::shared_ptr* out) { - auto impl = std::make_shared(p); - *out = impl; - return arrow::Status::OK(); - } - arrow::Status Init() override { - if (initialized_) { - return arrow::Status::OK(); - } - if (p_->action_name_list_.empty()) { - return arrow::Status::Invalid( - "ExprVisitor::SplitArrayListWithAction have empty action_name_list, " - "this " - "is invalid."); - } - - std::vector> type_list; - for (auto col_name : p_->action_param_list_) { - std::shared_ptr field; - int col_id; - RETURN_NOT_OK(GetColumnIdAndFieldByName(p_->schema_, col_name, &col_id, &field)); - p_->result_fields_.push_back(field); - col_id_list_.push_back(col_id); - type_list.push_back(field->type()); - } - RETURN_NOT_OK(extra::SplitArrayListWithActionKernel::Make( - &p_->ctx_, p_->action_name_list_, type_list, &kernel_)); - initialized_ = true; - finish_return_type_ = ArrowComputeResultType::Batch; - return arrow::Status::OK(); - } - - arrow::Status Eval() override { - switch (p_->dependency_result_type_) { - case ArrowComputeResultType::Array: { - ArrayList col_list; - for (auto col_id : col_id_list_) { - if (col_id >= p_->in_record_batch_->num_columns()) { - return arrow::Status::Invalid( - "SplitArrayListWithActionVisitorImpl Eval col_id is bigger than input " - "batch numColumns."); - } - auto col = p_->in_record_batch_->column(col_id); - col_list.push_back(col); - } - TIME_MICRO_OR_RAISE(p_->elapse_time_, kernel_->Evaluate(col_list, p_->in_array_)); - p_->dependency_result_type_ = ArrowComputeResultType::None; - } break; - default: - return arrow::Status::NotImplemented( - "SplitArrayListWithActionVisitorImpl: Does not support this type of input."); - } - return arrow::Status::OK(); - } - - arrow::Status Finish() override { - RETURN_NOT_OK(ExprVisitorImpl::Finish()); - switch (finish_return_type_) { - case ArrowComputeResultType::Batch: { - RETURN_NOT_OK(kernel_->Finish(&p_->result_batch_)); - p_->return_type_ = ArrowComputeResultType::Batch; - } break; - default: { - return arrow::Status::NotImplemented( - "SplitArrayListWithActionVisitorImpl only support finish_return_type as " - "Batch."); - break; - } - } - return arrow::Status::OK(); - } - - arrow::Status MakeResultIterator(std::shared_ptr schema, - std::shared_ptr* out) override { - switch (finish_return_type_) { - case ArrowComputeResultType::Batch: { - std::shared_ptr> iter_out; - TIME_MICRO_OR_RAISE(p_->elapse_time_, - kernel_->MakeResultIterator(schema, &iter_out)); - *out = std::dynamic_pointer_cast(iter_out); - p_->return_type_ = ArrowComputeResultType::BatchIterator; - } break; - default: - return arrow::Status::Invalid( - "SplitArrayListWithActionVisitorImpl Finish does not support dependency type " - "other than Batch."); - } - return arrow::Status::OK(); - } - - private: - std::vector col_id_list_; -}; - -////////////////////////// AggregateVisitorImpl /////////////////////// -class AggregateVisitorImpl : public ExprVisitorImpl { - public: - AggregateVisitorImpl(ExprVisitor* p, std::string func_name) - : ExprVisitorImpl(p), func_name_(func_name) {} - static arrow::Status Make(ExprVisitor* p, std::string func_name, - std::shared_ptr* out) { - auto impl = std::make_shared(p, func_name); - *out = impl; - return arrow::Status::OK(); - } - arrow::Status Init() override { - if (initialized_) { - return arrow::Status::OK(); - } - for (auto col_name : p_->param_field_names_) { - std::shared_ptr field; - int col_id; - RETURN_NOT_OK(GetColumnIdAndFieldByName(p_->schema_, col_name, &col_id, &field)); - p_->result_fields_.push_back(field); - col_id_list_.push_back(col_id); - } - auto data_type = p_->result_fields_[0]->type(); - - if (func_name_.compare("sum") == 0) { - RETURN_NOT_OK(extra::SumArrayKernel::Make(&p_->ctx_, data_type, &kernel_)); - kernel_list_.push_back(kernel_); - } else if (func_name_.compare("count") == 0) { - RETURN_NOT_OK(extra::CountArrayKernel::Make(&p_->ctx_, data_type, &kernel_)); - kernel_list_.push_back(kernel_); - } else if (func_name_.compare("sum_count") == 0) { - p_->result_fields_.push_back(arrow::field("cnt", arrow::int64())); - RETURN_NOT_OK(extra::SumCountArrayKernel::Make(&p_->ctx_, data_type, &kernel_)); - kernel_list_.push_back(kernel_); - } else if (func_name_.compare("sum_count_merge") == 0) { - RETURN_NOT_OK(extra::SumArrayKernel::Make(&p_->ctx_, data_type, &kernel_)); - kernel_list_.push_back(kernel_); - RETURN_NOT_OK(extra::SumArrayKernel::Make(&p_->ctx_, p_->result_fields_[1]->type(), - &kernel_)); - kernel_list_.push_back(kernel_); - } else if (func_name_.compare("avgByCount") == 0) { - p_->result_fields_.erase(p_->result_fields_.end() - 1); - RETURN_NOT_OK(extra::AvgByCountArrayKernel::Make(&p_->ctx_, data_type, &kernel_)); - kernel_list_.push_back(kernel_); - } else if (func_name_.compare("min") == 0) { - RETURN_NOT_OK(extra::MinArrayKernel::Make(&p_->ctx_, data_type, &kernel_)); - kernel_list_.push_back(kernel_); - } else if (func_name_.compare("max") == 0) { - RETURN_NOT_OK(extra::MaxArrayKernel::Make(&p_->ctx_, data_type, &kernel_)); - kernel_list_.push_back(kernel_); - } else if (func_name_.compare("stddev_samp_partial") == 0) { - p_->result_fields_.push_back(arrow::field("avg", arrow::int64())); - p_->result_fields_.push_back(arrow::field("m2", arrow::int64())); - RETURN_NOT_OK( - extra::StddevSampPartialArrayKernel::Make(&p_->ctx_, data_type, &kernel_)); - kernel_list_.push_back(kernel_); - } else if (func_name_.compare("stddev_samp_final") == 0) { - p_->result_fields_.erase(p_->result_fields_.end() - 1); - p_->result_fields_.erase(p_->result_fields_.end() - 1); - RETURN_NOT_OK( - extra::StddevSampFinalArrayKernel::Make(&p_->ctx_, data_type, &kernel_)); - kernel_list_.push_back(kernel_); - } - initialized_ = true; - finish_return_type_ = ArrowComputeResultType::Batch; - return arrow::Status::OK(); - } - - arrow::Status Eval() override { - switch (p_->dependency_result_type_) { - case ArrowComputeResultType::None: { - ArrayList in; - for (auto col_id : col_id_list_) { - if (col_id >= p_->in_record_batch_->num_columns()) { - return arrow::Status::Invalid( - "AggregateVisitorImpl Eval col_id is bigger than input " - "batch numColumns."); - } - auto col = p_->in_record_batch_->column(col_id); - in.push_back(col); - } - for (int i = 0; i < kernel_list_.size(); i++) { - if (kernel_list_.size() > 1) { - RETURN_NOT_OK(kernel_list_[i]->Evaluate({in[i]})); - } else { - RETURN_NOT_OK(kernel_list_[i]->Evaluate(in)); - } - } - } break; - default: - return arrow::Status::NotImplemented( - "AggregateVisitorImpl: Does not support this type of input."); - } - return arrow::Status::OK(); - } - - arrow::Status Finish() override { - RETURN_NOT_OK(ExprVisitorImpl::Finish()); - switch (finish_return_type_) { - case ArrowComputeResultType::Batch: { - for (auto kernel : kernel_list_) { - RETURN_NOT_OK(kernel->Finish(&p_->result_batch_)); - } - p_->return_type_ = ArrowComputeResultType::Batch; - } break; - default: { - return arrow::Status::NotImplemented( - "AggregateVisitorImpl only support finish_return_type as " - "Array."); - break; - } - } - return arrow::Status::OK(); - } - - private: - std::vector col_id_list_; - std::string func_name_; - std::vector> kernel_list_; -}; - class WindowVisitorImpl : public ExprVisitorImpl { public: WindowVisitorImpl(ExprVisitor* p, std::vector window_function_names, @@ -1224,6 +1005,84 @@ class ConcatArrayListVisitorImpl : public ExprVisitorImpl { std::vector> ret_fields_; }; +////////////////////////// HashAggregateArraysImpl /////////////////////// +class HashAggregateArraysImpl : public ExprVisitorImpl { + public: + HashAggregateArraysImpl(std::vector> field_list, + std::shared_ptr root_node, + std::vector> ret_fields, + ExprVisitor* p) + : root_node_(root_node), + field_list_(field_list), + ret_fields_(ret_fields), + ExprVisitorImpl(p) { + finish_return_type_ = ArrowComputeResultType::BatchIterator; + } + static arrow::Status Make(std::vector> field_list, + std::shared_ptr root_node, + std::vector> ret_fields, + ExprVisitor* p, std::shared_ptr* out) { + auto impl = + std::make_shared(field_list, root_node, ret_fields, p); + *out = impl; + return arrow::Status::OK(); + } + + arrow::Status Init() override { + if (initialized_) { + return arrow::Status::OK(); + } + auto function_node = std::dynamic_pointer_cast(root_node_); + auto field_node_list = + std::dynamic_pointer_cast(function_node->children()[0]) + ->children(); + auto action_node_list = + std::dynamic_pointer_cast(function_node->children()[1]) + ->children(); + + gandiva::NodeVector result_field_node_list; + gandiva::NodeVector result_expr_node_list; + if (function_node->children().size() == 4) { + result_field_node_list = + std::dynamic_pointer_cast(function_node->children()[2]) + ->children(); + result_expr_node_list = + std::dynamic_pointer_cast(function_node->children()[3]) + ->children(); + } + RETURN_NOT_OK(extra::HashAggregateKernel::Make( + &p_->ctx_, field_node_list, action_node_list, result_field_node_list, + result_expr_node_list, &kernel_)); + p_->signature_ = kernel_->GetSignature(); + initialized_ = true; + finish_return_type_ = ArrowComputeResultType::BatchIterator; + return arrow::Status::OK(); + } + + arrow::Status MakeResultIterator(std::shared_ptr schema, + std::shared_ptr* out) override { + switch (finish_return_type_) { + case ArrowComputeResultType::BatchIterator: { + std::shared_ptr> iter_out; + TIME_MICRO_OR_RAISE(p_->elapse_time_, + kernel_->MakeResultIterator(schema, &iter_out)); + *out = std::dynamic_pointer_cast(iter_out); + p_->return_type_ = ArrowComputeResultType::Batch; + } break; + default: + return arrow::Status::Invalid( + "HashAggregateArraysImpl MakeResultIterator does not support " + "dependency type other than Batch."); + } + return arrow::Status::OK(); + } + + private: + std::shared_ptr root_node_; + std::vector> field_list_; + std::vector> ret_fields_; +}; + ////////////////////////// CodegenProbeArraysVisitorImpl /////////////////////// class CodegenProbeArraysVisitorImpl : public ExprVisitorImpl { public: diff --git a/cpp/src/codegen/arrow_compute/ext/actions_impl.cc b/cpp/src/codegen/arrow_compute/ext/actions_impl.cc index 0949889b7..b858f634b 100644 --- a/cpp/src/codegen/arrow_compute/ext/actions_impl.cc +++ b/cpp/src/codegen/arrow_compute/ext/actions_impl.cc @@ -18,6 +18,15 @@ #include "codegen/arrow_compute/ext/actions_impl.h" #include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include namespace sparkcolumnarplugin { @@ -69,10 +78,18 @@ arrow::Status ActionBase::Submit(const std::shared_ptr& in, return arrow::Status::NotImplemented("ActionBase Submit is abstract."); } +arrow::Status ActionBase::EvaluateCountLiteral(const int& len) { + return arrow::Status::NotImplemented("ActionBase EvaluateCountLiteral is abstract."); +} + arrow::Status ActionBase::Evaluate(int dest_group_id) { return arrow::Status::NotImplemented("ActionBase Evaluate is abstract."); } +arrow::Status ActionBase::Evaluate(const arrow::ArrayVector& in) { + return arrow::Status::NotImplemented("ActionBase Evaluate is abstract."); +} + arrow::Status ActionBase::Evaluate(int dest_group_id, void* data) { return arrow::Status::NotImplemented("ActionBase Evaluate is abstract."); } @@ -274,8 +291,7 @@ class CountAction : public ActionBase { std::cout << "Construct CountAction" << std::endl; #endif std::unique_ptr array_builder; - arrow::MakeBuilder(ctx_->memory_pool(), arrow::TypeTraits::type_singleton(), - &array_builder); + arrow::MakeBuilder(ctx_->memory_pool(), arrow::int64(), &array_builder); builder_.reset( arrow::internal::checked_cast(array_builder.release())); } @@ -333,6 +349,19 @@ class CountAction : public ActionBase { return arrow::Status::OK(); } + arrow::Status Evaluate(const arrow::ArrayVector& in) { + if (cache_.empty()) { + cache_.resize(1, 0); + length_ = 1; + } + arrow::compute::Datum output; + arrow::compute::CountOptions option(arrow::compute::CountOptions::COUNT_ALL); + RETURN_NOT_OK(arrow::compute::Count(ctx_, option, *in[0].get(), &output)); + auto typed_scalar = std::dynamic_pointer_cast(output.scalar()); + cache_[0] += typed_scalar->value; + return arrow::Status::OK(); + } + arrow::Status Evaluate(int dest_group_id, void* data) { auto target_group_size = dest_group_id + 1; if (cache_.size() <= target_group_size) GrowByFactor(target_group_size); @@ -377,14 +406,15 @@ class CountAction : public ActionBase { } private: - using ResArrayType = typename arrow::TypeTraits::ArrayType; - using ResBuilderType = typename arrow::TypeTraits::BuilderType; + using ResArrayType = typename arrow::TypeTraits::ArrayType; + using ResBuilderType = typename arrow::TypeTraits::BuilderType; + using ScalarType = typename arrow::TypeTraits::ScalarType; // input arrow::compute::FunctionContext* ctx_; std::shared_ptr in_; int32_t row_id; // result - using CType = typename arrow::TypeTraits::CType; + using CType = typename arrow::TypeTraits::CType; std::vector cache_; std::unique_ptr builder_; uint64_t length_ = 0; @@ -443,6 +473,20 @@ class CountLiteralAction : public ActionBase { return arrow::Status::OK(); } + arrow::Status EvaluateCountLiteral(const int& len) { + if (cache_.empty()) { + cache_.resize(1, 0); + length_ = 1; + } + cache_[0] += len; + return arrow::Status::OK(); + } + + arrow::Status Evaluate(const arrow::ArrayVector& in) { + return arrow::Status::NotImplemented( + "CountLiteralAction Non-Groupby Evaluate is unsupported."); + } + arrow::Status Evaluate(int dest_group_id) { auto target_group_size = dest_group_id + 1; if (cache_.size() <= target_group_size) GrowByFactor(target_group_size); @@ -581,6 +625,33 @@ class MinAction : public ActionBase { return arrow::Status::OK(); } + arrow::Status Evaluate(const arrow::ArrayVector& in) { + if (cache_validity_.empty()) { + cache_.resize(1, 0); + cache_validity_.resize(1, false); + length_ = 1; + } + + arrow::compute::Datum minMaxOut; + arrow::compute::MinMaxOptions option; + RETURN_NOT_OK(arrow::compute::MinMax(ctx_, option, *in[0].get(), &minMaxOut)); + if (!minMaxOut.is_collection()) { + return arrow::Status::Invalid("MinMax return an invalid result."); + } + auto col = minMaxOut.collection(); + if (col.size() < 2) { + return arrow::Status::Invalid("MinMax return an invalid result."); + } + auto typed_scalar = std::dynamic_pointer_cast(col[0].scalar()); + if (!cache_validity_[0]) { + cache_validity_[0] = true; + cache_[0] = typed_scalar->value; + } else { + if (cache_[0] > typed_scalar->value) cache_[0] = typed_scalar->value; + } + return arrow::Status::OK(); + } + arrow::Status Evaluate(int dest_group_id, void* data) { auto target_group_size = dest_group_id + 1; if (cache_validity_.size() <= target_group_size) GrowByFactor(target_group_size); @@ -635,6 +706,7 @@ class MinAction : public ActionBase { private: using CType = typename arrow::TypeTraits::CType; + using ScalarType = typename arrow::TypeTraits::ScalarType; using BuilderType = typename arrow::TypeTraits::BuilderType; // input @@ -731,6 +803,33 @@ class MaxAction : public ActionBase { return arrow::Status::OK(); } + arrow::Status Evaluate(const arrow::ArrayVector& in) { + if (cache_validity_.empty()) { + cache_.resize(1, 0); + cache_validity_.resize(1, false); + length_ = 1; + } + + arrow::compute::Datum minMaxOut; + arrow::compute::MinMaxOptions option; + RETURN_NOT_OK(arrow::compute::MinMax(ctx_, option, *in[0].get(), &minMaxOut)); + if (!minMaxOut.is_collection()) { + return arrow::Status::Invalid("MinMax return an invalid result."); + } + auto col = minMaxOut.collection(); + if (col.size() < 2) { + return arrow::Status::Invalid("MinMax return an invalid result."); + } + auto typed_scalar = std::dynamic_pointer_cast(col[1].scalar()); + if (!cache_validity_[0]) { + cache_validity_[0] = true; + cache_[0] = typed_scalar->value; + } else { + if (cache_[0] < typed_scalar->value) cache_[0] = typed_scalar->value; + } + return arrow::Status::OK(); + } + arrow::Status Evaluate(int dest_group_id, void* data) { auto target_group_size = dest_group_id + 1; if (cache_validity_.size() <= target_group_size) GrowByFactor(target_group_size); @@ -785,6 +884,7 @@ class MaxAction : public ActionBase { private: using CType = typename arrow::TypeTraits::CType; + using ScalarType = typename arrow::TypeTraits::ScalarType; using BuilderType = typename arrow::TypeTraits::BuilderType; // input arrow::compute::FunctionContext* ctx_; @@ -871,6 +971,20 @@ class SumAction : public ActionBase { return arrow::Status::OK(); } + arrow::Status Evaluate(const arrow::ArrayVector& in) { + if (cache_validity_.empty()) { + cache_.resize(1, 0); + cache_validity_.resize(1, false); + length_ = 1; + } + arrow::compute::Datum output; + RETURN_NOT_OK(arrow::compute::Sum(ctx_, *in[0].get(), &output)); + auto typed_scalar = std::dynamic_pointer_cast(output.scalar()); + cache_[0] += typed_scalar->value; + if (!cache_validity_[0]) cache_validity_[0] = true; + return arrow::Status::OK(); + } + arrow::Status Evaluate(int dest_group_id, void* data) { auto target_group_size = dest_group_id + 1; if (cache_validity_.size() <= target_group_size) GrowByFactor(target_group_size); @@ -921,6 +1035,7 @@ class SumAction : public ActionBase { private: using CType = typename arrow::TypeTraits::CType; using ResDataType = typename FindAccumulatorType::Type; + using ScalarType = typename arrow::TypeTraits::ScalarType; using ResCType = typename arrow::TypeTraits::CType; using ResArrayType = typename arrow::TypeTraits::ArrayType; using ResBuilderType = typename arrow::TypeTraits::BuilderType; @@ -1013,6 +1128,27 @@ class AvgAction : public ActionBase { return arrow::Status::OK(); } + arrow::Status Evaluate(const arrow::ArrayVector& in) { + if (cache_validity_.empty()) { + cache_sum_.resize(1, 0); + cache_count_.resize(1, 0); + cache_validity_.resize(1, false); + length_ = 1; + } + arrow::compute::Datum output; + RETURN_NOT_OK(arrow::compute::Sum(ctx_, *in[0].get(), &output)); + auto typed_scalar = std::dynamic_pointer_cast(output.scalar()); + cache_sum_[0] += typed_scalar->value; + + arrow::compute::CountOptions option(arrow::compute::CountOptions::COUNT_ALL); + RETURN_NOT_OK(arrow::compute::Count(ctx_, option, *in[0].get(), &output)); + auto count_typed_scalar = std::dynamic_pointer_cast(output.scalar()); + cache_count_[0] += count_typed_scalar->value; + + if (!cache_validity_[0]) cache_validity_[0] = true; + return arrow::Status::OK(); + } + arrow::Status Evaluate(int dest_group_id, void* data) { auto target_group_size = dest_group_id + 1; if (cache_validity_.size() <= target_group_size) GrowByFactor(target_group_size); @@ -1070,6 +1206,8 @@ class AvgAction : public ActionBase { private: using CType = typename arrow::TypeTraits::CType; using ResDataType = typename FindAccumulatorType::Type; + using ScalarType = typename arrow::TypeTraits::ScalarType; + using CountScalarType = typename arrow::TypeTraits::ScalarType; using ResCType = typename arrow::TypeTraits::CType; using ResArrayType = typename arrow::TypeTraits::ArrayType; std::unique_ptr builder_; @@ -1080,7 +1218,7 @@ class AvgAction : public ActionBase { int row_id; // result std::vector cache_sum_; - std::vector cache_count_; + std::vector cache_count_; std::vector cache_validity_; uint64_t length_ = 0; }; @@ -1166,6 +1304,25 @@ class SumCountAction : public ActionBase { return arrow::Status::OK(); } + arrow::Status Evaluate(const arrow::ArrayVector& in) { + if (cache_sum_.empty()) { + cache_sum_.resize(1, 0); + cache_count_.resize(1, 0); + length_ = 1; + } + arrow::compute::Datum output; + RETURN_NOT_OK(arrow::compute::Sum(ctx_, *in[0].get(), &output)); + auto typed_scalar = std::dynamic_pointer_cast(output.scalar()); + cache_sum_[0] += typed_scalar->value; + + arrow::compute::CountOptions option(arrow::compute::CountOptions::COUNT_ALL); + RETURN_NOT_OK(arrow::compute::Count(ctx_, option, *in[0].get(), &output)); + auto count_typed_scalar = std::dynamic_pointer_cast(output.scalar()); + cache_count_[0] += count_typed_scalar->value; + + return arrow::Status::OK(); + } + arrow::Status Evaluate(int dest_group_id, void* data) { auto target_group_size = dest_group_id + 1; if (cache_sum_.size() <= target_group_size) GrowByFactor(target_group_size); @@ -1221,6 +1378,8 @@ class SumCountAction : public ActionBase { private: using CType = typename arrow::TypeTraits::CType; using ResDataType = typename FindAccumulatorType::Type; + using ScalarType = typename arrow::TypeTraits::ScalarType; + using CountScalarType = typename arrow::TypeTraits::ScalarType; using ResCType = typename arrow::TypeTraits::CType; using ResArrayType = typename arrow::TypeTraits::ArrayType; std::unique_ptr sum_builder_; @@ -1318,6 +1477,24 @@ class SumCountMergeAction : public ActionBase { return arrow::Status::OK(); } + arrow::Status Evaluate(const arrow::ArrayVector& in) { + if (cache_sum_.empty()) { + cache_sum_.resize(1, 0); + cache_count_.resize(1, 0); + length_ = 1; + } + arrow::compute::Datum output; + RETURN_NOT_OK(arrow::compute::Sum(ctx_, *in[0].get(), &output)); + auto typed_scalar = std::dynamic_pointer_cast(output.scalar()); + cache_sum_[0] += typed_scalar->value; + + RETURN_NOT_OK(arrow::compute::Sum(ctx_, *in[1].get(), &output)); + auto count_typed_scalar = std::dynamic_pointer_cast(output.scalar()); + cache_count_[0] += count_typed_scalar->value; + + return arrow::Status::OK(); + } + arrow::Status Evaluate(int dest_group_id, void* data, void* data2) { auto target_group_size = dest_group_id + 1; if (cache_sum_.size() <= target_group_size) GrowByFactor(target_group_size); @@ -1374,6 +1551,8 @@ class SumCountMergeAction : public ActionBase { using CType = typename arrow::TypeTraits::CType; using ArrayType = typename arrow::TypeTraits::ArrayType; using ResDataType = typename FindAccumulatorType::Type; + using ScalarType = typename arrow::TypeTraits::ScalarType; + using CountScalarType = typename arrow::TypeTraits::ScalarType; using ResCType = typename arrow::TypeTraits::CType; using ResArrayType = typename arrow::TypeTraits::ArrayType; std::unique_ptr sum_builder_; @@ -1468,6 +1647,26 @@ class AvgByCountAction : public ActionBase { return arrow::Status::OK(); } + arrow::Status Evaluate(const arrow::ArrayVector& in) { + if (cache_validity_.empty()) { + cache_sum_.resize(1, 0); + cache_count_.resize(1, 0); + cache_validity_.resize(1, false); + length_ = 1; + } + arrow::compute::Datum output; + RETURN_NOT_OK(arrow::compute::Sum(ctx_, *in[0].get(), &output)); + auto typed_scalar = std::dynamic_pointer_cast(output.scalar()); + cache_sum_[0] += typed_scalar->value; + + RETURN_NOT_OK(arrow::compute::Sum(ctx_, *in[1].get(), &output)); + auto count_typed_scalar = std::dynamic_pointer_cast(output.scalar()); + cache_count_[0] += count_typed_scalar->value; + + if (!cache_validity_[0]) cache_validity_[0] = true; + return arrow::Status::OK(); + } + arrow::Status Evaluate(int dest_group_id, void* data, void* data2) { auto target_group_size = dest_group_id + 1; if (cache_validity_.size() <= target_group_size) GrowByFactor(target_group_size); @@ -1534,6 +1733,8 @@ class AvgByCountAction : public ActionBase { using CType = typename arrow::TypeTraits::CType; using ArrayType = typename arrow::TypeTraits::ArrayType; using ResDataType = typename FindAccumulatorType::Type; + using ScalarType = typename arrow::TypeTraits::ScalarType; + using CountScalarType = typename arrow::TypeTraits::ScalarType; using ResCType = typename arrow::TypeTraits::CType; using ResArrayType = typename arrow::TypeTraits::ArrayType; std::unique_ptr builder_; @@ -1655,6 +1856,43 @@ class StddevSampPartialAction : public ActionBase { return arrow::Status::OK(); } + arrow::Status Evaluate(const arrow::ArrayVector& in) { + if (cache_validity_.empty()) { + cache_m2_.resize(1, 0); + cache_sum_.resize(1, 0); + cache_count_.resize(1, 0); + cache_validity_.resize(1, false); + length_ = 1; + } + + /*arrow::compute::Datum sum_out; + arrow::compute::Datum cnt_out; + arrow::compute::Datum mean_out; + arrow::compute::Datum m2_out; + arrow::compute::CountOptions option(arrow::compute::CountOptions::COUNT_ALL); + RETURN_NOT_OK(arrow::compute::Sum(ctx_, *in[0].get(), &sum_out)); + RETURN_NOT_OK(arrow::compute::Count(ctx_, option, *in[0].get(), &cnt_out)); + RETURN_NOT_OK(arrow::compute::Mean(ctx_, *in[0].get(), &mean_out)); + RETURN_NOT_OK(M2(ctx_, *in[0].get(), mean_out, &m2_out)); + auto typed_scalar = std::dynamic_pointer_cast(sum_out.scalar()); + cache_sum_[0] += typed_scalar->value; + + typed_scalar = std::dynamic_pointer_cast(cnt_out.scalar()); + cache_count_[0] += typed_scalar->value; + + typed_scalar = std::dynamic_pointer_cast(m2_out.scalar()); + cache_m2_[0] += typed_scalar->value;*/ + auto len = in[0]->length(); + auto typed_in_0 = std::dynamic_pointer_cast(in[0]); + for (int i = 0; i < len; i++) { + auto v = typed_in_0->GetView(i); + RETURN_NOT_OK(Evaluate(0, (void*)&v)); + } + + if (!cache_validity_[0]) cache_validity_[0] = true; + return arrow::Status::OK(); + } + arrow::Status Evaluate(int dest_group_id, void* data) { auto target_group_size = dest_group_id + 1; if (cache_validity_.size() <= target_group_size) GrowByFactor(target_group_size); @@ -1749,9 +1987,51 @@ class StddevSampPartialAction : public ActionBase { return arrow::Status::OK(); } + /*arrow::Status getM2(arrow::compute::FunctionContext* ctx, + const arrow::compute::Datum& value, + const arrow::compute::Datum& mean, arrow::compute::Datum* out) { + using MeanCType = typename arrow::TypeTraits::CType; + using MeanScalarType = typename arrow::TypeTraits::ScalarType; + using ValueCType = typename arrow::TypeTraits::CType; + std::shared_ptr mean_scalar = mean.scalar(); + auto mean_typed_scalar = std::dynamic_pointer_cast(mean_scalar); + double mean_res = mean_typed_scalar->value * 1.0; + double m2_res = 0; + + if (!value.is_array()) { + return arrow::Status::Invalid("AggregateKernel expects Array"); + } + auto array = value.make_array(); + auto typed_array = std::static_pointer_cast>(array); + const ValueCType* input = typed_array->raw_values(); + for (int64_t i = 0; i < (*array).length(); i++) { + auto val = input[i]; + if (val) { + m2_res += (input[i] * 1.0 - mean_res) * (input[i] * 1.0 - mean_res); + } + } + *out = arrow::MakeScalar(m2_res); + return arrow::Status::OK(); + } + + arrow::Status M2(arrow::compute::FunctionContext* ctx, const arrow::Array& array, + const arrow::compute::Datum& mean, arrow::compute::Datum* out) { + arrow::compute::Datum value = array.data(); + auto data_type = value.type(); + + if (data_type == nullptr) + return arrow::Status::Invalid("Datum must be array-like"); + else if (!is_integer(data_type->id()) && !is_floating(data_type->id())) + return arrow::Status::Invalid("Datum must contain a NumericType"); + RETURN_NOT_OK(getM2(ctx, value, mean, out)); + return arrow::Status::OK(); + }*/ + private: using CType = typename arrow::TypeTraits::CType; + using ArrayType = typename arrow::TypeTraits::ArrayType; using ResDataType = typename FindAccumulatorType::Type; + using ScalarType = typename arrow::TypeTraits::ScalarType; using ResCType = typename arrow::TypeTraits::CType; using ResArrayType = typename arrow::TypeTraits::ArrayType; std::unique_ptr count_builder_; @@ -1866,6 +2146,45 @@ class StddevSampFinalAction : public ActionBase { return arrow::Status::OK(); } + arrow::Status Evaluate(const arrow::ArrayVector& in) { + /*if (cache_validity_.empty()) { + cache_avg_.resize(1, 0); + cache_m2_.resize(1, 0); + cache_count_.resize(1, 0); + cache_validity_.resize(1, false); + length_ = 1; + } + + arrow::compute::Datum cnt_out; + arrow::compute::Datum avg_out; + arrow::compute::Datum m2_out; + RETURN_NOT_OK(arrow::compute::Sum(ctx_, *in[0].get(), &cnt_out)); + RETURN_NOT_OK( + updateValue(ctx_, *in[0].get(), *in[1].get(), *in[2].get(), &avg_out, &m2_out)); + + auto typed_scalar = std::dynamic_pointer_cast(cnt_out.scalar()); + cache_count_[0] += typed_scalar->value; + + typed_scalar = std::dynamic_pointer_cast(avg_out.scalar()); + cache_avg_[0] += typed_scalar->value; + + typed_scalar = std::dynamic_pointer_cast(m2_out.scalar()); + cache_m2_[0] += typed_scalar->value;*/ + + auto len = in[0]->length(); + auto typed_in_0 = std::dynamic_pointer_cast(in[0]); + auto typed_in_1 = std::dynamic_pointer_cast(in[1]); + auto typed_in_2 = std::dynamic_pointer_cast(in[2]); + for (int i = 0; i < len; i++) { + auto v1 = typed_in_0->GetView(i); + auto v2 = typed_in_1->GetView(i); + auto v3 = typed_in_2->GetView(i); + RETURN_NOT_OK(Evaluate(0, (void*)&v1, (void*)&v2, (void*)&v3)); + } + if (!cache_validity_[0]) cache_validity_[0] = true; + return arrow::Status::OK(); + } + arrow::Status Evaluate(int dest_group_id, void* data, void* data2, void* data3) { auto target_group_size = dest_group_id + 1; if (cache_validity_.size() <= target_group_size) GrowByFactor(target_group_size); @@ -1932,10 +2251,77 @@ class StddevSampFinalAction : public ActionBase { return arrow::Status::OK(); } + /*arrow::Status getAvgM2(arrow::compute::FunctionContext* ctx, + const arrow::compute::Datum& cnt_value, + const arrow::compute::Datum& avg_value, + const arrow::compute::Datum& m2_value, + arrow::compute::Datum* avg_out, arrow::compute::Datum* m2_out) { + using MeanCType = typename arrow::TypeTraits::CType; + using MeanScalarType = typename arrow::TypeTraits::ScalarType; + using ValueCType = typename arrow::TypeTraits::CType; + + if (!(cnt_value.is_array() && avg_value.is_array() && m2_value.is_array())) { + return arrow::Status::Invalid("AggregateKernel expects Array datum"); + } + + auto cnt_array = cnt_value.make_array(); + auto avg_array = avg_value.make_array(); + auto m2_array = m2_value.make_array(); + + auto cnt_typed_array = std::static_pointer_cast(cnt_array); + auto avg_typed_array = std::static_pointer_cast(avg_array); + auto m2_typed_array = std::static_pointer_cast(m2_array); + const ValueCType* cnt_input = cnt_typed_array->raw_values(); + const MeanCType* avg_input = avg_typed_array->raw_values(); + const MeanCType* m2_input = m2_typed_array->raw_values(); + + double cnt_res = 0; + double avg_res = 0; + double m2_res = 0; + for (int64_t i = 0; i < (*cnt_array).length(); i++) { + double cnt_val = cnt_input[i]; + double avg_val = avg_input[i]; + double m2_val = m2_input[i]; + if (i == 0) { + cnt_res = cnt_val; + avg_res = avg_val; + m2_res = m2_val; + } else { + if (cnt_val > 0) { + double delta = avg_val - avg_res; + double deltaN = (cnt_res + cnt_val) > 0 ? delta / (cnt_res + cnt_val) : 0; + avg_res += deltaN * cnt_val; + m2_res += (m2_val + delta * deltaN * cnt_res * cnt_val); + cnt_res += cnt_val; + } + } + } + *avg_out = arrow::MakeScalar(avg_res); + *m2_out = arrow::MakeScalar(m2_res); + return arrow::Status::OK(); + } + + arrow::Status updateValue(arrow::compute::FunctionContext* ctx, + const arrow::Array& cnt_array, const arrow::Array& avg_array, + const arrow::Array& m2_array, arrow::compute::Datum* avg_out, + arrow::compute::Datum* m2_out) { + arrow::compute::Datum cnt_value = cnt_array.data(); + arrow::compute::Datum avg_value = avg_array.data(); + arrow::compute::Datum m2_value = m2_array.data(); + auto cnt_data_type = cnt_value.type(); + if (cnt_data_type == nullptr) + return arrow::Status::Invalid("Datum must be array-like"); + else if (!is_integer(cnt_data_type->id()) && !is_floating(cnt_data_type->id())) + return arrow::Status::Invalid("Datum must contain a NumericType"); + RETURN_NOT_OK(getAvgM2(ctx, cnt_value, avg_value, m2_value, avg_out, m2_out)); + return arrow::Status::OK(); + }*/ + private: using CType = typename arrow::TypeTraits::CType; using ArrayType = typename arrow::TypeTraits::ArrayType; using ResDataType = typename FindAccumulatorType::Type; + using ScalarType = typename arrow::TypeTraits::ScalarType; using ResCType = typename arrow::TypeTraits::CType; using ResArrayType = typename arrow::TypeTraits::ArrayType; std::unique_ptr builder_; @@ -2001,14 +2387,14 @@ arrow::Status MakeUniqueAction(arrow::compute::FunctionContext* ctx, arrow::Status MakeCountAction(arrow::compute::FunctionContext* ctx, std::shared_ptr* out) { - auto action_ptr = std::make_shared>(ctx); + auto action_ptr = std::make_shared>(ctx); *out = std::dynamic_pointer_cast(action_ptr); return arrow::Status::OK(); } arrow::Status MakeCountLiteralAction(arrow::compute::FunctionContext* ctx, int arg, std::shared_ptr* out) { - auto action_ptr = std::make_shared>(ctx, arg); + auto action_ptr = std::make_shared>(ctx, arg); *out = std::dynamic_pointer_cast(action_ptr); return arrow::Status::OK(); } diff --git a/cpp/src/codegen/arrow_compute/ext/actions_impl.h b/cpp/src/codegen/arrow_compute/ext/actions_impl.h index 67efb897b..b35c52d0a 100644 --- a/cpp/src/codegen/arrow_compute/ext/actions_impl.h +++ b/cpp/src/codegen/arrow_compute/ext/actions_impl.h @@ -52,6 +52,8 @@ class ActionBase { virtual arrow::Status Submit(const std::shared_ptr& in, std::function* on_valid, std::function* on_null); + virtual arrow::Status EvaluateCountLiteral(const int& len); + virtual arrow::Status Evaluate(const arrow::ArrayVector& in); virtual arrow::Status Evaluate(int dest_group_id); virtual arrow::Status Evaluate(int dest_group_id, void* data); virtual arrow::Status Evaluate(int dest_group_id, void* data1, void* data2); diff --git a/cpp/src/codegen/arrow_compute/ext/hash_aggregate_kernel.cc b/cpp/src/codegen/arrow_compute/ext/hash_aggregate_kernel.cc index 71c5d9ce4..7f690be59 100644 --- a/cpp/src/codegen/arrow_compute/ext/hash_aggregate_kernel.cc +++ b/cpp/src/codegen/arrow_compute/ext/hash_aggregate_kernel.cc @@ -22,11 +22,11 @@ #include #include -#include #include #include #include +#include "codegen/arrow_compute/ext/actions_impl.h" #include "codegen/arrow_compute/ext/array_item_index.h" #include "codegen/arrow_compute/ext/code_generator_base.h" #include "codegen/arrow_compute/ext/codegen_common.h" @@ -36,6 +36,8 @@ #include "codegen/arrow_compute/ext/kernels_ext.h" #include "codegen/arrow_compute/ext/typed_action_codegen_impl.h" #include "precompile/gandiva_projector.h" +#include "precompile/hash_map.h" +#include "precompile/sparse_hash_map.h" #include "utils/macros.h" namespace sparkcolumnarplugin { @@ -43,6 +45,7 @@ namespace codegen { namespace arrowcompute { namespace extra { using ArrayList = std::vector>; +using precompile::StringHashMap; /////////////// SortArraysToIndices //////////////// class HashAggregateKernel::Impl { @@ -59,6 +62,53 @@ class HashAggregateKernel::Impl { input_field_list_.push_back( std::dynamic_pointer_cast(node)->field()); } + + // map action_list nodes to keys or other aggregate actions + for (auto node : action_list_) { + auto func_node = std::dynamic_pointer_cast(node); + auto func_name = func_node->descriptor()->name(); + std::shared_ptr type; + if (func_node->children().size() > 0) { + type = func_node->children()[0]->return_type(); + } else { + type = func_node->return_type(); + } + if (func_name.compare(0, 7, "action_") == 0) { + action_name_list_.push_back(std::make_pair(func_name, type)); + std::vector child_prepare_idxs; + if (func_name.compare(0, 20, "action_countLiteral_") == 0) { + action_prepare_index_list_.push_back(child_prepare_idxs); + continue; + } + for (auto child_node : func_node->children()) { + bool found = false; + for (int i = 0; i < prepare_function_list_.size(); i++) { + auto tmp_node = prepare_function_list_[i]; + if (tmp_node->ToString() == child_node->ToString()) { + child_prepare_idxs.push_back(i); + if (func_name == "action_groupby") { + key_index_list_.push_back(i); + key_node_list_.push_back(child_node); + } + found = true; + break; + } + } + if (!found) { + if (func_name == "action_groupby") { + key_index_list_.push_back(prepare_function_list_.size()); + key_node_list_.push_back(child_node); + } + child_prepare_idxs.push_back(prepare_function_list_.size()); + prepare_function_list_.push_back(child_node); + } + } + action_prepare_index_list_.push_back(child_prepare_idxs); + } else { + THROW_NOT_OK(arrow::Status::Invalid("Expected some with action_ prefix.")); + } + } + if (result_field_node_list.size() == result_expr_node_list.size()) { auto tmp_size = result_field_node_list.size(); bool no_result_project = true; @@ -81,6 +131,68 @@ class HashAggregateKernel::Impl { virtual arrow::Status MakeResultIterator( std::shared_ptr schema, std::shared_ptr>* out) { + // 1. create pre project + std::shared_ptr pre_process_projector; + if (!prepare_function_list_.empty()) { + auto pre_process_expr_list = GetGandivaKernel(prepare_function_list_); + pre_process_projector = std::make_shared( + ctx_, arrow::schema(input_field_list_), pre_process_expr_list); + } + + // 2. action_impl_list + std::vector> action_impl_list; + RETURN_NOT_OK(PrepareActionList(action_name_list_, &action_impl_list)); + + // 3. create post project + std::shared_ptr post_process_projector; + if (!result_expr_list_.empty()) { + auto post_process_expr_list = GetGandivaKernel(result_expr_list_); + post_process_projector = std::make_shared( + ctx_, arrow::schema(result_field_list_), post_process_expr_list); + } + + auto result_schema = schema; + // 4. create ResultIterator + if (key_node_list_.size() > 1 || + (key_node_list_.size() > 0 && + key_node_list_[0]->return_type()->id() == arrow::Type::STRING)) { + *out = std::make_shared>( + ctx_, result_schema, key_index_list_, action_prepare_index_list_, + pre_process_projector, post_process_projector, action_impl_list); + } else { + auto type = arrow::int32(); + if (key_node_list_.size() > 0) type = key_node_list_[0]->return_type(); +#define PROCESS_SUPPORTED_TYPES(PROCESS) \ + PROCESS(arrow::BooleanType) \ + PROCESS(arrow::UInt8Type) \ + PROCESS(arrow::Int8Type) \ + PROCESS(arrow::UInt16Type) \ + PROCESS(arrow::Int16Type) \ + PROCESS(arrow::UInt32Type) \ + PROCESS(arrow::Int32Type) \ + PROCESS(arrow::UInt64Type) \ + PROCESS(arrow::Int64Type) \ + PROCESS(arrow::FloatType) \ + PROCESS(arrow::DoubleType) \ + PROCESS(arrow::Date32Type) \ + PROCESS(arrow::Date64Type) + switch (type->id()) { +#define PROCESS(InType) \ + case TypeTraits::type_id: { \ + *out = std::make_shared>( \ + ctx_, result_schema, key_index_list_, action_prepare_index_list_, \ + pre_process_projector, post_process_projector, action_impl_list); \ + } break; + PROCESS_SUPPORTED_TYPES(PROCESS) +#undef PROCESS + default: { + return arrow::Status::NotImplemented( + "HashAggregateResultIterator doesn't suppoty type ", type->ToString()); + } break; + } +#undef PROCESS_SUPPORTED_TYPES + } + return arrow::Status::OK(); } @@ -107,61 +219,18 @@ class HashAggregateKernel::Impl { std::stringstream finish_condition_ss; std::stringstream process_ss; std::stringstream action_list_define_function_ss; - std::vector> action_name_list; - std::vector> action_prepare_index_list; + std::vector> action_name_list = + action_name_list_; + std::vector> action_prepare_index_list = action_prepare_index_list_; - std::vector key_index_list; - std::vector prepare_function_list; - std::vector key_node_list; + std::vector key_index_list = key_index_list_; + std::vector prepare_function_list = prepare_function_list_; + std::vector key_node_list = key_node_list_; std::vector key_hash_field_list; std::vector, gandiva::DataTypePtr>> project_output_list; // 1. Get action list and action_prepare_project_list - for (auto node : action_list_) { - auto func_node = std::dynamic_pointer_cast(node); - auto func_name = func_node->descriptor()->name(); - std::shared_ptr type; - if (func_node->children().size() > 0) { - type = func_node->children()[0]->return_type(); - } else { - type = func_node->return_type(); - } - if (func_name.compare(0, 7, "action_") == 0) { - action_name_list.push_back(std::make_pair(func_name, type)); - std::vector child_prepare_idxs; - if (func_name.compare(0, 20, "action_countLiteral_") == 0) { - action_prepare_index_list.push_back(child_prepare_idxs); - continue; - } - for (auto child_node : func_node->children()) { - bool found = false; - for (int i = 0; i < prepare_function_list.size(); i++) { - auto tmp_node = prepare_function_list[i]; - if (tmp_node->ToString() == child_node->ToString()) { - child_prepare_idxs.push_back(i); - if (func_name == "action_groupby") { - key_index_list.push_back(i); - key_node_list.push_back(child_node); - } - found = true; - break; - } - } - if (!found) { - if (func_name == "action_groupby") { - key_index_list.push_back(prepare_function_list.size()); - key_node_list.push_back(child_node); - } - child_prepare_idxs.push_back(prepare_function_list.size()); - prepare_function_list.push_back(child_node); - } - } - action_prepare_index_list.push_back(child_prepare_idxs); - } else { - THROW_NOT_OK(arrow::Status::Invalid("Expected some with action_ prefix.")); - } - } if (key_node_list.size() > 1 || (key_node_list.size() > 0 && @@ -245,9 +314,9 @@ class HashAggregateKernel::Impl { prepare_ss << "}" << std::endl; idx++; } - prepare_ss << "auto " << unsafe_row_name << " = std::string(" << unsafe_row_name - << "_unsafe_row->data, " << unsafe_row_name << "_unsafe_row->cursor);" - << std::endl; + prepare_ss << "auto " << unsafe_row_name << " = arrow::util::string_view(" + << unsafe_row_name << "_unsafe_row->data, " << unsafe_row_name + << "_unsafe_row->cursor);" << std::endl; } } @@ -425,8 +494,365 @@ class HashAggregateKernel::Impl { std::vector> action_list_; std::vector> result_field_list_; std::vector> result_expr_list_; -}; + std::vector prepare_function_list_; + std::vector key_node_list_; + std::vector key_index_list_; + + std::vector> action_name_list_; + std::vector> action_prepare_index_list_; + + arrow::Status PrepareActionList( + std::vector> action_name_list, + std::vector>* action_list) { + for (int action_id = 0; action_id < action_name_list.size(); action_id++) { + std::shared_ptr action; + auto action_name = action_name_list[action_id].first; + auto action_input_type = action_name_list[action_id].second; + if (action_name.compare("action_groupby") == 0) { + RETURN_NOT_OK(MakeUniqueAction(ctx_, action_input_type, &action)); + } else if (action_name.compare("action_count") == 0) { + RETURN_NOT_OK(MakeCountAction(ctx_, &action)); + } else if (action_name.compare("action_sum") == 0) { + RETURN_NOT_OK(MakeSumAction(ctx_, action_input_type, &action)); + } else if (action_name.compare("action_avg") == 0) { + RETURN_NOT_OK(MakeAvgAction(ctx_, action_input_type, &action)); + } else if (action_name.compare("action_min") == 0) { + RETURN_NOT_OK(MakeMinAction(ctx_, action_input_type, &action)); + } else if (action_name.compare("action_max") == 0) { + RETURN_NOT_OK(MakeMaxAction(ctx_, action_input_type, &action)); + } else if (action_name.compare("action_sum_count") == 0) { + RETURN_NOT_OK(MakeSumCountAction(ctx_, action_input_type, &action)); + } else if (action_name.compare("action_sum_count_merge") == 0) { + RETURN_NOT_OK(MakeSumCountMergeAction(ctx_, action_input_type, &action)); + } else if (action_name.compare("action_avgByCount") == 0) { + RETURN_NOT_OK(MakeAvgByCountAction(ctx_, action_input_type, &action)); + } else if (action_name.compare(0, 20, "action_countLiteral_") == 0) { + int arg = std::stoi(action_name.substr(20)); + RETURN_NOT_OK(MakeCountLiteralAction(ctx_, arg, &action)); + } else if (action_name.compare("action_stddev_samp_partial") == 0) { + RETURN_NOT_OK(MakeStddevSampPartialAction(ctx_, action_input_type, &action)); + } else if (action_name.compare("action_stddev_samp_final") == 0) { + RETURN_NOT_OK(MakeStddevSampFinalAction(ctx_, action_input_type, &action)); + } else { + return arrow::Status::NotImplemented(action_name, " is not implementetd."); + } + (*action_list).push_back(action); + } + return arrow::Status::OK(); + } + + template + class HashAggregateResultIterator {}; + + template + class HashAggregateResultIterator> + : public ResultIterator { + public: + using T = typename arrow::TypeTraits::CType; + using ArrayType = typename arrow::TypeTraits::ArrayType; + HashAggregateResultIterator( + arrow::compute::FunctionContext* ctx, gandiva::SchemaPtr result_schema, + std::vector& key_index_list, + const std::vector>& action_prepare_index_list, + std::shared_ptr pre_process_projector, + std::shared_ptr post_process_projector, + std::vector> action_impl_list) + : ctx_(ctx), + result_schema_(result_schema), + key_index_list_(key_index_list), + action_prepare_index_list_(action_prepare_index_list), + pre_process_projector_(pre_process_projector), + post_process_projector_(post_process_projector), + action_impl_list_(action_impl_list) { + aggr_hash_table_ = std::make_shared>(ctx->memory_pool()); + } + + arrow::Status ProcessAndCacheOne( + const std::vector>& orig_in, + const std::shared_ptr& selection = nullptr) override { + if (orig_in.size() == 0 || orig_in[0]->length() == 0) return arrow::Status::OK(); + // 1. do pre process and prepare action_func + arrow::ArrayVector in; + if (pre_process_projector_) { + in = pre_process_projector_->Evaluate(orig_in); + } else { + in = orig_in; + } + + // 2.1 handle no groupby scenario + if (key_index_list_.size() == 0) { + arrow::ArrayVector cols; + for (int i = 0; i < action_impl_list_.size(); i++) { + auto action = action_impl_list_[i]; + cols.clear(); + for (auto idx : action_prepare_index_list_[i]) { + cols.push_back(in[idx]); + } + if (cols.empty()) { + // There is a special case, when we need to do no groupby count literal + RETURN_NOT_OK(action->EvaluateCountLiteral(in[0]->length())); + + } else { + RETURN_NOT_OK(action->Evaluate(cols)); + } + } + total_out_length_ = 1; + return arrow::Status::OK(); + } + + // 2.2 Get each row's group by key + auto typed_key_in = std::dynamic_pointer_cast(in[key_index_list_[0]]); + auto length = in[0]->length(); + + std::vector indices; + indices.resize(length, -1); + + for (int i = 0; i < length; i++) { + auto aggr_key = typed_key_in->GetView(i); + auto aggr_key_validity = + typed_key_in->null_count() == 0 ? true : !typed_key_in->IsNull(i); + + // 3. get key from hash_table + int memo_index = 0; + if (!aggr_key_validity) { + memo_index = aggr_hash_table_->GetOrInsertNull([](int) {}, [](int) {}); + } else { + aggr_hash_table_->GetOrInsert( + aggr_key, [](int) {}, [](int) {}, &memo_index); + } + + if (memo_index > max_group_id_) { + max_group_id_ = memo_index; + } + indices[i] = memo_index; + } + + total_out_length_ = max_group_id_ + 1; + // 4. prepare action func and evaluate + std::vector> eval_func_list; + std::vector> eval_null_func_list; + arrow::ArrayVector cols; + for (int i = 0; i < action_impl_list_.size(); i++) { + auto action = action_impl_list_[i]; + cols.clear(); + for (auto idx : action_prepare_index_list_[i]) { + cols.push_back(in[idx]); + } + std::function func; + std::function null_func; + action->Submit(cols, max_group_id_, &func, &null_func); + eval_func_list.push_back(func); + eval_null_func_list.push_back(null_func); + } + + for (auto memo_index : indices) { + if (memo_index == -1) { + for (auto eval_func : eval_null_func_list) { + RETURN_NOT_OK(eval_func()); + } + } else { + for (auto eval_func : eval_func_list) { + RETURN_NOT_OK(eval_func(memo_index)); + } + } + } + + return arrow::Status::OK(); + } + + bool HasNext() { return offset_ < total_out_length_; } + + arrow::Status Next(std::shared_ptr* out) { + uint64_t out_length = 0; + int gp_idx = 0; + std::vector> outputs; + for (auto action : action_impl_list_) { + action->Finish(offset_, 10000, &outputs); + } + if (outputs.size() > 0) { + out_length += outputs[0]->length(); + offset_ += outputs[0]->length(); + } + if (post_process_projector_) { + RETURN_NOT_OK(post_process_projector_->Evaluate(&outputs)); + } + *out = arrow::RecordBatch::Make(result_schema_, out_length, outputs); + return arrow::Status::OK(); + } + + private: + arrow::compute::FunctionContext* ctx_; + std::vector> action_impl_list_; + std::shared_ptr> aggr_hash_table_; + const std::vector& key_index_list_; + const std::vector>& action_prepare_index_list_; + std::shared_ptr pre_process_projector_; + std::shared_ptr post_process_projector_; + std::shared_ptr result_schema_; + int max_group_id_ = -1; + int offset_ = 0; + int total_out_length_ = 0; + }; + + template + class HashAggregateResultIterator> + : public ResultIterator { + public: + HashAggregateResultIterator( + arrow::compute::FunctionContext* ctx, gandiva::SchemaPtr result_schema, + const std::vector& key_index_list, + const std::vector>& action_prepare_index_list, + std::shared_ptr pre_process_projector, + std::shared_ptr post_process_projector, + std::vector> action_impl_list) + : ctx_(ctx), + result_schema_(result_schema), + key_index_list_(key_index_list), + action_prepare_index_list_(action_prepare_index_list), + pre_process_projector_(pre_process_projector), + post_process_projector_(post_process_projector), + action_impl_list_(action_impl_list) { + aggr_hash_table_ = std::make_shared(ctx->memory_pool()); + if (key_index_list.size() > 1) { + aggr_key_unsafe_row = std::make_shared(key_index_list.size()); + } + } + + arrow::Status ProcessAndCacheOne( + const std::vector>& orig_in, + const std::shared_ptr& selection = nullptr) override { + if (orig_in.size() == 0 || orig_in[0]->length() == 0) return arrow::Status::OK(); + // 1. do pre process and prepare action_func + arrow::ArrayVector in; + if (pre_process_projector_) { + in = pre_process_projector_->Evaluate(orig_in); + } else { + in = orig_in; + } + + // 2. handle multiple keys + std::vector> payloads; + std::shared_ptr typed_key_in; + if (aggr_key_unsafe_row) { + for (int idx = 0; idx < key_index_list_.size(); idx++) { + auto arr = in[key_index_list_[idx]]; + std::shared_ptr payload; + MakeUnsafeArray(arr->type(), idx, arr, &payload); + payloads.push_back(payload); + } + } else { + typed_key_in = + std::dynamic_pointer_cast(in[key_index_list_[0]]); + } + + // 3. Get each row's group by key + auto length = in[0]->length(); + std::vector indices; + indices.resize(length, -1); + for (int i = 0; i < length; i++) { + auto aggr_key_validity = true; + arrow::util::string_view aggr_key; + if (aggr_key_unsafe_row) { + aggr_key_unsafe_row->reset(); + int idx = 0; + for (auto payload_arr : payloads) { + payload_arr->Append(i, &aggr_key_unsafe_row); + } + aggr_key = arrow::util::string_view(aggr_key_unsafe_row->data, + aggr_key_unsafe_row->cursor); + } else { + aggr_key = typed_key_in->GetView(i); + aggr_key_validity = + typed_key_in->null_count() == 0 ? true : !typed_key_in->IsNull(i); + } + + // for (int n = 0; n < aggr_key.size(); ++n) printf("%0X ", *(aggr_key.data() + + // n)); std::cout << std::endl; + + // 3. get key from hash_table + int memo_index = 0; + if (!aggr_key_validity) { + memo_index = aggr_hash_table_->GetOrInsertNull([](int) {}, [](int) {}); + } else { + aggr_hash_table_->GetOrInsert( + aggr_key, [](int) {}, [](int) {}, &memo_index); + } + + if (memo_index > max_group_id_) { + max_group_id_ = memo_index; + } + indices[i] = memo_index; + } + + total_out_length_ = max_group_id_ + 1; + // 4. prepare action func and evaluate + std::vector> eval_func_list; + std::vector> eval_null_func_list; + arrow::ArrayVector cols; + for (int i = 0; i < action_impl_list_.size(); i++) { + auto action = action_impl_list_[i]; + cols.clear(); + for (auto idx : action_prepare_index_list_[i]) { + cols.push_back(in[idx]); + } + std::function func; + std::function null_func; + action->Submit(cols, max_group_id_, &func, &null_func); + eval_func_list.push_back(func); + eval_null_func_list.push_back(null_func); + } + + for (auto memo_index : indices) { + if (memo_index == -1) { + for (auto eval_func : eval_null_func_list) { + RETURN_NOT_OK(eval_func()); + } + } else { + for (auto eval_func : eval_func_list) { + RETURN_NOT_OK(eval_func(memo_index)); + } + } + } + return arrow::Status::OK(); + } + + bool HasNext() { return offset_ < total_out_length_; } + + arrow::Status Next(std::shared_ptr* out) { + uint64_t out_length = 0; + int gp_idx = 0; + std::vector> outputs; + for (auto action : action_impl_list_) { + action->Finish(offset_, 10000, &outputs); + } + if (outputs.size() > 0) { + out_length += outputs[0]->length(); + offset_ += outputs[0]->length(); + } + + if (post_process_projector_) { + RETURN_NOT_OK(post_process_projector_->Evaluate(&outputs)); + } + *out = arrow::RecordBatch::Make(result_schema_, out_length, outputs); + return arrow::Status::OK(); + } + + private: + arrow::compute::FunctionContext* ctx_; + std::vector> action_impl_list_; + std::shared_ptr aggr_hash_table_; + const std::vector& key_index_list_; + const std::vector>& action_prepare_index_list_; + std::shared_ptr pre_process_projector_; + std::shared_ptr post_process_projector_; + std::shared_ptr aggr_key_unsafe_row; + std::shared_ptr result_schema_; + int max_group_id_ = -1; + int offset_ = 0; + int total_out_length_ = 0; + }; +}; arrow::Status HashAggregateKernel::Make( arrow::compute::FunctionContext* ctx, std::vector> input_field_list, diff --git a/cpp/src/codegen/arrow_compute/ext/kernels_ext.cc b/cpp/src/codegen/arrow_compute/ext/kernels_ext.cc index 9dbd6e8c1..ba35ddd1f 100644 --- a/cpp/src/codegen/arrow_compute/ext/kernels_ext.cc +++ b/cpp/src/codegen/arrow_compute/ext/kernels_ext.cc @@ -23,13 +23,7 @@ #include #include #include -#include -#include -#include -#include -#include #include -#include #include #include #include @@ -63,1107 +57,6 @@ namespace extra { using ArrayList = std::vector>; -/////////////// SplitArrayListWithAction //////////////// -class SplitArrayListWithActionKernel::Impl { - public: - Impl(arrow::compute::FunctionContext* ctx, std::vector action_name_list, - std::vector> type_list) - : ctx_(ctx), action_name_list_(action_name_list) { - InitActionList(type_list); - } - virtual ~Impl() {} - - arrow::Status InitActionList(std::vector> type_list) { - int type_id = 0; -#ifdef DEBUG - std::cout << "action_name_list_ has " << action_name_list_.size() - << " elements, and type_list has " << type_list.size() << " elements." - << std::endl; -#endif - for (int action_id = 0; action_id < action_name_list_.size(); action_id++) { - std::shared_ptr action; - if (action_name_list_[action_id].compare("action_unique") == 0) { - RETURN_NOT_OK(MakeUniqueAction(ctx_, type_list[type_id], &action)); - } else if (action_name_list_[action_id].compare("action_count") == 0) { - RETURN_NOT_OK(MakeCountAction(ctx_, &action)); - } else if (action_name_list_[action_id].compare("action_sum") == 0) { - RETURN_NOT_OK(MakeSumAction(ctx_, type_list[type_id], &action)); - } else if (action_name_list_[action_id].compare("action_avg") == 0) { - RETURN_NOT_OK(MakeAvgAction(ctx_, type_list[type_id], &action)); - } else if (action_name_list_[action_id].compare("action_min") == 0) { - RETURN_NOT_OK(MakeMinAction(ctx_, type_list[type_id], &action)); - } else if (action_name_list_[action_id].compare("action_max") == 0) { - RETURN_NOT_OK(MakeMaxAction(ctx_, type_list[type_id], &action)); - } else if (action_name_list_[action_id].compare("action_sum_count") == 0) { - RETURN_NOT_OK(MakeSumCountAction(ctx_, type_list[type_id], &action)); - } else if (action_name_list_[action_id].compare("action_sum_count_merge") == 0) { - RETURN_NOT_OK(MakeSumCountMergeAction(ctx_, type_list[type_id], &action)); - } else if (action_name_list_[action_id].compare("action_avgByCount") == 0) { - RETURN_NOT_OK(MakeAvgByCountAction(ctx_, type_list[type_id], &action)); - } else if (action_name_list_[action_id].compare(0, 20, "action_countLiteral_") == - 0) { - int arg = std::stoi(action_name_list_[action_id].substr(20)); - RETURN_NOT_OK(MakeCountLiteralAction(ctx_, arg, &action)); - } else if (action_name_list_[action_id].compare("action_stddev_samp_partial") == - 0) { - RETURN_NOT_OK(MakeStddevSampPartialAction(ctx_, type_list[type_id], &action)); - } else if (action_name_list_[action_id].compare("action_stddev_samp_final") == 0) { - RETURN_NOT_OK(MakeStddevSampFinalAction(ctx_, type_list[type_id], &action)); - } else { - return arrow::Status::NotImplemented(action_name_list_[action_id], - " is not implementetd."); - } - type_id += action->RequiredColNum(); - action_list_.push_back(action); - } - return arrow::Status::OK(); - } - - arrow::Status Evaluate(const ArrayList& in, - const std::shared_ptr& in_dict) { - if (!in_dict) { - return arrow::Status::Invalid("input data is invalid"); - } - - // TODO: We used to use arrow::Minmax, while I noticed when there is null or -1 data - // inside array, max will output incorrect result, change to handmade function for now - int32_t max_group_id = 0; - auto typed_in_dict = std::dynamic_pointer_cast(in_dict); - for (int i = 0; i < typed_in_dict->length(); i++) { - if (typed_in_dict->IsValid(i)) { - if (typed_in_dict->GetView(i) > max_group_id) { - max_group_id = typed_in_dict->GetView(i); - } - } - } - - std::vector> eval_func_list; - std::vector> eval_null_func_list; - int col_id = 0; - ArrayList cols; - for (int i = 0; i < action_list_.size(); i++) { - cols.clear(); - auto action = action_list_[i]; - for (int j = 0; j < action->RequiredColNum(); j++) { - cols.push_back(in[col_id++]); - } - std::function func; - std::function null_func; - action->Submit(cols, max_group_id, &func, &null_func); - eval_func_list.push_back(func); - eval_null_func_list.push_back(null_func); - } - - for (int row_id = 0; row_id < in_dict->length(); row_id++) { - if (in_dict->IsValid(row_id)) { - auto group_id = typed_in_dict->GetView(row_id); - for (auto eval_func : eval_func_list) { - eval_func(group_id); - } - } else { - for (auto eval_func : eval_null_func_list) { - eval_func(); - } - } - } - return arrow::Status::OK(); - } - - arrow::Status Finish(ArrayList* out) { - for (auto action : action_list_) { - RETURN_NOT_OK(action->Finish(out)); - } - return arrow::Status::OK(); - } - - arrow::Status MakeResultIterator( - std::shared_ptr schema, - std::shared_ptr>* out) { - uint64_t total_length = action_list_[0]->GetResultLength(); - auto eval_func = [this, schema](uint64_t offset, uint64_t length, - std::shared_ptr* out) { - ArrayList arr_list; - for (auto action : action_list_) { - RETURN_NOT_OK(action->Finish(offset, length, &arr_list)); - } - *out = arrow::RecordBatch::Make(schema, length, arr_list); - return arrow::Status::OK(); - }; - *out = std::make_shared(ctx_, total_length, - eval_func); - return arrow::Status::OK(); - } - - private: - arrow::compute::FunctionContext* ctx_; - std::vector action_name_list_; - std::vector> action_list_; - - class SplitArrayWithActionResultIterator : public ResultIterator { - public: - SplitArrayWithActionResultIterator( - arrow::compute::FunctionContext* ctx, uint64_t total_length, - std::function* out)> - eval_func) - : ctx_(ctx), total_length_(total_length), eval_func_(eval_func) {} - ~SplitArrayWithActionResultIterator() {} - - std::string ToString() override { return "SplitArrayWithActionResultIterator"; } - - bool HasNext() override { - if (offset_ >= total_length_) { - return false; - } - return true; - } - - arrow::Status Next(std::shared_ptr* out) { - if (offset_ >= total_length_) { - *out = nullptr; - return arrow::Status::OK(); - } - auto length = (total_length_ - offset_) > GetBatchSize() - ? GetBatchSize() - : (total_length_ - offset_); - TIME_MICRO_OR_RAISE(elapse_time_, eval_func_(offset_, length, out)); - offset_ += length; - // arrow::PrettyPrint(*(*out).get(), 2, &std::cout); - return arrow::Status::OK(); - } - - private: - arrow::compute::FunctionContext* ctx_; - std::function* out)> - eval_func_; - uint64_t offset_ = 0; - const uint64_t total_length_; - uint64_t elapse_time_ = 0; - }; -}; - -arrow::Status SplitArrayListWithActionKernel::Make( - arrow::compute::FunctionContext* ctx, std::vector action_name_list, - std::vector> type_list, - std::shared_ptr* out) { - *out = - std::make_shared(ctx, action_name_list, type_list); - return arrow::Status::OK(); -} - -SplitArrayListWithActionKernel::SplitArrayListWithActionKernel( - arrow::compute::FunctionContext* ctx, std::vector action_name_list, - std::vector> type_list) { - impl_.reset(new Impl(ctx, action_name_list, type_list)); - kernel_name_ = "SplitArrayListWithActionKernel"; -} - -arrow::Status SplitArrayListWithActionKernel::Evaluate( - const ArrayList& in, const std::shared_ptr& dict) { - return impl_->Evaluate(in, dict); -} - -arrow::Status SplitArrayListWithActionKernel::Finish(ArrayList* out) { - return impl_->Finish(out); -} - -arrow::Status SplitArrayListWithActionKernel::MakeResultIterator( - std::shared_ptr schema, - std::shared_ptr>* out) { - return impl_->MakeResultIterator(schema, out); -} - -/////////////// UniqueArray //////////////// -/*class UniqueArrayKernel::Impl { - public: - Impl(arrow::compute::FunctionContext* ctx) : ctx_(ctx) {} - ~Impl() {} - arrow::Status Evaluate(const std::shared_ptr& in) { - std::shared_ptr out; - if (in->length() == 0) { - return arrow::Status::OK(); - } - arrow::compute::Datum input_datum(in); - RETURN_NOT_OK(arrow::compute::Unique(ctx_, input_datum, &out)); - if (!builder) { - RETURN_NOT_OK(MakeArrayBuilder(out->type(), ctx_->memory_pool(), &builder)); - } - - RETURN_NOT_OK(builder->AppendArrayItem(&(*out.get()), 0, 0)); - - return arrow::Status::OK(); - } - - arrow::Status Finish(std::shared_ptr* out) { - RETURN_NOT_OK(builder->Finish(out)); - return arrow::Status::OK(); - } - - private: - arrow::compute::FunctionContext* ctx_; - std::shared_ptr builder; -}; - -arrow::Status UniqueArrayKernel::Make(arrow::compute::FunctionContext* ctx, - std::shared_ptr* out) { - *out = std::make_shared(ctx); - return arrow::Status::OK(); -} - -UniqueArrayKernel::UniqueArrayKernel(arrow::compute::FunctionContext* ctx) { - impl_.reset(new Impl(ctx)); - kernel_name_ = "UniqueArrayKernel"; -} - -arrow::Status UniqueArrayKernel::Evaluate(const std::shared_ptr& in) { - return impl_->Evaluate(in); -} - -arrow::Status UniqueArrayKernel::Finish(std::shared_ptr* out) { - return impl_->Finish(out); -}*/ - -#define PROCESS_SUPPORTED_TYPES(PROCESS) \ - PROCESS(arrow::UInt8Type) \ - PROCESS(arrow::Int8Type) \ - PROCESS(arrow::UInt16Type) \ - PROCESS(arrow::Int16Type) \ - PROCESS(arrow::UInt32Type) \ - PROCESS(arrow::Int32Type) \ - PROCESS(arrow::UInt64Type) \ - PROCESS(arrow::Int64Type) \ - PROCESS(arrow::FloatType) \ - PROCESS(arrow::DoubleType) - -/////////////// SumArray //////////////// -class SumArrayKernel::Impl { - public: - Impl(arrow::compute::FunctionContext* ctx, std::shared_ptr data_type) - : ctx_(ctx), data_type_(data_type) {} - virtual ~Impl() {} - arrow::Status Evaluate(const ArrayList& in) { - arrow::compute::Datum output; - RETURN_NOT_OK(arrow::compute::Sum(ctx_, *in[0].get(), &output)); - res_data_type_ = output.scalar()->type; - scalar_list_.push_back(output.scalar()); - return arrow::Status::OK(); - } - - arrow::Status Finish(ArrayList* out) { - switch (res_data_type_->id()) { -#define PROCESS(DataType) \ - case DataType::type_id: { \ - RETURN_NOT_OK(FinishInternal(out)); \ - } break; - PROCESS_SUPPORTED_TYPES(PROCESS) -#undef PROCESS - } - return arrow::Status::OK(); - } - - template - arrow::Status FinishInternal(ArrayList* out) { - using ScalarType = typename arrow::TypeTraits::ScalarType; - using CType = typename arrow::TypeTraits::CType; - CType res = 0; - for (auto scalar_item : scalar_list_) { - auto typed_scalar = std::dynamic_pointer_cast(scalar_item); - res += typed_scalar->value; - } - std::shared_ptr arr_out; - std::shared_ptr scalar_out; - scalar_out = arrow::MakeScalar(res); - RETURN_NOT_OK(arrow::MakeArrayFromScalar(*scalar_out.get(), 1, &arr_out)); - out->push_back(arr_out); - return arrow::Status::OK(); - } - - private: - arrow::compute::FunctionContext* ctx_; - std::shared_ptr data_type_; - std::shared_ptr res_data_type_; - std::vector> scalar_list_; -}; - -arrow::Status SumArrayKernel::Make(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type, - std::shared_ptr* out) { - *out = std::make_shared(ctx, data_type); - return arrow::Status::OK(); -} - -SumArrayKernel::SumArrayKernel(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type) { - impl_.reset(new Impl(ctx, data_type)); - kernel_name_ = "SumArrayKernel"; -} - -arrow::Status SumArrayKernel::Evaluate(const ArrayList& in) { - return impl_->Evaluate(in); -} - -arrow::Status SumArrayKernel::Finish(ArrayList* out) { return impl_->Finish(out); } - -/////////////// CountArray //////////////// -class CountArrayKernel::Impl { - public: - Impl(arrow::compute::FunctionContext* ctx, std::shared_ptr data_type) - : ctx_(ctx), data_type_(data_type) {} - virtual ~Impl() {} - arrow::Status Evaluate(const ArrayList& in) { - arrow::compute::Datum output; - arrow::compute::CountOptions option(arrow::compute::CountOptions::COUNT_ALL); - RETURN_NOT_OK(arrow::compute::Count(ctx_, option, *in[0].get(), &output)); - scalar_list_.push_back(output.scalar()); - return arrow::Status::OK(); - } - - arrow::Status Finish(ArrayList* out) { - RETURN_NOT_OK(FinishInternal(out)); - return arrow::Status::OK(); - } - - template - arrow::Status FinishInternal(ArrayList* out) { - using CType = typename arrow::TypeTraits::CType; - using ScalarType = typename arrow::TypeTraits::ScalarType; - CType res = 0; - for (auto scalar_item : scalar_list_) { - auto typed_scalar = std::dynamic_pointer_cast(scalar_item); - res += typed_scalar->value; - } - std::shared_ptr arr_out; - std::shared_ptr scalar_out; - scalar_out = arrow::MakeScalar(res); - RETURN_NOT_OK(arrow::MakeArrayFromScalar(*scalar_out.get(), 1, &arr_out)); - out->push_back(arr_out); - return arrow::Status::OK(); - } - - private: - arrow::compute::FunctionContext* ctx_; - std::shared_ptr data_type_; - std::vector> scalar_list_; -}; - -arrow::Status CountArrayKernel::Make(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type, - std::shared_ptr* out) { - *out = std::make_shared(ctx, data_type); - return arrow::Status::OK(); -} - -CountArrayKernel::CountArrayKernel(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type) { - impl_.reset(new Impl(ctx, data_type)); - kernel_name_ = "CountArrayKernel"; -} - -arrow::Status CountArrayKernel::Evaluate(const ArrayList& in) { - return impl_->Evaluate(in); -} - -arrow::Status CountArrayKernel::Finish(ArrayList* out) { return impl_->Finish(out); } - -/////////////// SumCountArray //////////////// -class SumCountArrayKernel::Impl { - public: - Impl(arrow::compute::FunctionContext* ctx, std::shared_ptr data_type) - : ctx_(ctx), data_type_(data_type) {} - virtual ~Impl() {} - arrow::Status Evaluate(const ArrayList& in) { - arrow::compute::Datum sum_out; - arrow::compute::Datum cnt_out; - arrow::compute::CountOptions option(arrow::compute::CountOptions::COUNT_ALL); - RETURN_NOT_OK(arrow::compute::Sum(ctx_, *in[0].get(), &sum_out)); - RETURN_NOT_OK(arrow::compute::Count(ctx_, option, *in[0].get(), &cnt_out)); - res_data_type_ = sum_out.scalar()->type; - sum_scalar_list_.push_back(sum_out.scalar()); - cnt_scalar_list_.push_back(cnt_out.scalar()); - return arrow::Status::OK(); - } - - arrow::Status Finish(ArrayList* out) { - switch (res_data_type_->id()) { -#define PROCESS(DataType) \ - case DataType::type_id: { \ - RETURN_NOT_OK(FinishInternal(out)); \ - } break; - PROCESS_SUPPORTED_TYPES(PROCESS) -#undef PROCESS - } - return arrow::Status::OK(); - } - - template - arrow::Status FinishInternal(ArrayList* out) { - using CType = typename arrow::TypeTraits::CType; - using ScalarType = typename arrow::TypeTraits::ScalarType; - using CntScalarType = typename arrow::TypeTraits::ScalarType; - double sum_res = 0; - int64_t cnt_res = 0; - for (size_t i = 0; i < sum_scalar_list_.size(); i++) { - auto sum_typed_scalar = std::dynamic_pointer_cast(sum_scalar_list_[i]); - auto cnt_typed_scalar = - std::dynamic_pointer_cast(cnt_scalar_list_[i]); - sum_res += sum_typed_scalar->value; - cnt_res += cnt_typed_scalar->value; - } - std::shared_ptr sum_out; - std::shared_ptr sum_scalar_out; - sum_scalar_out = arrow::MakeScalar(sum_res); - RETURN_NOT_OK(arrow::MakeArrayFromScalar(*sum_scalar_out.get(), 1, &sum_out)); - - std::shared_ptr cnt_out; - std::shared_ptr cnt_scalar_out; - cnt_scalar_out = arrow::MakeScalar(cnt_res); - RETURN_NOT_OK(arrow::MakeArrayFromScalar(*cnt_scalar_out.get(), 1, &cnt_out)); - - out->push_back(sum_out); - out->push_back(cnt_out); - - return arrow::Status::OK(); - } - - private: - arrow::compute::FunctionContext* ctx_; - std::shared_ptr res_data_type_; - std::shared_ptr data_type_; - std::vector> sum_scalar_list_; - std::vector> cnt_scalar_list_; -}; - -arrow::Status SumCountArrayKernel::Make(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type, - std::shared_ptr* out) { - *out = std::make_shared(ctx, data_type); - return arrow::Status::OK(); -} - -SumCountArrayKernel::SumCountArrayKernel(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type) { - impl_.reset(new Impl(ctx, data_type)); - kernel_name_ = "SumCountArrayKernel"; -} - -arrow::Status SumCountArrayKernel::Evaluate(const ArrayList& in) { - return impl_->Evaluate(in); -} - -arrow::Status SumCountArrayKernel::Finish(ArrayList* out) { return impl_->Finish(out); } - -/////////////// AvgByCountArray //////////////// -class AvgByCountArrayKernel::Impl { - public: - Impl(arrow::compute::FunctionContext* ctx, std::shared_ptr data_type) - : ctx_(ctx), data_type_(data_type) {} - virtual ~Impl() {} - arrow::Status Evaluate(const ArrayList& in) { - arrow::compute::Datum sum_out; - arrow::compute::Datum cnt_out; - RETURN_NOT_OK(arrow::compute::Sum(ctx_, *in[0].get(), &sum_out)); - RETURN_NOT_OK(arrow::compute::Sum(ctx_, *in[1].get(), &cnt_out)); - sum_scalar_list_.push_back(sum_out.scalar()); - cnt_scalar_list_.push_back(cnt_out.scalar()); - sum_res_data_type_ = sum_out.scalar()->type; - cnt_res_data_type_ = cnt_out.scalar()->type; - return arrow::Status::OK(); - } - -#define PROCESS_INTERNAL(SumDataType, CntDataType) \ - case CntDataType::type_id: { \ - FinishInternal(out); \ - } break; - - arrow::Status Finish(ArrayList* out) { - switch (sum_res_data_type_->id()) { -#define PROCESS(SumDataType) \ - case SumDataType::type_id: { \ - switch (cnt_res_data_type_->id()) { \ - PROCESS_INTERNAL(SumDataType, arrow::UInt64Type) \ - PROCESS_INTERNAL(SumDataType, arrow::Int64Type) \ - PROCESS_INTERNAL(SumDataType, arrow::DoubleType) \ - } \ - } break; - PROCESS_SUPPORTED_TYPES(PROCESS) -#undef PROCESS -#undef PROCESS_INTERNAL - } - return arrow::Status::OK(); - } - - template - arrow::Status FinishInternal(ArrayList* out) { - using SumCType = typename arrow::TypeTraits::CType; - using CntCType = typename arrow::TypeTraits::CType; - using SumScalarType = typename arrow::TypeTraits::ScalarType; - using CntScalarType = typename arrow::TypeTraits::ScalarType; - SumCType sum_res = 0; - CntCType cnt_res = 0; - for (size_t i = 0; i < sum_scalar_list_.size(); i++) { - auto sum_typed_scalar = - std::dynamic_pointer_cast(sum_scalar_list_[i]); - auto cnt_typed_scalar = - std::dynamic_pointer_cast(cnt_scalar_list_[i]); - sum_res += sum_typed_scalar->value; - cnt_res += cnt_typed_scalar->value; - } - double res = sum_res * 1.0 / cnt_res; - std::shared_ptr arr_out; - std::shared_ptr scalar_out; - scalar_out = arrow::MakeScalar(res); - RETURN_NOT_OK(arrow::MakeArrayFromScalar(*scalar_out.get(), 1, &arr_out)); - - out->push_back(arr_out); - - return arrow::Status::OK(); - } - - private: - arrow::compute::FunctionContext* ctx_; - std::shared_ptr data_type_; - std::shared_ptr cnt_res_data_type_; - std::shared_ptr sum_res_data_type_; - std::vector> sum_scalar_list_; - std::vector> cnt_scalar_list_; -}; // namespace extra - -arrow::Status AvgByCountArrayKernel::Make(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type, - std::shared_ptr* out) { - *out = std::make_shared(ctx, data_type); - return arrow::Status::OK(); -} - -AvgByCountArrayKernel::AvgByCountArrayKernel(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type) { - impl_.reset(new Impl(ctx, data_type)); - kernel_name_ = "AvgByCountArrayKernel"; -} - -arrow::Status AvgByCountArrayKernel::Evaluate(const ArrayList& in) { - return impl_->Evaluate(in); -} - -arrow::Status AvgByCountArrayKernel::Finish(ArrayList* out) { return impl_->Finish(out); } - -/////////////// MinArray //////////////// -class MinArrayKernel::Impl { - public: - Impl(arrow::compute::FunctionContext* ctx, std::shared_ptr data_type) - : ctx_(ctx), data_type_(data_type) {} - virtual ~Impl() {} - arrow::Status Evaluate(const ArrayList& in) { - arrow::compute::Datum minMaxOut; - arrow::compute::MinMaxOptions option; - RETURN_NOT_OK(arrow::compute::MinMax(ctx_, option, *in[0].get(), &minMaxOut)); - if (!minMaxOut.is_collection()) { - return arrow::Status::Invalid("MinMax return an invalid result."); - } - auto col = minMaxOut.collection(); - if (col.size() < 2) { - return arrow::Status::Invalid("MinMax return an invalid result."); - } - auto min = col[0].scalar(); - scalar_list_.push_back(min); - return arrow::Status::OK(); - } - - arrow::Status Finish(ArrayList* out) { - switch (data_type_->id()) { -#define PROCESS(DataType) \ - case DataType::type_id: { \ - RETURN_NOT_OK(FinishInternal(out)); \ - } break; - PROCESS_SUPPORTED_TYPES(PROCESS) -#undef PROCESS - } - return arrow::Status::OK(); - } - - template - arrow::Status FinishInternal(ArrayList* out) { - using CType = typename arrow::TypeTraits::CType; - using ScalarType = typename arrow::TypeTraits::ScalarType; - auto typed_scalar = std::dynamic_pointer_cast(scalar_list_[0]); - CType res = typed_scalar->value; - for (size_t i = 1; i < scalar_list_.size(); i++) { - auto typed_scalar = std::dynamic_pointer_cast(scalar_list_[i]); - if (typed_scalar->value < res) res = typed_scalar->value; - } - std::shared_ptr arr_out; - std::shared_ptr scalar_out; - scalar_out = arrow::MakeScalar(res); - RETURN_NOT_OK(arrow::MakeArrayFromScalar(*scalar_out.get(), 1, &arr_out)); - out->push_back(arr_out); - return arrow::Status::OK(); - } - - private: - arrow::compute::FunctionContext* ctx_; - std::shared_ptr data_type_; - std::vector> scalar_list_; - std::unique_ptr array_builder_; -}; - -arrow::Status MinArrayKernel::Make(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type, - std::shared_ptr* out) { - *out = std::make_shared(ctx, data_type); - return arrow::Status::OK(); -} - -MinArrayKernel::MinArrayKernel(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type) { - impl_.reset(new Impl(ctx, data_type)); - kernel_name_ = "MinArrayKernel"; -} - -arrow::Status MinArrayKernel::Evaluate(const ArrayList& in) { - return impl_->Evaluate(in); -} - -arrow::Status MinArrayKernel::Finish(ArrayList* out) { return impl_->Finish(out); } - -/////////////// MaxArray //////////////// -class MaxArrayKernel::Impl { - public: - Impl(arrow::compute::FunctionContext* ctx, std::shared_ptr data_type) - : ctx_(ctx), data_type_(data_type) {} - virtual ~Impl() {} - arrow::Status Evaluate(const ArrayList& in) { - arrow::compute::Datum minMaxOut; - arrow::compute::MinMaxOptions option; - RETURN_NOT_OK(arrow::compute::MinMax(ctx_, option, *in[0].get(), &minMaxOut)); - if (!minMaxOut.is_collection()) { - return arrow::Status::Invalid("MinMax return an invalid result."); - } - auto col = minMaxOut.collection(); - if (col.size() < 2) { - return arrow::Status::Invalid("MinMax return an invalid result."); - } - auto max = col[1].scalar(); - scalar_list_.push_back(max); - return arrow::Status::OK(); - } - - arrow::Status Finish(ArrayList* out) { - switch (data_type_->id()) { -#define PROCESS(DataType) \ - case DataType::type_id: { \ - RETURN_NOT_OK(FinishInternal(out)); \ - } break; - PROCESS_SUPPORTED_TYPES(PROCESS) -#undef PROCESS - } - return arrow::Status::OK(); - } - - template - arrow::Status FinishInternal(ArrayList* out) { - using CType = typename arrow::TypeTraits::CType; - using ScalarType = typename arrow::TypeTraits::ScalarType; - auto typed_scalar = std::dynamic_pointer_cast(scalar_list_[0]); - CType res = typed_scalar->value; - for (size_t i = 1; i < scalar_list_.size(); i++) { - auto typed_scalar = std::dynamic_pointer_cast(scalar_list_[i]); - if (typed_scalar->value > res) res = typed_scalar->value; - } - std::shared_ptr arr_out; - std::shared_ptr scalar_out; - scalar_out = arrow::MakeScalar(res); - RETURN_NOT_OK(arrow::MakeArrayFromScalar(*scalar_out.get(), 1, &arr_out)); - out->push_back(arr_out); - return arrow::Status::OK(); - } - - private: - arrow::compute::FunctionContext* ctx_; - std::shared_ptr data_type_; - std::vector> scalar_list_; - std::unique_ptr array_builder_; -}; - -arrow::Status MaxArrayKernel::Make(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type, - std::shared_ptr* out) { - *out = std::make_shared(ctx, data_type); - return arrow::Status::OK(); -} - -MaxArrayKernel::MaxArrayKernel(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type) { - impl_.reset(new Impl(ctx, data_type)); - kernel_name_ = "MaxArrayKernel"; -} - -arrow::Status MaxArrayKernel::Evaluate(const ArrayList& in) { - return impl_->Evaluate(in); -} - -arrow::Status MaxArrayKernel::Finish(ArrayList* out) { return impl_->Finish(out); } - -/////////////// StddevSampPartialArray //////////////// -class StddevSampPartialArrayKernel::Impl { - public: - Impl(arrow::compute::FunctionContext* ctx, std::shared_ptr data_type) - : ctx_(ctx), data_type_(data_type) {} - virtual ~Impl() {} - - template - arrow::Status getM2(arrow::compute::FunctionContext* ctx, - const arrow::compute::Datum& value, - const arrow::compute::Datum& mean, arrow::compute::Datum* out) { - using MeanCType = typename arrow::TypeTraits::CType; - using MeanScalarType = typename arrow::TypeTraits::ScalarType; - using ValueCType = typename arrow::TypeTraits::CType; - std::shared_ptr mean_scalar = mean.scalar(); - auto mean_typed_scalar = std::dynamic_pointer_cast(mean_scalar); - double mean_res = mean_typed_scalar->value * 1.0; - double m2_res = 0; - - if (!value.is_array()) { - return arrow::Status::Invalid("AggregateKernel expects Array"); - } - auto array = value.make_array(); - auto typed_array = std::static_pointer_cast>(array); - const ValueCType* input = typed_array->raw_values(); - for (int64_t i = 0; i < (*array).length(); i++) { - auto val = input[i]; - if (val) { - m2_res += (input[i] * 1.0 - mean_res) * (input[i] * 1.0 - mean_res); - } - } - *out = arrow::MakeScalar(m2_res); - return arrow::Status::OK(); - } - - arrow::Status M2(arrow::compute::FunctionContext* ctx, const arrow::Array& array, - const arrow::compute::Datum& mean, arrow::compute::Datum* out) { - arrow::compute::Datum value = array.data(); - auto data_type = value.type(); - - if (data_type == nullptr) - return arrow::Status::Invalid("Datum must be array-like"); - else if (!is_integer(data_type->id()) && !is_floating(data_type->id())) - return arrow::Status::Invalid("Datum must contain a NumericType"); - switch (data_type_->id()) { -#define PROCESS(DataType) \ - case DataType::type_id: { \ - RETURN_NOT_OK(getM2(ctx, value, mean, out)); \ - } break; - PROCESS_SUPPORTED_TYPES(PROCESS) -#undef PROCESS - } - return arrow::Status::OK(); - } - - arrow::Status Evaluate(const ArrayList& in) { - arrow::compute::Datum sum_out; - arrow::compute::Datum cnt_out; - arrow::compute::Datum mean_out; - arrow::compute::Datum m2_out; - arrow::compute::CountOptions option(arrow::compute::CountOptions::COUNT_ALL); - RETURN_NOT_OK(arrow::compute::Sum(ctx_, *in[0].get(), &sum_out)); - RETURN_NOT_OK(arrow::compute::Count(ctx_, option, *in[0].get(), &cnt_out)); - RETURN_NOT_OK(arrow::compute::Mean(ctx_, *in[0].get(), &mean_out)); - RETURN_NOT_OK(M2(ctx_, *in[0].get(), mean_out, &m2_out)); - sum_scalar_list_.push_back(sum_out.scalar()); - cnt_scalar_list_.push_back(cnt_out.scalar()); - mean_scalar_list_.push_back(mean_out.scalar()); - m2_scalar_list_.push_back(m2_out.scalar()); - sum_res_data_type_ = sum_out.scalar()->type; - cnt_res_data_type_ = cnt_out.scalar()->type; - mean_res_data_type_ = mean_out.scalar()->type; - m2_res_data_type_ = m2_out.scalar()->type; - return arrow::Status::OK(); - } - -#define PROCESS_INTERNAL(SumDataType, CntDataType) \ - case CntDataType::type_id: { \ - FinishInternal(out); \ - } break; - - arrow::Status Finish(ArrayList* out) { - switch (sum_res_data_type_->id()) { -#define PROCESS(SumDataType) \ - case SumDataType::type_id: { \ - switch (cnt_res_data_type_->id()) { \ - PROCESS_INTERNAL(SumDataType, arrow::UInt64Type) \ - PROCESS_INTERNAL(SumDataType, arrow::Int64Type) \ - PROCESS_INTERNAL(SumDataType, arrow::DoubleType) \ - } \ - } break; - PROCESS_SUPPORTED_TYPES(PROCESS) -#undef PROCESS -#undef PROCESS_INTERNAL - } - return arrow::Status::OK(); - } - - template - arrow::Status FinishInternal(ArrayList* out) { - using SumCType = typename arrow::TypeTraits::CType; - using CntCType = typename arrow::TypeTraits::CType; - using DoubleCType = typename arrow::TypeTraits::CType; - using SumScalarType = typename arrow::TypeTraits::ScalarType; - using CntScalarType = typename arrow::TypeTraits::ScalarType; - using DoubleScalarType = typename arrow::TypeTraits::ScalarType; - SumCType sum_res = 0; - DoubleCType cnt_res = 0; - DoubleCType mean_res = 0; - DoubleCType m2_res = 0; - for (size_t i = 0; i < sum_scalar_list_.size(); i++) { - auto sum_typed_scalar = - std::dynamic_pointer_cast(sum_scalar_list_[i]); - auto cnt_typed_scalar = - std::dynamic_pointer_cast(cnt_scalar_list_[i]); - auto mean_typed_scalar = - std::dynamic_pointer_cast(mean_scalar_list_[i]); - auto m2_typed_scalar = - std::dynamic_pointer_cast(m2_scalar_list_[i]); - if (cnt_typed_scalar->value > 0) { - double pre_avg = sum_res * 1.0 / (cnt_res > 0 ? cnt_res : 1); - double delta = mean_typed_scalar->value - pre_avg; - double newN = (cnt_res + cnt_typed_scalar->value) * 1.0; - double deltaN = newN > 0 ? delta / newN : 0.0; - m2_res += - m2_typed_scalar->value + delta * deltaN * cnt_res * cnt_typed_scalar->value; - sum_res += sum_typed_scalar->value; - cnt_res += cnt_typed_scalar->value * 1.0; - } - } - double avg = 0; - if (cnt_res > 0) { - avg = sum_res * 1.0 / cnt_res; - } else { - m2_res = 0; - } - std::shared_ptr cnt_out; - std::shared_ptr cnt_scalar_out; - cnt_scalar_out = arrow::MakeScalar(cnt_res); - RETURN_NOT_OK(arrow::MakeArrayFromScalar(*cnt_scalar_out.get(), 1, &cnt_out)); - - std::shared_ptr avg_out; - std::shared_ptr avg_scalar_out; - avg_scalar_out = arrow::MakeScalar(avg); - RETURN_NOT_OK(arrow::MakeArrayFromScalar(*avg_scalar_out.get(), 1, &avg_out)); - - std::shared_ptr m2_out; - std::shared_ptr m2_scalar_out; - m2_scalar_out = arrow::MakeScalar(m2_res); - RETURN_NOT_OK(arrow::MakeArrayFromScalar(*m2_scalar_out.get(), 1, &m2_out)); - - out->push_back(cnt_out); - out->push_back(avg_out); - out->push_back(m2_out); - - return arrow::Status::OK(); - } - - private: - arrow::compute::FunctionContext* ctx_; - std::shared_ptr data_type_; - std::shared_ptr sum_res_data_type_; - std::shared_ptr cnt_res_data_type_; - std::shared_ptr mean_res_data_type_; - std::shared_ptr m2_res_data_type_; - std::vector> sum_scalar_list_; - std::vector> cnt_scalar_list_; - std::vector> mean_scalar_list_; - std::vector> m2_scalar_list_; -}; - -arrow::Status StddevSampPartialArrayKernel::Make( - arrow::compute::FunctionContext* ctx, std::shared_ptr data_type, - std::shared_ptr* out) { - *out = std::make_shared(ctx, data_type); - return arrow::Status::OK(); -} - -StddevSampPartialArrayKernel::StddevSampPartialArrayKernel( - arrow::compute::FunctionContext* ctx, std::shared_ptr data_type) { - impl_.reset(new Impl(ctx, data_type)); - kernel_name_ = "StddevSampPartialArrayKernel"; -} - -arrow::Status StddevSampPartialArrayKernel::Evaluate(const ArrayList& in) { - return impl_->Evaluate(in); -} - -arrow::Status StddevSampPartialArrayKernel::Finish(ArrayList* out) { - return impl_->Finish(out); -} - -/////////////// StddevSampFinalArray //////////////// -class StddevSampFinalArrayKernel::Impl { - public: - Impl(arrow::compute::FunctionContext* ctx, std::shared_ptr data_type) - : ctx_(ctx), data_type_(data_type) {} - virtual ~Impl() {} - - arrow::Status getAvgM2(arrow::compute::FunctionContext* ctx, - const arrow::compute::Datum& cnt_value, - const arrow::compute::Datum& avg_value, - const arrow::compute::Datum& m2_value, - arrow::compute::Datum* avg_out, arrow::compute::Datum* m2_out) { - using MeanCType = typename arrow::TypeTraits::CType; - using MeanScalarType = typename arrow::TypeTraits::ScalarType; - using ValueCType = typename arrow::TypeTraits::CType; - - if (!(cnt_value.is_array() && avg_value.is_array() && m2_value.is_array())) { - return arrow::Status::Invalid("AggregateKernel expects Array datum"); - } - - auto cnt_array = cnt_value.make_array(); - auto avg_array = avg_value.make_array(); - auto m2_array = m2_value.make_array(); - - auto cnt_typed_array = std::static_pointer_cast(cnt_array); - auto avg_typed_array = std::static_pointer_cast(avg_array); - auto m2_typed_array = std::static_pointer_cast(m2_array); - const ValueCType* cnt_input = cnt_typed_array->raw_values(); - const MeanCType* avg_input = avg_typed_array->raw_values(); - const MeanCType* m2_input = m2_typed_array->raw_values(); - - double cnt_res = 0; - double avg_res = 0; - double m2_res = 0; - for (int64_t i = 0; i < (*cnt_array).length(); i++) { - double cnt_val = cnt_input[i]; - double avg_val = avg_input[i]; - double m2_val = m2_input[i]; - if (i == 0) { - cnt_res = cnt_val; - avg_res = avg_val; - m2_res = m2_val; - } else { - if (cnt_val > 0) { - double delta = avg_val - avg_res; - double deltaN = (cnt_res + cnt_val) > 0 ? delta / (cnt_res + cnt_val) : 0; - avg_res += deltaN * cnt_val; - m2_res += (m2_val + delta * deltaN * cnt_res * cnt_val); - cnt_res += cnt_val; - } - } - } - *avg_out = arrow::MakeScalar(avg_res); - *m2_out = arrow::MakeScalar(m2_res); - return arrow::Status::OK(); - } - - arrow::Status updateValue(arrow::compute::FunctionContext* ctx, - const arrow::Array& cnt_array, const arrow::Array& avg_array, - const arrow::Array& m2_array, arrow::compute::Datum* avg_out, - arrow::compute::Datum* m2_out) { - arrow::compute::Datum cnt_value = cnt_array.data(); - arrow::compute::Datum avg_value = avg_array.data(); - arrow::compute::Datum m2_value = m2_array.data(); - auto cnt_data_type = cnt_value.type(); - if (cnt_data_type == nullptr) - return arrow::Status::Invalid("Datum must be array-like"); - else if (!is_integer(cnt_data_type->id()) && !is_floating(cnt_data_type->id())) - return arrow::Status::Invalid("Datum must contain a NumericType"); - RETURN_NOT_OK(getAvgM2(ctx, cnt_value, avg_value, m2_value, avg_out, m2_out)); - return arrow::Status::OK(); - } - - arrow::Status Evaluate(const ArrayList& in) { - arrow::compute::Datum cnt_out; - arrow::compute::Datum avg_out; - arrow::compute::Datum m2_out; - RETURN_NOT_OK(arrow::compute::Sum(ctx_, *in[0].get(), &cnt_out)); - RETURN_NOT_OK( - updateValue(ctx_, *in[0].get(), *in[1].get(), *in[2].get(), &avg_out, &m2_out)); - cnt_scalar_list_.push_back(cnt_out.scalar()); - avg_scalar_list_.push_back(avg_out.scalar()); - m2_scalar_list_.push_back(m2_out.scalar()); - cnt_res_data_type_ = cnt_out.scalar()->type; - return arrow::Status::OK(); - } - - arrow::Status Finish(ArrayList* out) { - using DoubleCType = typename arrow::TypeTraits::CType; - using DoubleScalarType = typename arrow::TypeTraits::ScalarType; - - DoubleCType cnt_res = 0; - DoubleCType avg_res = 0; - DoubleCType m2_res = 0; - for (size_t i = 0; i < cnt_scalar_list_.size(); i++) { - auto cnt_typed_scalar = - std::dynamic_pointer_cast(cnt_scalar_list_[i]); - auto avg_typed_scalar = - std::dynamic_pointer_cast(avg_scalar_list_[i]); - auto m2_typed_scalar = - std::dynamic_pointer_cast(m2_scalar_list_[i]); - if (i == 0) { - cnt_res = cnt_typed_scalar->value; - avg_res = avg_typed_scalar->value; - m2_res = m2_typed_scalar->value; - } else { - if (cnt_typed_scalar->value > 0) { - double delta = avg_typed_scalar->value - avg_res; - double newN = cnt_res + cnt_typed_scalar->value; - double deltaN = newN > 0 ? delta / newN : 0; - avg_res += deltaN * cnt_typed_scalar->value; - m2_res += (m2_typed_scalar->value + - delta * deltaN * cnt_res * cnt_typed_scalar->value); - cnt_res += cnt_typed_scalar->value; - } - } - } - - std::shared_ptr stddev_samp_out; - std::shared_ptr stddev_samp_scalar_out; - if (cnt_res - 1 < 0.00001) { - double stddev_samp = std::numeric_limits::quiet_NaN(); - stddev_samp_scalar_out = arrow::MakeScalar(stddev_samp); - } else if (cnt_res < 0.00001) { - stddev_samp_scalar_out = MakeNullScalar(arrow::float64()); - } else { - double stddev_samp = sqrt(m2_res / (cnt_res > 1 ? (cnt_res - 1) : 1)); - stddev_samp_scalar_out = arrow::MakeScalar(stddev_samp); - } - RETURN_NOT_OK( - arrow::MakeArrayFromScalar(*stddev_samp_scalar_out.get(), 1, &stddev_samp_out)); - out->push_back(stddev_samp_out); - - return arrow::Status::OK(); - } - - private: - arrow::compute::FunctionContext* ctx_; - std::shared_ptr data_type_; - std::shared_ptr cnt_res_data_type_; - std::vector> cnt_scalar_list_; - std::vector> avg_scalar_list_; - std::vector> m2_scalar_list_; -}; - -arrow::Status StddevSampFinalArrayKernel::Make(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type, - std::shared_ptr* out) { - *out = std::make_shared(ctx, data_type); - return arrow::Status::OK(); -} - -StddevSampFinalArrayKernel::StddevSampFinalArrayKernel( - arrow::compute::FunctionContext* ctx, std::shared_ptr data_type) { - impl_.reset(new Impl(ctx, data_type)); - kernel_name_ = "StddevSampFinalArrayKernel"; -} - -arrow::Status StddevSampFinalArrayKernel::Evaluate(const ArrayList& in) { - return impl_->Evaluate(in); -} - -arrow::Status StddevSampFinalArrayKernel::Finish(ArrayList* out) { - return impl_->Finish(out); -} - -#undef PROCESS_SUPPORTED_TYPES - /////////////// EncodeArray //////////////// class EncodeArrayKernel::Impl { public: diff --git a/cpp/src/codegen/arrow_compute/ext/kernels_ext.h b/cpp/src/codegen/arrow_compute/ext/kernels_ext.h index 203ca5e09..3f5c6b9d1 100644 --- a/cpp/src/codegen/arrow_compute/ext/kernels_ext.h +++ b/cpp/src/codegen/arrow_compute/ext/kernels_ext.h @@ -105,28 +105,6 @@ class KernalBase { std::string kernel_name_; }; -class SplitArrayListWithActionKernel : public KernalBase { - public: - static arrow::Status Make(arrow::compute::FunctionContext* ctx, - std::vector action_name_list, - std::vector> type_list, - std::shared_ptr* out); - SplitArrayListWithActionKernel(arrow::compute::FunctionContext* ctx, - std::vector action_name_list, - std::vector> type_list); - arrow::Status Evaluate(const ArrayList& in, - const std::shared_ptr& dict) override; - arrow::Status Finish(ArrayList* out) override; - arrow::Status MakeResultIterator( - std::shared_ptr schema, - std::shared_ptr>* out) override; - - private: - class Impl; - std::unique_ptr impl_; - arrow::compute::FunctionContext* ctx_; -}; - class EncodeArrayKernel : public KernalBase { public: static arrow::Status Make(arrow::compute::FunctionContext* ctx, @@ -184,134 +162,6 @@ class HashArrayKernel : public KernalBase { arrow::compute::FunctionContext* ctx_; }; -class SumArrayKernel : public KernalBase { - public: - static arrow::Status Make(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type, - std::shared_ptr* out); - SumArrayKernel(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type); - arrow::Status Evaluate(const ArrayList& in) override; - arrow::Status Finish(ArrayList* out) override; - - private: - class Impl; - std::unique_ptr impl_; - arrow::compute::FunctionContext* ctx_; -}; - -class CountArrayKernel : public KernalBase { - public: - static arrow::Status Make(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type, - std::shared_ptr* out); - CountArrayKernel(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type); - arrow::Status Evaluate(const ArrayList& in) override; - arrow::Status Finish(ArrayList* out) override; - - private: - class Impl; - std::unique_ptr impl_; - arrow::compute::FunctionContext* ctx_; -}; - -class SumCountArrayKernel : public KernalBase { - public: - static arrow::Status Make(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type, - std::shared_ptr* out); - SumCountArrayKernel(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type); - arrow::Status Evaluate(const ArrayList& in) override; - arrow::Status Finish(ArrayList* out) override; - - private: - class Impl; - std::unique_ptr impl_; - arrow::compute::FunctionContext* ctx_; -}; - -class AvgByCountArrayKernel : public KernalBase { - public: - static arrow::Status Make(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type, - std::shared_ptr* out); - AvgByCountArrayKernel(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type); - arrow::Status Evaluate(const ArrayList& in) override; - arrow::Status Finish(ArrayList* out) override; - - private: - class Impl; - std::unique_ptr impl_; - arrow::compute::FunctionContext* ctx_; -}; - -class MinArrayKernel : public KernalBase { - public: - static arrow::Status Make(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type, - std::shared_ptr* out); - MinArrayKernel(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type); - arrow::Status Evaluate(const ArrayList& in) override; - arrow::Status Finish(ArrayList* out) override; - - private: - class Impl; - std::unique_ptr impl_; - arrow::compute::FunctionContext* ctx_; -}; - -class MaxArrayKernel : public KernalBase { - public: - static arrow::Status Make(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type, - std::shared_ptr* out); - MaxArrayKernel(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type); - arrow::Status Evaluate(const ArrayList& in) override; - arrow::Status Finish(ArrayList* out) override; - - private: - class Impl; - std::unique_ptr impl_; - arrow::compute::FunctionContext* ctx_; -}; - -class StddevSampPartialArrayKernel : public KernalBase { - public: - static arrow::Status Make(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type, - std::shared_ptr* out); - StddevSampPartialArrayKernel(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type); - arrow::Status Evaluate(const ArrayList& in) override; - arrow::Status Finish(ArrayList* out) override; - - private: - class Impl; - std::unique_ptr impl_; - arrow::compute::FunctionContext* ctx_; -}; - -class StddevSampFinalArrayKernel : public KernalBase { - public: - static arrow::Status Make(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type, - std::shared_ptr* out); - StddevSampFinalArrayKernel(arrow::compute::FunctionContext* ctx, - std::shared_ptr data_type); - arrow::Status Evaluate(const ArrayList& in) override; - arrow::Status Finish(ArrayList* out) override; - - private: - class Impl; - std::unique_ptr impl_; - arrow::compute::FunctionContext* ctx_; -}; - class SortArraysToIndicesKernel : public KernalBase { public: static arrow::Status Make(arrow::compute::FunctionContext* ctx, diff --git a/cpp/src/precompile/gandiva_projector.cc b/cpp/src/precompile/gandiva_projector.cc index a253da13e..5c04a8c27 100644 --- a/cpp/src/precompile/gandiva_projector.cc +++ b/cpp/src/precompile/gandiva_projector.cc @@ -21,6 +21,17 @@ class GandivaProjector::Impl { RETURN_NOT_OK(gandiva::Projector::Make(schema_, exprs, configuration, &projector_)); return arrow::Status::OK(); } + + arrow::ArrayVector Evaluate(const arrow::ArrayVector& in) { + arrow::ArrayVector outputs; + if (in.size() > 0) { + auto length = in[0]->length(); + auto in_batch = arrow::RecordBatch::Make(schema_, length, in); + THROW_NOT_OK(projector_->Evaluate(*in_batch.get(), ctx_->memory_pool(), &outputs)); + } + return outputs; + } + arrow::Status Evaluate(arrow::ArrayVector* in) { if ((*in).size() > 0) { arrow::ArrayVector outputs; @@ -46,4 +57,8 @@ GandivaProjector::GandivaProjector(arrow::compute::FunctionContext* ctx, arrow::Status GandivaProjector::Evaluate(arrow::ArrayVector* in) { return impl_->Evaluate(in); +} + +arrow::ArrayVector GandivaProjector::Evaluate(const arrow::ArrayVector& in) { + return impl_->Evaluate(in); } \ No newline at end of file diff --git a/cpp/src/precompile/gandiva_projector.h b/cpp/src/precompile/gandiva_projector.h index 7ac22b5df..318991157 100644 --- a/cpp/src/precompile/gandiva_projector.h +++ b/cpp/src/precompile/gandiva_projector.h @@ -13,6 +13,7 @@ class GandivaProjector { GandivaProjector(arrow::compute::FunctionContext* ctx, gandiva::SchemaPtr input_schema, gandiva::ExpressionVector exprs); arrow::Status Evaluate(arrow::ArrayVector* in); + arrow::ArrayVector Evaluate(const arrow::ArrayVector& in); private: class Impl; diff --git a/cpp/src/tests/CMakeLists.txt b/cpp/src/tests/CMakeLists.txt index eb87fa6ea..230422c4f 100644 --- a/cpp/src/tests/CMakeLists.txt +++ b/cpp/src/tests/CMakeLists.txt @@ -1,5 +1,4 @@ package_add_test(TestArrowComputeAggregate arrow_compute_test_aggregate.cc) -package_add_test(TestArrowComputeHashAggregate arrow_compute_test_hashaggregate.cc) package_add_test(TestArrowComputeJoin arrow_compute_test_join.cc) package_add_test(TestArrowComputeMergeJoin arrow_compute_test_join_smj.cc) package_add_test(TestArrowComputeSort arrow_compute_test_sort.cc) diff --git a/cpp/src/tests/arrow_compute_test_aggregate.cc b/cpp/src/tests/arrow_compute_test_aggregate.cc index 1a5799395..7b1832e47 100644 --- a/cpp/src/tests/arrow_compute_test_aggregate.cc +++ b/cpp/src/tests/arrow_compute_test_aggregate.cc @@ -27,776 +27,491 @@ namespace sparkcolumnarplugin { namespace codegen { -TEST(TestArrowCompute, AggregatewithMultipleBatchTest) { +TEST(TestArrowCompute, AggregateTest) { ////////////////////// prepare expr_vector /////////////////////// - auto f0 = field("f0", uint32()); - auto f1 = field("f1", uint32()); - auto f2 = field("f2", uint64()); - auto f_sum = field("sum", uint64()); - auto f_count = field("count", int32()); - auto f_float = field("float", float64()); - auto f_res = field("res", uint32()); + auto f0 = field("f0", int32()); + auto f1 = field("f1", int32()); + auto f2 = field("f2", float64()); + auto f3 = field("f3", float64()); + auto f4 = field("f4", float64()); + auto arg_0 = TreeExprBuilder::MakeField(f0); auto arg_1 = TreeExprBuilder::MakeField(f1); auto arg_2 = TreeExprBuilder::MakeField(f2); - auto n_sum = TreeExprBuilder::MakeFunction("sum", {arg_0}, uint64()); - auto n_count = TreeExprBuilder::MakeFunction("count", {arg_0}, uint64()); - auto n_sum_count = TreeExprBuilder::MakeFunction("sum_count", {arg_0}, uint64()); - auto n_avg = TreeExprBuilder::MakeFunction("avgByCount", {arg_2, arg_1}, uint64()); - auto n_min = TreeExprBuilder::MakeFunction("min", {arg_0}, uint64()); - auto n_max = TreeExprBuilder::MakeFunction("max", {arg_0}, uint64()); - auto n_stddev_samp_partial = - TreeExprBuilder::MakeFunction("stddev_samp_partial", {arg_0}, float64()); - - auto sum_expr = TreeExprBuilder::MakeExpression(n_sum, f_res); - auto count_expr = TreeExprBuilder::MakeExpression(n_count, f_res); - auto sum_count_expr = TreeExprBuilder::MakeExpression(n_sum_count, f_res); - auto avg_expr = TreeExprBuilder::MakeExpression(n_avg, f_res); - auto min_expr = TreeExprBuilder::MakeExpression(n_min, f_res); - auto max_expr = TreeExprBuilder::MakeExpression(n_max, f_res); - auto stddev_samp_partial_expr = - TreeExprBuilder::MakeExpression(n_stddev_samp_partial, f_res); - - std::vector> expr_vector = { - sum_expr, count_expr, sum_count_expr, avg_expr, - min_expr, max_expr, stddev_samp_partial_expr}; - auto sch = arrow::schema({f0, f1, f2}); + auto arg_3 = TreeExprBuilder::MakeField(f3); + auto arg_4 = TreeExprBuilder::MakeField(f4); + + auto n_sum = TreeExprBuilder::MakeFunction("action_sum", {arg_0}, int64()); + auto n_count = TreeExprBuilder::MakeFunction("action_count", {arg_0}, int64()); + auto n_sum_count = TreeExprBuilder::MakeFunction("action_sum_count", {arg_0}, int64()); + auto n_avg = + TreeExprBuilder::MakeFunction("action_avgByCount", {arg_2, arg_1}, float64()); + auto n_min = TreeExprBuilder::MakeFunction("action_min", {arg_0}, int64()); + auto n_max = TreeExprBuilder::MakeFunction("action_max", {arg_0}, int64()); + auto n_stddev = TreeExprBuilder::MakeFunction("action_stddev_samp_final", + {arg_2, arg_3, arg_4}, float64()); + auto n_count_literal = + TreeExprBuilder::MakeFunction("action_countLiteral_1", {}, int64()); + + auto f_sum = field("sum", int64()); + auto f_count = field("count", int64()); + auto f_sum_count = field("sum_count", int64()); + auto f_avg = field("avg", float64()); + auto f_min = field("min", int64()); + auto f_max = field("max", int64()); + auto f_stddev = field("stddev", float64()); + auto f_count_literal = field("count_all", int64()); + auto f_res = field("res", int32()); + + auto n_proj = TreeExprBuilder::MakeFunction( + "aggregateExpressions", {arg_0, arg_1, arg_2, arg_3, arg_4}, uint32()); + auto n_action = TreeExprBuilder::MakeFunction( + "aggregateActions", + {n_sum, n_count, n_sum_count, n_avg, n_min, n_max, n_stddev, n_count_literal}, + uint32()); + + auto n_aggr = + TreeExprBuilder::MakeFunction("hashAggregateArrays", {n_proj, n_action}, uint32()); + auto n_child = TreeExprBuilder::MakeFunction("standalone", {n_aggr}, uint32()); + auto aggr_expr = TreeExprBuilder::MakeExpression(n_child, f_res); + + auto sch = arrow::schema({f0, f1, f2, f3, f4}); std::vector> ret_types = { - f_sum, f_count, f_sum, f_count, f_float, f_res, f_res, f_float, f_float, f_float}; + f_sum, f_count, f_sum, f_count, f_avg, f_min, f_max, f_stddev, f_count_literal}; ///////////////////// Calculation ////////////////// std::shared_ptr expr; arrow::compute::FunctionContext ctx; - ASSERT_NOT_OK(CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)); + ASSERT_NOT_OK( + CreateCodeGenerator(ctx.memory_pool(), sch, {aggr_expr}, ret_types, &expr, true)); + + std::shared_ptr> aggr_result_iterator; + std::shared_ptr aggr_result_iterator_base; + ASSERT_NOT_OK(expr->finish(&aggr_result_iterator_base)); + aggr_result_iterator = std::dynamic_pointer_cast>( + aggr_result_iterator_base); + std::shared_ptr input_batch; - std::vector> result_batch; - std::vector input_data_string = {"[8, 10, 9, 20, 55, 42, 28, 32, 54, 70]", - "[1, 1, 1, 1, 1, 1, 1, 1, 1, 1]", - "[8, 10, 9, 20, 55, 42, 28, 32, 54, 70]"}; + std::vector input_data_string = { + "[1, 2, 3, 4, 5, null, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]", + "[1, 2, 3, 4, 5, null, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]", + "[2, 4, 5, 7, 8, 2, 45, 32, 23, 12, 14, 16, 18, 19, 23, 25, 57, 59, 12, 1]", + "[2, 4, 5, 7, 8, 2, 45, 32, 23, 12, 14, 16, 18, 19, 23, 25, 57, 59, 12, 1]", + "[2, 4, 5, 7, 8, 2, 45, 32, 23, 12, 14, 16, 18, 19, 23, 25, 57, 59, 12, 1]"}; MakeInputBatch(input_data_string, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &result_batch)); + ASSERT_NOT_OK(aggr_result_iterator->ProcessAndCacheOne(input_batch->columns())); std::vector input_data_2_string = { - "[8, 10, 9, 20, null, 42, 28, 32, 54, 70]", "[1, 1, 1, 1, 1, 1, 1, 1, 1, 1]", - "[8, 10, 9, 20, null, 42, 28, 32, 54, 70]"}; + "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]", + "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]", + "[7, 8, 4, 5, 6, 1, 34, 54, 65, 66, 78, 12, 32, 24, 32, 45, 12, 24, 35, 46]", + "[2, 4, 5, 7, 8, 2, 45, 32, 23, 12, 14, 16, 18, 19, 23, 25, 57, 59, 12, 1]", + "[2, 4, 5, 7, 8, 2, 45, 32, 23, 12, 14, 16, 18, 19, 23, 25, 57, 59, 12, 1]"}; MakeInputBatch(input_data_2_string, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &result_batch)); - ASSERT_NOT_OK(expr->finish(&result_batch)); + ASSERT_NOT_OK(aggr_result_iterator->ProcessAndCacheOne(input_batch->columns())); std::shared_ptr expected_result; + std::shared_ptr result_batch; std::vector expected_result_string = { - "[601]", "[19]", "[601]", "[19]", "[30.05]", - "[8]", "[70]", "[19]", "[31.6316]", "[8080.42]"}; - auto res_sch = arrow::schema( - {f_sum, f_count, f_sum, f_count, f_float, f_res, f_res, f_float, f_float, f_float}); + "[221]", "[39]", "[221]", "[39]", "[4.40724]", "[1]", "[10]", "[17.2996]", "[40]"}; + auto res_sch = arrow::schema(ret_types); MakeInputBatch(expected_result_string, res_sch, &expected_result); - ASSERT_NOT_OK(Equals(*expected_result.get(), *(result_batch[0]).get())); + if (aggr_result_iterator->HasNext()) { + ASSERT_NOT_OK(aggr_result_iterator->Next(&result_batch)); + ASSERT_NOT_OK(Equals(*expected_result.get(), *result_batch.get())); + } } -TEST(TestArrowCompute, GroupByAggregateWithMultipleBatchTest) { +TEST(TestArrowCompute, GroupByAggregateTest) { ////////////////////// prepare expr_vector /////////////////////// - auto f0 = field("f0", uint32()); + auto f0 = field("f0", int64()); auto f1 = field("f1", uint32()); - auto f_unique = field("unique", uint32()); - auto f_sum = field("sum", uint32()); - auto f_count = field("count", uint32()); - auto f_avg = field("avg", uint32()); - auto f_res = field("res", uint32()); - auto f_m2 = field("m2", uint32()); + auto f2 = field("f2", float64()); + auto f3 = field("f3", float64()); + auto f4 = field("f4", float64()); - auto arg_pre = TreeExprBuilder::MakeField(f0); - auto n_pre = TreeExprBuilder::MakeFunction("encodeArray", {arg_pre}, uint32()); + auto f_unique = field("unique", int64()); + auto f_sum = field("sum", float64()); + auto f_sum_count_multiply = field("sum_count_multiply", float64()); + auto f_count = field("count", int64()); + auto f_min = field("min", uint32()); + auto f_max = field("max", uint32()); + auto f_avg = field("avg", float64()); + auto f_stddev = field("stddev", float64()); + auto f_res = field("res", uint32()); auto arg0 = TreeExprBuilder::MakeField(f0); auto arg1 = TreeExprBuilder::MakeField(f1); - auto n_split = TreeExprBuilder::MakeFunction("splitArrayListWithAction", - {n_pre, arg0, arg1}, uint32()); - auto arg_res = TreeExprBuilder::MakeField(f_res); - auto n_unique = - TreeExprBuilder::MakeFunction("action_unique", {n_split, arg0}, uint32()); - auto n_sum = TreeExprBuilder::MakeFunction("action_sum", {n_split, arg1}, uint32()); - auto n_count = TreeExprBuilder::MakeFunction("action_count", {n_split, arg1}, uint32()); - auto n_avg = TreeExprBuilder::MakeFunction("action_avg", {n_split, arg1}, uint32()); - auto n_sum_count = - TreeExprBuilder::MakeFunction("action_sum_count", {n_split, arg1}, uint32()); - auto n_min = TreeExprBuilder::MakeFunction("action_min", {n_split, arg1}, uint32()); - auto n_max = TreeExprBuilder::MakeFunction("action_max", {n_split, arg1}, uint32()); - auto n_stddev_samp_partial = TreeExprBuilder::MakeFunction("action_stddev_samp_partial", - {n_split, arg1}, uint32()); - - auto unique_expr = TreeExprBuilder::MakeExpression(n_unique, f_res); - auto sum_expr = TreeExprBuilder::MakeExpression(n_sum, f_res); - auto count_expr = TreeExprBuilder::MakeExpression(n_count, f_res); - auto avg_expr = TreeExprBuilder::MakeExpression(n_avg, f_res); - auto sum_count_expr = TreeExprBuilder::MakeExpression(n_sum_count, f_res); - auto avg_min = TreeExprBuilder::MakeExpression(n_min, f_res); - auto avg_max = TreeExprBuilder::MakeExpression(n_max, f_res); - auto stddev_samp_partial_expr = - TreeExprBuilder::MakeExpression(n_stddev_samp_partial, f_res); - - std::vector> expr_vector = { - unique_expr, sum_expr, count_expr, avg_expr, - sum_count_expr, avg_min, avg_max, stddev_samp_partial_expr}; - auto sch = arrow::schema({f0, f1}); - std::vector> ret_types = {f_unique, f_sum, f_count, f_avg, - f_sum, f_count, f_res, f_res, - f_count, f_avg, f_m2}; + auto arg2 = TreeExprBuilder::MakeField(f2); + auto arg3 = TreeExprBuilder::MakeField(f3); + auto arg4 = TreeExprBuilder::MakeField(f4); + + auto n_groupby = TreeExprBuilder::MakeFunction("action_groupby", {arg0}, uint32()); + auto n_sum_count = TreeExprBuilder::MakeFunction("action_sum_count", {arg1}, uint32()); + auto n_min = TreeExprBuilder::MakeFunction("action_min", {arg1}, uint32()); + auto n_max = TreeExprBuilder::MakeFunction("action_max", {arg1}, uint32()); + auto n_avg = TreeExprBuilder::MakeFunction("action_avgByCount", {arg2, arg0}, uint32()); + auto n_stddev = TreeExprBuilder::MakeFunction("action_stddev_samp_final", + {arg2, arg3, arg4}, uint32()); + auto n_proj = TreeExprBuilder::MakeFunction("aggregateExpressions", + {arg0, arg1, arg2, arg3, arg4}, uint32()); + auto n_action = TreeExprBuilder::MakeFunction( + "aggregateActions", {n_groupby, n_sum_count, n_min, n_max, n_avg, n_stddev}, + uint32()); + auto n_result = TreeExprBuilder::MakeFunction( + "resultSchema", + {TreeExprBuilder::MakeField(f_unique), TreeExprBuilder::MakeField(f_sum), + TreeExprBuilder::MakeField(f_count), TreeExprBuilder::MakeField(f_min), + TreeExprBuilder::MakeField(f_max), TreeExprBuilder::MakeField(f_avg), + TreeExprBuilder::MakeField(f_stddev)}, + uint32()); + auto n_multiply = TreeExprBuilder::MakeFunction( + "multiply", + { + TreeExprBuilder::MakeField(f_sum), + TreeExprBuilder::MakeFunction("castFloat8", + {TreeExprBuilder::MakeField(f_count)}, float64()), + }, + float64()); + auto n_result_expr = TreeExprBuilder::MakeFunction( + "resultExpressions", + {TreeExprBuilder::MakeField(f_unique), n_multiply, + TreeExprBuilder::MakeField(f_min), TreeExprBuilder::MakeField(f_avg), + TreeExprBuilder::MakeField(f_stddev)}, + uint32()); + auto n_aggr = TreeExprBuilder::MakeFunction( + "hashAggregateArrays", {n_proj, n_action, n_result, n_result_expr}, uint32()); + auto n_child = TreeExprBuilder::MakeFunction("standalone", {n_aggr}, uint32()); + auto aggr_expr = TreeExprBuilder::MakeExpression(n_child, f_res); + + std::vector> expr_vector = {aggr_expr}; + + auto sch = arrow::schema({f0, f1, f2, f3, f4}); + std::vector> ret_types = {f_unique, f_sum_count_multiply, f_min, + f_avg, f_stddev}; /////////////////////// Create Expression Evaluator //////////////////// std::shared_ptr expr; arrow::compute::FunctionContext ctx; - ASSERT_NOT_OK(CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)); + ASSERT_NOT_OK( + CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)); std::shared_ptr input_batch; std::vector> output_batch_list; + std::shared_ptr> aggr_result_iterator; + std::shared_ptr aggr_result_iterator_base; + ASSERT_NOT_OK(expr->finish(&aggr_result_iterator_base)); + aggr_result_iterator = std::dynamic_pointer_cast>( + aggr_result_iterator_base); + ////////////////////// calculation ///////////////////// std::vector input_data = { "[1, 2, 3, 4, 5, null, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]", - "[1, 2, 3, 4, 5, 5, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]"}; + "[1, 2, 3, 4, 5, 5, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]", + "[2, 4, 5, 7, 8, 2, 45, 32, 23, 12, 14, 16, 18, 19, 23, 25, 57, 59, 12, 1]", + "[2, 4, 5, 7, 8, 2, 45, 32, 23, 12, 14, 16, 18, 19, 23, 25, 57, 59, 12, 1]", + "[2, 4, 5, 7, 8, 2, 45, 32, 23, 12, 14, 16, 18, 19, 23, 25, 57, 59, 12, 1]"}; MakeInputBatch(input_data, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); + ASSERT_NOT_OK(aggr_result_iterator->ProcessAndCacheOne(input_batch->columns())); std::vector input_data_2 = { "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]", - "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]"}; + "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]", + "[7, 8, 4, 5, 6, 1, 34, 54, 65, 66, 78, 12, 32, 24, 32, 45, 12, 24, 35, 46]", + "[2, 4, 5, 7, 8, 2, 45, 32, 23, 12, 14, 16, 18, 19, 23, 25, 57, 59, 12, 1]", + "[2, 4, 5, 7, 8, 2, 45, 32, 23, 12, 14, 16, 18, 19, 23, 25, 57, 59, 12, 1]"}; MakeInputBatch(input_data_2, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_3 = { - "[1, 2, 3, 8, 5, 5, 10, 1, 2, 7, 6, 6, 1, 9, 4, 9, null, 8, 5, 5]", - "[1, 2, 3, 8, 5, 5, 10, 1, 2, 7, 6, 6, 1, 9, 4, 9, 5, 8, 5, 5]"}; - MakeInputBatch(input_data_3, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); + ASSERT_NOT_OK(aggr_result_iterator->ProcessAndCacheOne(input_batch->columns())); ////////////////////// Finish ////////////////////////// - std::vector> result_batch; - ASSERT_NOT_OK(expr->finish(&result_batch)); - + std::shared_ptr result_batch; std::shared_ptr expected_result; std::vector expected_result_string = { - "[1, 2, 3, 4, 5, null, 6, 7, 8 ,9, 10]", - "[8, 10, 9, 20, 45, 10, 42, 28, 32, 54, 70]", - "[8, 5, 3, 5, 9, 2, 7, 4, 4, 6, 7]", - "[1, 2, 3, 4, 5, 5, 6, 7, 8, 9, 10]", - "[8, 10, 9, 20, 45, 10, 42, 28, 32, 54, 70]", - "[8, 5, 3, 5, 9, 2, 7, 4, 4, 6, 7]", - "[1, 2, 3, 4, 5, 5, 6, 7, 8, 9, 10]", + "[1, 2, 3, 4, 5, null, 6, 7, 8, 9, 10]", + "[25, 18, 12, 64, 125, 5, 150, 63, 32, 144, 360]", "[1, 2, 3, 4, 5, 5, 6, 7, 8, 9, 10]", - "[8, 5, 3, 5, 9, 2, 7, 4, 4, 6, 7]", - "[1, 2, 3, 4, 5, 5, 6, 7, 8, 9, 10]", - "[0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0]"}; - auto res_sch = arrow::schema({f_unique, f_sum, f_count, f_avg, f_sum, f_count, f_res, - f_res, f_count, f_avg, f_m2}); + "[16.4, 6.5, 5, 5.875, 5.48, null, 6.1, 6.61905, 3.0625, 2.63889, 2.06667]", + "[8.49255, 6.93137, 7.6489, 13.5708, 17.4668, 1.41421, 8.52779, 6.23633, 5.58903, " + "12.535, 24.3544]"}; + auto res_sch = arrow::schema(ret_types); MakeInputBatch(expected_result_string, res_sch, &expected_result); - ASSERT_NOT_OK(Equals(*expected_result.get(), *(result_batch[0]).get())); + if (aggr_result_iterator->HasNext()) { + ASSERT_NOT_OK(aggr_result_iterator->Next(&result_batch)); + ASSERT_NOT_OK(Equals(*expected_result.get(), *result_batch.get())); + } } -TEST(TestArrowCompute, GroupByAvgWithMultipleBatchTest) { - ////////////////////// prepare expr_vector /////////////////////// +TEST(TestArrowCompute, GroupByCountAll) { auto f0 = field("f0", utf8()); - auto f_sum = field("sum", float64()); - auto f_count = field("count", int64()); auto f_unique = field("unique", utf8()); - auto f_avg = field("avg", float64()); + auto f_count = field("count", int64()); auto f_res = field("res", uint32()); auto arg0 = TreeExprBuilder::MakeField(f0); - auto n_pre = TreeExprBuilder::MakeFunction("encodeArray", {arg0}, utf8()); - - auto arg_sum = TreeExprBuilder::MakeField(f_sum); - auto arg_count = TreeExprBuilder::MakeField(f_count); - auto n_split = TreeExprBuilder::MakeFunction( - "splitArrayListWithAction", {n_pre, arg0, arg_sum, arg_count}, uint32()); - auto arg_res = TreeExprBuilder::MakeField(f_res); - auto n_unique = TreeExprBuilder::MakeFunction("action_unique", {n_split, arg0}, utf8()); - auto n_avg = TreeExprBuilder::MakeFunction("action_avgByCount", - {n_split, arg_sum, arg_count}, uint32()); - - auto unique_expr = TreeExprBuilder::MakeExpression(n_unique, f_res); - auto avg_expr = TreeExprBuilder::MakeExpression(n_avg, f_res); - - std::vector> expr_vector = {unique_expr, - avg_expr}; - auto sch = arrow::schema({f0, f_sum, f_count}); - std::vector> ret_types = {f_unique, f_avg}; - - /////////////////////// Create Expression Evaluator //////////////////// - std::shared_ptr expr; - arrow::compute::FunctionContext ctx; - ASSERT_NOT_OK(CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)); - std::shared_ptr input_batch; - std::vector> output_batch_list; - - ////////////////////// calculation ///////////////////// - std::vector input_data = { - R"(["BJ", "SH", "SZ", "HZ", "WH", "WH", "HZ", "BJ", "SH", "SH", "BJ", "BJ", "BJ", "HZ", "HZ", "SZ", "WH", "WH", "WH", "WH"])", - "[1, 4, 9, 16, 25, 25, 16, 1, 4, 4, 1, 1, 1, 16, 16, 9, 25, 25, 25, 25]", - "[1, 2, 3, 4, 5, 5, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]"}; - MakeInputBatch(input_data, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_2 = { - R"(["CD", "DL", "NY", "LA", "AU", "AU", "LA", "CD", "DL", "DL", "CD", "CD", "CD", "LA", "LA", "NY", "AU", "AU", "AU", "AU"])", - "[36, 49, 64, 81, 100, 100, 81, 36, 49, 49, 36, 36, 36, 81, 81, 64, 100, 100, 100, " - "100]", - "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]"}; - MakeInputBatch(input_data_2, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_3 = { - R"(["BJ", "SH", "SZ", "NY", "WH", "WH", "AU", "BJ", "SH", "DL", "CD", "CD", "BJ", "LA", "HZ", "LA", "WH", "NY", "WH", "WH"])", - "[1, 4, 9, 64, 25, 25, 100, 1, 4, 49, 36, 36, 1, 81, 16, 81, 25, 64, 25, 25]", - "[1, 2, 3, 8, 5, 5, 10, 1, 2, 7, 6, 6, 1, 9, 4, 9, 5, 8, 5, 5]"}; - MakeInputBatch(input_data_3, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - ////////////////////// Finish ////////////////////////// - std::vector> result_batch; - ASSERT_NOT_OK(expr->finish(&result_batch)); - - std::shared_ptr expected_result; - std::vector expected_result_string = { - R"(["BJ", "SH", "SZ", "HZ", "WH", "CD", "DL", "NY" ,"LA", "AU"])", - "[1, 2, 3, 4, 5, 6, 7, 8, 9, 10]"}; - auto res_sch = arrow::schema({f_unique, f_avg}); - MakeInputBatch(expected_result_string, res_sch, &expected_result); - ASSERT_NOT_OK(Equals(*expected_result.get(), *(result_batch[0]).get())); -} - -TEST(TestArrowCompute, GroupByCountAllWithMultipleBatchTest) { - ////////////////////// prepare expr_vector /////////////////////// - auto f0 = field("f0", utf8()); - auto f_unique = field("unique", utf8()); - auto f_count = field("avg", uint64()); - - auto arg0 = TreeExprBuilder::MakeField(f0); - auto n_pre = TreeExprBuilder::MakeFunction("encodeArray", {arg0}, utf8()); - auto n_split = - TreeExprBuilder::MakeFunction("splitArrayListWithAction", {n_pre, arg0}, uint32()); + auto n_groupby = TreeExprBuilder::MakeFunction("action_groupby", {arg0}, uint32()); + auto n_count = TreeExprBuilder::MakeFunction("action_countLiteral_1", {arg0}, uint32()); + auto n_proj = TreeExprBuilder::MakeFunction("aggregateExpressions", {arg0}, uint32()); + auto n_action = + TreeExprBuilder::MakeFunction("aggregateActions", {n_groupby, n_count}, uint32()); - auto n_unique = TreeExprBuilder::MakeFunction("action_unique", {n_split, arg0}, utf8()); - auto n_count = - TreeExprBuilder::MakeFunction("action_countLiteral_1", {n_split}, uint32()); + auto n_aggr = + TreeExprBuilder::MakeFunction("hashAggregateArrays", {n_proj, n_action}, uint32()); + auto n_child = TreeExprBuilder::MakeFunction("standalone", {n_aggr}, uint32()); + auto aggr_expr = TreeExprBuilder::MakeExpression(n_child, f_res); - auto unique_expr = TreeExprBuilder::MakeExpression(n_unique, f_unique); - auto count_expr = TreeExprBuilder::MakeExpression(n_count, f_count); + std::vector> expr_vector = {aggr_expr}; - std::vector> expr_vector = {unique_expr, - count_expr}; - auto sch = arrow::schema({f0, f_count}); + auto sch = arrow::schema({f0}); std::vector> ret_types = {f_unique, f_count}; /////////////////////// Create Expression Evaluator //////////////////// std::shared_ptr expr; arrow::compute::FunctionContext ctx; - ASSERT_NOT_OK(CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)); + ASSERT_NOT_OK( + CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)) + std::shared_ptr input_batch; std::vector> output_batch_list; ////////////////////// calculation ///////////////////// + + std::shared_ptr> aggr_result_iterator; + std::shared_ptr aggr_result_iterator_base; + ASSERT_NOT_OK(expr->finish(&aggr_result_iterator_base)); + aggr_result_iterator = std::dynamic_pointer_cast>( + aggr_result_iterator_base); + std::vector input_data = { R"(["BJ", "SH", "SZ", "HZ", "WH", "WH", "HZ", "BJ", "SH", "SH", "BJ", "BJ", "BJ", "HZ", "HZ", "SZ", "WH", "WH", "WH", "WH"])"}; MakeInputBatch(input_data, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); + ASSERT_NOT_OK(aggr_result_iterator->ProcessAndCacheOne(input_batch->columns())); std::vector input_data_2 = { R"(["CD", "DL", "NY", "LA", "AU", "AU", "LA", "CD", "DL", "DL", "CD", "CD", "CD", "LA", "LA", "NY", "AU", "AU", "AU", "AU"])"}; MakeInputBatch(input_data_2, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); + ASSERT_NOT_OK(aggr_result_iterator->ProcessAndCacheOne(input_batch->columns())); std::vector input_data_3 = { R"(["BJ", "SH", "SZ", "NY", "WH", "WH", "AU", "BJ", "SH", "DL", "CD", "CD", "BJ", "LA", "HZ", "LA", "WH", "NY", "WH", "WH"])"}; MakeInputBatch(input_data_3, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); + ASSERT_NOT_OK(aggr_result_iterator->ProcessAndCacheOne(input_batch->columns())); ////////////////////// Finish ////////////////////////// - std::vector> result_batch; - ASSERT_NOT_OK(expr->finish(&result_batch)); std::shared_ptr expected_result; + std::shared_ptr result_batch; std::vector expected_result_string = { R"(["BJ", "SH", "SZ", "HZ", "WH", "CD", "DL", "NY" ,"LA", "AU"])", "[8, 5, 3, 5, 11, 7, 4, 4, 6, 7]"}; - auto res_sch = arrow::schema({f_unique, f_count}); - MakeInputBatch(expected_result_string, res_sch, &expected_result); - ASSERT_NOT_OK(Equals(*expected_result.get(), *(result_batch[0]).get())); -} - -TEST(TestArrowCompute, GroupByTwoAggregateWithMultipleBatchTest) { - ////////////////////// prepare expr_vector /////////////////////// - auto f0 = field("f0", uint32()); - auto f1 = field("f1", uint32()); - auto f2 = field("f2", uint32()); - auto f_unique_0 = field("unique", uint32()); - auto f_unique_1 = field("unique", uint32()); - auto f_sum = field("sum", uint64()); - auto f_res = field("res", uint64()); - - auto arg0 = TreeExprBuilder::MakeField(f0); - auto arg1 = TreeExprBuilder::MakeField(f1); - auto arg2 = TreeExprBuilder::MakeField(f2); - auto n_pre = TreeExprBuilder::MakeFunction("encodeArray", {arg0, arg1}, uint32()); - - auto n_split = TreeExprBuilder::MakeFunction("splitArrayListWithAction", - {n_pre, arg0, arg1, arg2}, uint32()); - auto arg_res = TreeExprBuilder::MakeField(f_res); - auto n_unique_0 = - TreeExprBuilder::MakeFunction("action_unique", {n_split, arg0}, uint32()); - auto n_unique_1 = - TreeExprBuilder::MakeFunction("action_unique", {n_split, arg1}, uint32()); - auto n_sum = TreeExprBuilder::MakeFunction("action_sum", {n_split, arg2}, uint32()); - - auto unique_expr_0 = TreeExprBuilder::MakeExpression(n_unique_0, f_res); - auto unique_expr_1 = TreeExprBuilder::MakeExpression(n_unique_1, f_res); - auto sum_expr = TreeExprBuilder::MakeExpression(n_sum, f_res); - - std::vector> expr_vector = { - unique_expr_0, unique_expr_1, sum_expr}; - auto sch = arrow::schema({f0, f1, f2}); - std::vector> ret_types = {f_unique_0, f_unique_1, f_sum}; - - /////////////////////// Create Expression Evaluator //////////////////// - std::shared_ptr expr; - arrow::compute::FunctionContext ctx; - ASSERT_NOT_OK(CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)); - std::shared_ptr input_batch; - std::vector> output_batch_list; - - ////////////////////// calculation ///////////////////// - std::vector input_data = { - "[1, 2, 3, 4, 5, 5, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]", - "[1, 2, 3, 4, 5, 5, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]", - "[1, 2, 3, 4, 5, 5, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]"}; - MakeInputBatch(input_data, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_2 = { - "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]", - "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]", - "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]"}; - MakeInputBatch(input_data_2, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_3 = { - "[1, 2, 3, 8, 5, 5, 10, 1, 2, 7, 6, 6, 1, 9, 4, 9, 5, 8, 5, 5]", - "[1, 2, 3, 8, 5, 5, 10, 1, 2, 7, 6, 6, 1, 9, 4, 9, 5, 8, 5, 5]", - "[1, 2, 3, 8, 5, 5, 10, 1, 2, 7, 6, 6, 1, 9, 4, 9, 5, 8, 5, 5]"}; - MakeInputBatch(input_data_3, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - ////////////////////// Finish ////////////////////////// - std::vector> result_batch; - ASSERT_NOT_OK(expr->finish(&result_batch)); - - std::shared_ptr expected_result; - std::vector expected_result_string = { - "[1, 2, 3, 4, 5, 6, 7, 8 ,9, 10]", "[1, 2, 3, 4, 5, 6, 7, 8 ,9, 10]", - "[8, 10, 9, 20, 55, 42, 28, 32, 54, 70]"}; - auto res_sch = arrow::schema({f_unique_0, f_unique_1, f_sum}); - MakeInputBatch(expected_result_string, res_sch, &expected_result); - ASSERT_NOT_OK(Equals(*expected_result.get(), *(result_batch[0]).get())); -} - -TEST(TestArrowCompute, GroupByTwoUtf8AggregateWithMultipleBatchTest) { - ////////////////////// prepare expr_vector /////////////////////// - auto f0 = field("f0", utf8()); - auto f1 = field("f1", utf8()); - auto f2 = field("f2", uint32()); - auto f_unique_0 = field("unique", utf8()); - auto f_unique_1 = field("unique", utf8()); - auto f_sum = field("sum", uint64()); - auto f_res = field("res", uint64()); - - auto arg0 = TreeExprBuilder::MakeField(f0); - auto arg1 = TreeExprBuilder::MakeField(f1); - auto arg2 = TreeExprBuilder::MakeField(f2); - auto n_pre = TreeExprBuilder::MakeFunction("encodeArray", {arg0, arg1}, uint32()); - - auto n_split = TreeExprBuilder::MakeFunction("splitArrayListWithAction", - {n_pre, arg0, arg1, arg2}, uint32()); - auto arg_res = TreeExprBuilder::MakeField(f_res); - auto n_unique_0 = - TreeExprBuilder::MakeFunction("action_unique", {n_split, arg0}, uint32()); - auto n_unique_1 = - TreeExprBuilder::MakeFunction("action_unique", {n_split, arg1}, uint32()); - auto n_sum = TreeExprBuilder::MakeFunction("action_sum", {n_split, arg2}, uint32()); - - auto unique_expr_0 = TreeExprBuilder::MakeExpression(n_unique_0, f_res); - auto unique_expr_1 = TreeExprBuilder::MakeExpression(n_unique_1, f_res); - auto sum_expr = TreeExprBuilder::MakeExpression(n_sum, f_res); - - std::vector> expr_vector = { - unique_expr_0, unique_expr_1, sum_expr}; - auto sch = arrow::schema({f0, f1, f2}); - std::vector> ret_types = {f_unique_0, f_unique_1, f_sum}; - - /////////////////////// Create Expression Evaluator //////////////////// - std::shared_ptr expr; - arrow::compute::FunctionContext ctx; - ASSERT_NOT_OK(CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)); - std::shared_ptr input_batch; - std::vector> output_batch_list; - - ////////////////////// calculation ///////////////////// - std::vector input_data = { - R"(["a", "b", "c", "d", "e", "e", "d", "a", "b", "b", "a", "a", "a", "d", "d", "c", "e", "e", "e", "e"])", - R"(["BJ", "SH", "SZ", "HZ", "WH", "WH", "HZ", "BJ", "SH", "SH", "BJ", "BJ", "BJ", "HZ", "HZ", "SZ", "WH", "WH", "WH", "WH"])", - R"([1, 2, 3, 4, 5, 5, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5])"}; - MakeInputBatch(input_data, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_2 = { - R"(["f", "g", "h", "i", "j", "j", "i", "g", "h", "i", "g", "g", "g", "j", "i", "f", "f", "i", "j", "j"])", - R"(["CD", "DL", "NY", "LA", "AU", "AU", "LA", "DL", "NY", "LA", "DL", "DL", "DL", "AU", "LA", "CD", "CD", "LA", "AU", "AU"])", - "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]"}; - MakeInputBatch(input_data_2, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - ////////////////////// Finish ////////////////////////// - std::vector> result_batch; - ASSERT_NOT_OK(expr->finish(&result_batch)); - - std::shared_ptr expected_result; - std::vector expected_result_string = { - R"(["a", "b", "c", "d", "e", "f", "g", "h", "i", "j"])", - R"(["BJ", "SH", "SZ", "HZ", "WH", "CD", "DL", "NY", "LA", "AU"])", - "[5, 6, 6, 16, 30, 24, 31, 15, 44, 49]"}; - auto res_sch = arrow::schema({f_unique_0, f_unique_1, f_sum}); + auto res_sch = arrow::schema(ret_types); MakeInputBatch(expected_result_string, res_sch, &expected_result); - - ASSERT_NOT_OK(Equals(*expected_result.get(), *(result_batch[0]).get())); + if (aggr_result_iterator->HasNext()) { + ASSERT_NOT_OK(aggr_result_iterator->Next(&result_batch)); + ASSERT_NOT_OK(Equals(*expected_result.get(), *result_batch.get())); + } } -TEST(TestArrowCompute, GroupByAggregateWithMultipleBatchOutputWoKeyTest) { +TEST(TestArrowCompute, GroupByTwoAggregateTest) { ////////////////////// prepare expr_vector /////////////////////// - auto f0 = field("f0", uint32()); + auto f0 = field("f0", int64()); auto f1 = field("f1", uint32()); - auto f_unique = field("unique", uint32()); - auto f_sum = field("sum", uint64()); - auto f_count = field("count", uint64()); - auto f_res = field("res", uint64()); - - auto arg_pre = TreeExprBuilder::MakeField(f0); - auto n_pre = TreeExprBuilder::MakeFunction("encodeArray", {arg_pre}, uint32()); - - auto arg0 = TreeExprBuilder::MakeField(f0); - auto arg1 = TreeExprBuilder::MakeField(f1); - auto n_split = - TreeExprBuilder::MakeFunction("splitArrayListWithAction", {n_pre, arg1}, uint32()); - auto arg_res = TreeExprBuilder::MakeField(f_res); - auto n_sum = TreeExprBuilder::MakeFunction("action_sum", {n_split, arg1}, uint32()); - auto n_count = TreeExprBuilder::MakeFunction("action_count", {n_split, arg1}, uint32()); - - auto sum_expr = TreeExprBuilder::MakeExpression(n_sum, f_res); - auto count_expr = TreeExprBuilder::MakeExpression(n_count, f_res); - - std::vector> expr_vector = {sum_expr, - count_expr}; - auto sch = arrow::schema({f0, f1}); - std::vector> ret_types = {f_sum, f_count}; - - /////////////////////// Create Expression Evaluator //////////////////// - std::shared_ptr expr; - arrow::compute::FunctionContext ctx; - ASSERT_NOT_OK(CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)); - std::shared_ptr input_batch; - std::vector> output_batch_list; - - ////////////////////// calculation ///////////////////// - std::vector input_data = { - "[1, 2, 3, 4, 5, 5, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]", - "[1, 2, 3, 4, 5, 5, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]"}; - MakeInputBatch(input_data, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_2 = { - "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]", - "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]"}; - MakeInputBatch(input_data_2, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_3 = { - "[1, 2, 3, 8, 5, 5, 10, 1, 2, 7, 6, 6, 1, 9, 4, 9, 5, 8, 5, 5]", - "[1, 2, 3, 8, 5, 5, 10, 1, 2, 7, 6, 6, 1, 9, 4, 9, 5, 8, 5, 5]"}; - MakeInputBatch(input_data_3, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - ////////////////////// Finish ////////////////////////// - std::vector> result_batch; - ASSERT_NOT_OK(expr->finish(&result_batch)); - - std::shared_ptr expected_result; - std::vector expected_result_string = { - "[8, 10, 9, 20, 55, 42, 28, 32, 54, 70]", "[8, 5, 3, 5, 11, 7, 4, 4, 6, 7]"}; - auto res_sch = arrow::schema({f_sum, f_count}); - MakeInputBatch(expected_result_string, res_sch, &expected_result); - ASSERT_NOT_OK(Equals(*expected_result.get(), *(result_batch[0]).get())); -} - -TEST(TestArrowCompute, StddevSampFinalTest) { - ////////////////////// prepare expr_vector /////////////////////// - auto f0 = field("f0", float64()); - auto f1 = field("f1", float64()); auto f2 = field("f2", float64()); + auto f3 = field("f3", float64()); + auto f4 = field("f4", float64()); + auto f5 = field("f5", utf8()); - auto f_float = field("float", float64()); - auto f_res = field("res", uint32()); - - auto arg_0 = TreeExprBuilder::MakeField(f0); - auto arg_1 = TreeExprBuilder::MakeField(f1); - auto arg_2 = TreeExprBuilder::MakeField(f2); - - auto n_stddev_samp_final = TreeExprBuilder::MakeFunction( - "stddev_samp_final", {arg_0, arg_1, arg_2}, float64()); - auto final_expr = TreeExprBuilder::MakeExpression(n_stddev_samp_final, f_res); - - std::vector> expr_vector = {final_expr}; - auto sch = arrow::schema({f0, f1, f2}); - std::vector> ret_types = {f_float}; - ///////////////////// Calculation ////////////////// - std::shared_ptr expr; - arrow::compute::FunctionContext ctx; - ASSERT_NOT_OK(CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)); - std::shared_ptr input_batch; - std::vector> result_batch; - std::vector input_data_string = { - "[2, 32, 14, 16, 18, 3, 4, 7, 9, 10]", "[2, 32, 14, 16, 18, 12, 32, 11, 12, 14]", - "[2, 16, 14, 16, 18, 23, 32, 45, 43, 12]"}; - MakeInputBatch(input_data_string, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &result_batch)); - std::vector input_data_2_string = {"[8, 57, 59, 12, 1, 12, 3, 5, 7, 8]", - "[8, 57, 59, 12, 1, 15, 21, 13, 15, 6]", - "[8, 57, 59, 12, 1, 8, 6, 3, 6, 12]"}; - MakeInputBatch(input_data_2_string, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &result_batch)); - ASSERT_NOT_OK(expr->finish(&result_batch)); - - std::shared_ptr expected_result; - std::vector expected_result_string = {"[21.0435]"}; - auto res_sch = arrow::schema({f_float}); - MakeInputBatch(expected_result_string, res_sch, &expected_result); - ASSERT_NOT_OK(Equals(*expected_result.get(), *(result_batch[0]).get())); -} - -TEST(TestArrowCompute, GroupByStddevSampPartialWithMultipleBatchTest) { - ////////////////////// prepare expr_vector /////////////////////// - auto f0 = field("f0", utf8()); - auto f_1 = field("f1", float64()); - auto f_2 = field("f2", int64()); - auto f_unique = field("unique", utf8()); + auto f_unique = field("unique_int64", int64()); + auto f_unique_1 = field("unique_str", utf8()); + auto f_sum = field("sum", float64()); + auto f_sum_count_multiply = field("sum_count_multiply", float64()); auto f_count = field("count", int64()); + auto f_min = field("min", uint32()); + auto f_max = field("max", uint32()); auto f_avg = field("avg", float64()); - auto f_res = field("res", uint32()); - auto f_m2 = field("m2", float64()); - - auto arg_0 = TreeExprBuilder::MakeField(f0); - auto n_pre = TreeExprBuilder::MakeFunction("encodeArray", {arg_0}, utf8()); - auto arg_1 = TreeExprBuilder::MakeField(f_1); - auto arg_2 = TreeExprBuilder::MakeField(f_2); - auto n_split = TreeExprBuilder::MakeFunction("splitArrayListWithAction", - {n_pre, arg_0, arg_1, arg_2}, uint32()); - auto arg_res = TreeExprBuilder::MakeField(f_res); - - auto n_unique = - TreeExprBuilder::MakeFunction("action_unique", {n_split, arg_0}, utf8()); - auto n_stddev_samp_partial = TreeExprBuilder::MakeFunction("action_stddev_samp_partial", - {n_split, arg_1}, uint32()); - - auto unique_expr = TreeExprBuilder::MakeExpression(n_unique, f_res); - auto stddev_samp_partial_expr = - TreeExprBuilder::MakeExpression(n_stddev_samp_partial, f_res); - - std::vector> expr_vector = { - unique_expr, stddev_samp_partial_expr}; - auto sch = arrow::schema({f0, f_1, f_2}); - std::vector> ret_types = {f_unique, f_count, f_avg, f_m2}; - - /////////////////////// Create Expression Evaluator //////////////////// - std::shared_ptr expr; - arrow::compute::FunctionContext ctx; - ASSERT_NOT_OK(CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)); - std::shared_ptr input_batch; - std::vector> output_batch_list; - - ////////////////////// calculation ///////////////////// - std::vector input_data = { - R"(["BJ", "SH", "SZ", "HZ", "WH", "WH", "HZ", "BJ", "SH", "SH", "BJ", "BJ", "BJ", "HZ", "HZ", "SZ", "WH", "WH", "WH", "WH"])", - "[2, 4, 9, 11, 12, 25, 12, 7, 5, 9, 9, 15, 23, 32, 2, 12, 23, 56, 35, 68]", - "[1, 2, 3, 4, 5, 5, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]"}; - MakeInputBatch(input_data, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_2 = { - R"(["CD", "DL", "NY", "LA", "AU", "AU", "LA", "CD", "DL", "DL", "CD", "CD", "CD", "LA", "LA", "NY", "AU", "AU", "AU", "AU"])", - "[12, 49, 64, 18, 20, 100, 81, 36, 12, 24, 23, 12, 6, 22, 12, 12, 12, 35, 76, 24]", - "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]"}; - MakeInputBatch(input_data_2, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_3 = { - R"(["BJ", "SH", "SZ", "NY", "WH", "WH", "AU", "BJ", "SH", "DL", "CD", "CD", "BJ", "LA", "HZ", "LA", "WH", "NY", "WH", "WH"])", - "[1, 12, 1, 25, 12, 9, 78, 10, 8, 15, 3, 7, 3, 1, 5, 6, 16, 25, 30, 22]", - "[1, 2, 3, 8, 5, 5, 10, 1, 2, 7, 6, 6, 1, 9, 4, 9, 5, 8, 5, 5]"}; - MakeInputBatch(input_data_3, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - ////////////////////// Finish ////////////////////////// - std::vector> result_batch; - ASSERT_NOT_OK(expr->finish(&result_batch)); - - std::shared_ptr expected_result; - std::vector expected_result_string = { - R"(["BJ", "SH", "SZ", "HZ", "WH", "CD", "DL", "NY" ,"LA", "AU"])", - "[8, 5, 3, 5, 11, 7, 4, 4, 6, 7]", - "[8.75, 7.6, 7.33333, 12.4, 28, 14.1429, 25, 31.5, 23.3333, 49.2857]", - "[385.5, 41.2, 64.6667, 549.2, 3524, 806.857, 846, 1521, 4283.33, 7201.43]"}; - auto res_sch = arrow::schema({f_unique, f_count, f_avg, f_m2}); - MakeInputBatch(expected_result_string, res_sch, &expected_result); - ASSERT_NOT_OK(Equals(*expected_result.get(), *(result_batch[0]).get())); -} - -TEST(TestArrowCompute, GroupByStddevSampFinalWithMultipleBatchTest) { - ////////////////////// prepare expr_vector /////////////////////// - auto f0 = field("f0", uint32()); - auto f1 = field("f1", float64()); - auto f2 = field("f2", float64()); - auto f3 = field("f3", float64()); - auto f_unique = field("unique", uint32()); auto f_stddev = field("stddev", float64()); auto f_res = field("res", uint32()); - auto arg_0 = TreeExprBuilder::MakeField(f0); - auto n_pre = TreeExprBuilder::MakeFunction("encodeArray", {arg_0}, uint32()); - auto arg_1 = TreeExprBuilder::MakeField(f1); - auto arg_2 = TreeExprBuilder::MakeField(f2); - auto arg_3 = TreeExprBuilder::MakeField(f3); - auto n_split = TreeExprBuilder::MakeFunction( - "splitArrayListWithAction", {n_pre, arg_0, arg_1, arg_2, arg_3}, uint32()); - auto arg_res = TreeExprBuilder::MakeField(f_res); - - auto n_unique = - TreeExprBuilder::MakeFunction("action_unique", {n_split, arg_0}, uint32()); + auto arg0 = TreeExprBuilder::MakeField(f0); + auto arg1 = TreeExprBuilder::MakeField(f1); + auto arg2 = TreeExprBuilder::MakeField(f2); + auto arg3 = TreeExprBuilder::MakeField(f3); + auto arg4 = TreeExprBuilder::MakeField(f4); + auto arg5 = TreeExprBuilder::MakeField(f5); + + auto n_groupby = TreeExprBuilder::MakeFunction("action_groupby", {arg0}, uint32()); + auto n_groupby_5 = TreeExprBuilder::MakeFunction("action_groupby", {arg5}, uint32()); + auto n_sum_count = TreeExprBuilder::MakeFunction("action_sum_count", {arg1}, uint32()); + auto n_min = TreeExprBuilder::MakeFunction("action_min", {arg1}, uint32()); + auto n_max = TreeExprBuilder::MakeFunction("action_max", {arg1}, uint32()); + auto n_avg = TreeExprBuilder::MakeFunction("action_avgByCount", {arg2, arg0}, uint32()); auto n_stddev = TreeExprBuilder::MakeFunction("action_stddev_samp_final", - {n_split, arg_1, arg_2, arg_3}, uint32()); + {arg2, arg3, arg4}, uint32()); + auto n_proj = TreeExprBuilder::MakeFunction( + "aggregateExpressions", {arg0, arg1, arg2, arg3, arg4, arg5}, uint32()); + auto n_action = TreeExprBuilder::MakeFunction( + "aggregateActions", + {n_groupby, n_groupby_5, n_sum_count, n_min, n_max, n_avg, n_stddev}, uint32()); - auto unique_expr = TreeExprBuilder::MakeExpression(n_unique, f_res); - auto stddev_expr = TreeExprBuilder::MakeExpression(n_stddev, f_res); + auto n_aggr = + TreeExprBuilder::MakeFunction("hashAggregateArrays", {n_proj, n_action}, uint32()); + auto n_child = TreeExprBuilder::MakeFunction("standalone", {n_aggr}, uint32()); + auto aggr_expr = TreeExprBuilder::MakeExpression(n_child, f_res); - std::vector> expr_vector = {unique_expr, - stddev_expr}; - auto sch = arrow::schema({f0, f1, f2, f3}); - std::vector> ret_types = {f_unique, f_stddev}; + std::vector> expr_vector = {aggr_expr}; + + auto sch = arrow::schema({f0, f1, f2, f3, f4, f5}); + std::vector> ret_types = {f_unique, f_unique_1, f_sum, f_count, + f_min, f_max, f_avg, f_stddev}; /////////////////////// Create Expression Evaluator //////////////////// std::shared_ptr expr; arrow::compute::FunctionContext ctx; - ASSERT_NOT_OK(CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)); + ASSERT_NOT_OK( + CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)); std::shared_ptr input_batch; std::vector> output_batch_list; + std::shared_ptr> aggr_result_iterator; + std::shared_ptr aggr_result_iterator_base; + ASSERT_NOT_OK(expr->finish(&aggr_result_iterator_base)); + aggr_result_iterator = std::dynamic_pointer_cast>( + aggr_result_iterator_base); + ////////////////////// calculation ///////////////////// std::vector input_data = { "[1, 2, 3, 4, 5, null, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]", + "[1, 2, 3, 4, 5, 5, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]", "[2, 4, 5, 7, 8, 2, 45, 32, 23, 12, 14, 16, 18, 19, 23, 25, 57, 59, 12, 1]", "[2, 4, 5, 7, 8, 2, 45, 32, 23, 12, 14, 16, 18, 19, 23, 25, 57, 59, 12, 1]", - "[2, 4, 5, 7, 8, 2, 45, 32, 23, 12, 14, 16, 18, 19, 23, 25, 57, 59, 12, 1]"}; + "[2, 4, 5, 7, 8, 2, 45, 32, 23, 12, 14, 16, 18, 19, 23, 25, 57, 59, 12, 1]", + R"(["BJ", "SH", "HZ", "BH", "NY", "SH", "BH", "BJ", "SH", "SH", "BJ", "BJ", "BJ", "BH", "BH", "HZ", "NY", "NY", "NY", "NY"])"}; MakeInputBatch(input_data, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); + ASSERT_NOT_OK(aggr_result_iterator->ProcessAndCacheOne(input_batch->columns())); std::vector input_data_2 = { + "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]", "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]", "[7, 8, 4, 5, 6, 1, 34, 54, 65, 66, 78, 12, 32, 24, 32, 45, 12, 24, 35, 46]", "[2, 4, 5, 7, 8, 2, 45, 32, 23, 12, 14, 16, 18, 19, 23, 25, 57, 59, 12, 1]", - "[2, 4, 5, 7, 8, 2, 45, 32, 23, 12, 14, 16, 18, 19, 23, 25, 57, 59, 12, 1]"}; + "[2, 4, 5, 7, 8, 2, 45, 32, 23, 12, 14, 16, 18, 19, 23, 25, 57, 59, 12, 1]", + R"(["BJ", "SH", "TK", "SH", "PH", "PH", "SH", "BJ", "SH", "SH", "BJ", "BJ", "BJ", "SH", "SH", "TK", "PH", "PH", "PH", "PH"])"}; MakeInputBatch(input_data_2, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); + ASSERT_NOT_OK(aggr_result_iterator->ProcessAndCacheOne(input_batch->columns())); ////////////////////// Finish ////////////////////////// - std::vector> result_batch; - ASSERT_NOT_OK(expr->finish(&result_batch)); - + std::shared_ptr result_batch; std::shared_ptr expected_result; std::vector expected_result_string = { - "[1, 2, 3, 4, 5, null, 6, 7, 8 ,9, 10]", + "[1, 2, 3, 4, 5, null, 6, 7, 8, 9, 10]", + R"(["BJ", "SH", "HZ", "BH", "NY", "SH", "BJ", "SH", "TK", "SH", "PH"])", + "[5, 6, 6, 16, 25, 5, 30, 21, 16, 36, 60]", + "[5, 3, 2, 4, 5, 1, 5, 3, 2, 4, 6]", + "[1, 2, 3, 4, 5, 5, 6, 7, 8, 9, 10]", + "[1, 2, 3, 4, 5, 5, 6, 7, 8, 9, 10]", + "[16.4, 6.5, 5, 5.875, 5.48, null, 6.1, 6.61905, 3.0625, 2.63889, 2.06667]", "[8.49255, 6.93137, 7.6489, 13.5708, 17.4668, 1.41421, 8.52779, 6.23633, 5.58903, " "12.535, 24.3544]"}; - auto res_sch = arrow::schema({f_unique, f_stddev}); + auto res_sch = arrow::schema(ret_types); MakeInputBatch(expected_result_string, res_sch, &expected_result); - ASSERT_NOT_OK(Equals(*expected_result.get(), *(result_batch[0]).get())); + if (aggr_result_iterator->HasNext()) { + ASSERT_NOT_OK(aggr_result_iterator->Next(&result_batch)); + ASSERT_NOT_OK(Equals(*expected_result.get(), *result_batch.get())); + } } -TEST(TestArrowCompute, GroupbySumCountMergeTest) { +TEST(TestArrowCompute, GroupByHashAggregateWithCaseWhenTest) { ////////////////////// prepare expr_vector /////////////////////// - auto f0 = field("f0", uint32()); - auto f1 = field("f1", float64()); - auto f2 = field("f2", int64()); - auto f_unique = field("unique", uint32()); + auto f_0 = field("f0", utf8()); + auto f_1 = field("f1", float64()); + auto f_unique = field("unique", utf8()); auto f_sum = field("sum", float64()); - auto f_count = field("count", int64()); - auto f_res = field("res", uint32()); - - auto arg_pre = TreeExprBuilder::MakeField(f0); - auto n_pre = TreeExprBuilder::MakeFunction("encodeArray", {arg_pre}, uint32()); + auto f_res = field("dummy_res", uint32()); - auto arg0 = TreeExprBuilder::MakeField(f0); - auto arg1 = TreeExprBuilder::MakeField(f1); - auto arg2 = TreeExprBuilder::MakeField(f2); - auto n_split = TreeExprBuilder::MakeFunction("splitArrayListWithAction", - {n_pre, arg0, arg1, arg2}, uint32()); - auto arg_res = TreeExprBuilder::MakeField(f_res); - auto n_unique = - TreeExprBuilder::MakeFunction("action_unique", {n_split, arg0}, uint32()); - auto n_merge = TreeExprBuilder::MakeFunction("action_sum_count_merge", - {n_split, arg1, arg2}, uint32()); - - auto unique_expr = TreeExprBuilder::MakeExpression(n_unique, f_res); - auto merge_expr = TreeExprBuilder::MakeExpression(n_merge, f_res); - - std::vector> expr_vector = {unique_expr, - merge_expr}; - auto sch = arrow::schema({f0, f1, f2}); - std::vector> ret_types = {f_unique, f_sum, f_count}; + auto arg_unique = TreeExprBuilder::MakeField(f_unique); + auto arg_sum = TreeExprBuilder::MakeField(f_sum); + auto n_groupby = TreeExprBuilder::MakeFunction( + "action_groupby", {TreeExprBuilder::MakeField(f_0)}, uint32()); + + auto n_when = TreeExprBuilder::MakeFunction( + "equal", + {TreeExprBuilder::MakeField(f_0), TreeExprBuilder::MakeStringLiteral("BJ")}, + arrow::boolean()); + auto n_then = TreeExprBuilder::MakeFunction( + "multiply", + {TreeExprBuilder::MakeField(f_1), TreeExprBuilder::MakeLiteral((double)0.3)}, + float64()); + auto n_else = TreeExprBuilder::MakeFunction( + "multiply", + {TreeExprBuilder::MakeField(f_1), TreeExprBuilder::MakeLiteral((double)1.3)}, + float64()); + auto n_projection = TreeExprBuilder::MakeIf(n_when, n_then, n_else, float64()); + + auto n_sum = TreeExprBuilder::MakeFunction("action_sum", {n_projection}, uint32()); + + auto n_proj = TreeExprBuilder::MakeFunction( + "aggregateExpressions", + {TreeExprBuilder::MakeField(f_0), TreeExprBuilder::MakeField(f_1)}, uint32()); + auto n_action = + TreeExprBuilder::MakeFunction("aggregateActions", {n_groupby, n_sum}, uint32()); + + auto n_aggr = + TreeExprBuilder::MakeFunction("hashAggregateArrays", {n_proj, n_action}, uint32()); + auto n_child = TreeExprBuilder::MakeFunction("standalone", {n_aggr}, uint32()); + auto aggr_expr = TreeExprBuilder::MakeExpression(n_child, f_res); + + std::vector> expr_vector = {aggr_expr}; + + auto sch = arrow::schema({f_0, f_1}); + std::vector> ret_types = {f_unique, f_sum}; /////////////////////// Create Expression Evaluator //////////////////// std::shared_ptr expr; arrow::compute::FunctionContext ctx; - ASSERT_NOT_OK(CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)); + ASSERT_NOT_OK( + CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)); std::shared_ptr input_batch; std::vector> output_batch_list; + std::shared_ptr> aggr_result_iterator; + std::shared_ptr aggr_result_iterator_base; + ASSERT_NOT_OK(expr->finish(&aggr_result_iterator_base)); + aggr_result_iterator = std::dynamic_pointer_cast>( + aggr_result_iterator_base); + ////////////////////// calculation ///////////////////// std::vector input_data = { - "[1, 2, 3, 4, 5, null, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]", - "[1, 2, 3, 4, 5, 5, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]", - "[1, 2, 3, 4, 5, 5, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]"}; + R"(["BJ", "SH", "SZ", "HZ", "WH", "WH", "HZ", "BJ", "SH", "SH", "BJ", "BJ", "BJ", +"HZ", "HZ", "SZ", "WH", "WH", "WH", "WH"])", + "[1, 4, 9, 16, 25, 25, 16, 1, 3, 5, 1, 1, 1, 16, 16, 9, 25, 25, 25, 25]"}; MakeInputBatch(input_data, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); + ASSERT_NOT_OK(aggr_result_iterator->ProcessAndCacheOne(input_batch->columns())); std::vector input_data_2 = { - "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]", - "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]", - "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]"}; + R"(["CD", "DL", "NY", "LA", "AU", "AU", "LA", "CD", "DL", "DL", "CD", "CD", "CD", +"LA", "LA", "NY", "AU", "AU", "AU", "AU"])", + "[36, 49, 64, 81, 100, 100, 81, 36, 49, 49, 36, 36, 36, 81, 81, 64, 100, 100, 100, " + "100]"}; MakeInputBatch(input_data_2, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); + ASSERT_NOT_OK(aggr_result_iterator->ProcessAndCacheOne(input_batch->columns())); std::vector input_data_3 = { - "[1, 2, 3, 8, 5, 5, 10, 1, 2, 7, 6, 6, 1, 9, 4, 9, null, 8, 5, 5]", - "[1, 2, 3, 8, 5, 5, 10, 1, 2, 7, 6, 6, 1, 9, 4, 9, 5, 8, 5, 5]", - "[1, 2, 3, 8, 5, 5, 10, 1, 2, 7, 6, 6, 1, 9, 4, 9, 5, 8, 5, 5]"}; + R"(["BJ", "SH", "SZ", "NY", "WH", "WH", "AU", "BJ", "SH", "DL", "CD", "CD", "BJ", +"LA", "HZ", "LA", "WH", "NY", "WH", "WH"])", + "[1, 4, 9, 64, 25, 25, 100, 1, 4, 49, 36, 36, 1, 81, 16, 81, 25, 64, 25, 25]"}; MakeInputBatch(input_data_3, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); + ASSERT_NOT_OK(aggr_result_iterator->ProcessAndCacheOne(input_batch->columns())); ////////////////////// Finish ////////////////////////// - std::vector> result_batch; - ASSERT_NOT_OK(expr->finish(&result_batch)); - + std::shared_ptr result_batch; std::shared_ptr expected_result; std::vector expected_result_string = { - "[1, 2, 3, 4, 5, null, 6, 7, 8 ,9, 10]", - "[8, 10, 9, 20, 45, 10, 42, 28, 32, 54, 70]", - "[8, 10, 9, 20, 45, 10, 42, 28, 32, 54, 70]"}; - auto res_sch = arrow::schema({f_unique, f_sum, f_count}); + R"(["BJ", "SH", "SZ", "HZ", "WH", "CD", "DL", "NY" ,"LA", "AU"])", + "[2.4, 26, 35.1, 104, 357.5, 327.6, 254.8, 332.8, 631.8, 910]"}; + auto res_sch = arrow::schema(ret_types); MakeInputBatch(expected_result_string, res_sch, &expected_result); - ASSERT_NOT_OK(Equals(*expected_result.get(), *(result_batch[0]).get())); + if (aggr_result_iterator->HasNext()) { + ASSERT_NOT_OK(aggr_result_iterator->Next(&result_batch)); + ASSERT_NOT_OK(Equals(*expected_result.get(), *result_batch.get())); + } } } // namespace codegen diff --git a/cpp/src/tests/arrow_compute_test_hashaggregate.cc b/cpp/src/tests/arrow_compute_test_hashaggregate.cc deleted file mode 100644 index 2294d4467..000000000 --- a/cpp/src/tests/arrow_compute_test_hashaggregate.cc +++ /dev/null @@ -1,961 +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. - */ - -#include -#include - -#include - -#include "codegen/code_generator.h" -#include "codegen/code_generator_factory.h" -#include "tests/test_utils.h" - -namespace sparkcolumnarplugin { -namespace codegen { - -TEST(TestArrowCompute, GroupByHashAggregateTest) { - ////////////////////// prepare expr_vector /////////////////////// - auto f0 = field("f0", uint32()); - auto f1 = field("f1", uint32()); - auto f_unique = field("unique", uint32()); - auto f_sum = field("sum", uint64()); - auto f_count = field("count", int64()); - auto f_avg = field("avg", float64()); - auto f_min = field("min", uint32()); - auto f_max = field("max", uint32()); - auto f_res = field("res", uint32()); - - auto arg0 = TreeExprBuilder::MakeField(f0); - auto arg1 = TreeExprBuilder::MakeField(f1); - auto n_groupby = TreeExprBuilder::MakeFunction("action_groupby", {arg0}, uint32()); - auto n_sum = TreeExprBuilder::MakeFunction("action_sum", {arg1}, uint32()); - auto n_count = TreeExprBuilder::MakeFunction("action_count", {arg1}, uint32()); - auto n_avg = TreeExprBuilder::MakeFunction("action_avg", {arg1}, uint32()); - auto n_sum_count = TreeExprBuilder::MakeFunction("action_sum_count", {arg1}, uint32()); - auto n_min = TreeExprBuilder::MakeFunction("action_min", {arg1}, uint32()); - auto n_max = TreeExprBuilder::MakeFunction("action_max", {arg1}, uint32()); - auto n_schema = TreeExprBuilder::MakeFunction( - "codegen_schema", {TreeExprBuilder::MakeField(f0), TreeExprBuilder::MakeField(f1)}, - uint32()); - auto n_aggr = TreeExprBuilder::MakeFunction( - "hashAggregateArrays", - {n_groupby, n_sum, n_count, n_avg, n_sum_count, n_min, n_max}, uint32()); - auto n_codegen_aggr = - TreeExprBuilder::MakeFunction("codegen_withOneInput", {n_aggr, n_schema}, uint32()); - - auto aggr_expr = TreeExprBuilder::MakeExpression(n_codegen_aggr, f_res); - - std::vector> expr_vector = {aggr_expr}; - - auto sch = arrow::schema({f0, f1}); - std::vector> ret_types = {f_unique, f_sum, f_count, f_avg, - f_sum, f_count, f_min, f_max}; - - /////////////////////// Create Expression Evaluator //////////////////// - std::shared_ptr expr; - arrow::compute::FunctionContext ctx; - ASSERT_NOT_OK(CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)); - std::shared_ptr input_batch; - std::vector> output_batch_list; - - ////////////////////// calculation ///////////////////// - std::vector input_data = { - "[1, 2, 3, 4, 5, null, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]", - "[1, 2, 3, 4, 5, 5, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]"}; - MakeInputBatch(input_data, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_2 = { - "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]", - "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]"}; - MakeInputBatch(input_data_2, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_3 = { - "[1, 2, 3, 8, 5, 5, 10, 1, 2, 7, 6, 6, 1, 9, 4, 9, null, 8, 5, 5]", - "[1, 2, 3, 8, 5, 5, 10, 1, 2, 7, 6, 6, 1, 9, 4, 9, 5, 8, 5, 5]"}; - MakeInputBatch(input_data_3, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - ////////////////////// Finish ////////////////////////// - std::shared_ptr result_batch; - std::shared_ptr aggr_result_iterator_base; - std::shared_ptr> aggr_result_iterator; - ASSERT_NOT_OK(expr->finish(&aggr_result_iterator_base)); - aggr_result_iterator = std::dynamic_pointer_cast>( - aggr_result_iterator_base); - - std::shared_ptr expected_result; - std::vector expected_result_string = { - "[1, 2, 3, 4, 5, null, 6, 7, 8 ,9, 10]", - "[8, 10, 9, 20, 45, 10, 42, 28, 32, 54, 70]", - "[8, 5, 3, 5, 9, 2, 7, 4, 4, 6, 7]", - "[1, 2, 3, 4, 5, 5, 6, 7, 8, 9, 10]", - "[8, 10, 9, 20, 45, 10, 42, 28, 32, 54, 70]", - "[8, 5, 3, 5, 9, 2, 7, 4, 4, 6, 7]", - "[1, 2, 3, 4, 5, 5, 6, 7, 8, 9, 10]", - "[1, 2, 3, 4, 5, 5, 6, 7, 8, 9, 10]"}; - auto res_sch = - arrow::schema({f_unique, f_sum, f_count, f_avg, f_sum, f_count, f_min, f_max}); - MakeInputBatch(expected_result_string, res_sch, &expected_result); - if (aggr_result_iterator->HasNext()) { - ASSERT_NOT_OK(aggr_result_iterator->Next(&result_batch)); - ASSERT_NOT_OK(Equals(*expected_result.get(), *result_batch.get())); - } -} - -TEST(TestArrowCompute, GroupByHashAggregateTest2) { - ////////////////////// prepare expr_vector /////////////////////// - auto f0 = field("f0", uint32()); - auto f1 = field("f1", uint32()); - auto f_unique = field("unique", uint32()); - auto f_sum = field("sum", uint64()); - auto f_count = field("count", int64()); - auto f_min = field("min", uint32()); - auto f_max = field("max", uint32()); - auto f_res = field("res", uint32()); - - auto arg0 = TreeExprBuilder::MakeField(f0); - auto arg1 = TreeExprBuilder::MakeField(f1); - auto n_groupby = TreeExprBuilder::MakeFunction("action_groupby", {arg0}, uint32()); - auto n_sum_count = TreeExprBuilder::MakeFunction("action_sum_count", {arg1}, uint32()); - auto n_min = TreeExprBuilder::MakeFunction("action_min", {arg1}, uint32()); - auto n_max = TreeExprBuilder::MakeFunction("action_max", {arg1}, uint32()); - auto n_sum = TreeExprBuilder::MakeFunction("action_sum", {arg1}, uint32()); - auto n_count = TreeExprBuilder::MakeFunction("action_count", {arg1}, uint32()); - auto n_schema = TreeExprBuilder::MakeFunction( - "codegen_schema", {TreeExprBuilder::MakeField(f0), TreeExprBuilder::MakeField(f1)}, - uint32()); - auto n_aggr = TreeExprBuilder::MakeFunction( - "hashAggregateArrays", {n_groupby, n_sum_count, n_min, n_max, n_sum, n_count}, - uint32()); - auto n_codegen_aggr = - TreeExprBuilder::MakeFunction("codegen_withOneInput", {n_aggr, n_schema}, uint32()); - - auto aggr_expr = TreeExprBuilder::MakeExpression(n_codegen_aggr, f_res); - - std::vector> expr_vector = {aggr_expr}; - - auto sch = arrow::schema({f0, f1}); - std::vector> ret_types = {f_unique, f_sum, f_count, f_min, - f_max, f_sum, f_count}; - - /////////////////////// Create Expression Evaluator //////////////////// - std::shared_ptr expr; - arrow::compute::FunctionContext ctx; - ASSERT_NOT_OK(CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)); - std::shared_ptr input_batch; - std::vector> output_batch_list; - - ////////////////////// calculation ///////////////////// - std::vector input_data = { - "[1, 2, 3, 4, 5, null, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]", - "[1, 2, 3, 4, 5, 5, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]"}; - MakeInputBatch(input_data, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_2 = { - "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]", - "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]"}; - MakeInputBatch(input_data_2, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_3 = { - "[1, 2, 3, 8, 5, 5, 10, 1, 2, 7, 6, 6, 1, 9, 4, 9, null, 8, 5, 5]", - "[1, 2, 3, 8, 5, 5, 10, 1, 2, 7, 6, 6, 1, 9, 4, 9, 5, 8, 5, 5]"}; - MakeInputBatch(input_data_3, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - ////////////////////// Finish ////////////////////////// - std::shared_ptr result_batch; - std::shared_ptr> aggr_result_iterator; - std::shared_ptr aggr_result_iterator_base; - ASSERT_NOT_OK(expr->finish(&aggr_result_iterator_base)); - aggr_result_iterator = std::dynamic_pointer_cast>( - aggr_result_iterator_base); - - std::shared_ptr expected_result; - std::vector expected_result_string = { - "[1, 2, 3, 4, 5, null, 6, 7, 8 ,9, 10]", - "[8, 10, 9, 20, 45, 10, 42, 28, 32, 54, 70]", - "[8, 5, 3, 5, 9, 2, 7, 4, 4, 6, 7]", - "[1, 2, 3, 4, 5, 5, 6, 7, 8, 9, 10]", - "[1, 2, 3, 4, 5, 5, 6, 7, 8, 9, 10]", - "[8, 10, 9, 20, 45, 10, 42, 28, 32, 54, 70]", - "[8, 5, 3, 5, 9, 2, 7, 4, 4, 6, 7]"}; - auto res_sch = arrow::schema(ret_types); - MakeInputBatch(expected_result_string, res_sch, &expected_result); - if (aggr_result_iterator->HasNext()) { - ASSERT_NOT_OK(aggr_result_iterator->Next(&result_batch)); - ASSERT_NOT_OK(Equals(*expected_result.get(), *result_batch.get())); - } -} - -TEST(TestArrowCompute, GroupByHashAggregateTest3) { - ////////////////////// prepare expr_vector /////////////////////// - auto f0 = field("f0", uint32()); - auto f1 = field("f1", uint32()); - auto f2 = field("f2", uint32()); - auto f_unique = field("unique", uint32()); - auto f_sum = field("sum", uint64()); - auto f_count = field("count", int64()); - auto f_min = field("min", uint32()); - auto f_max = field("max", uint32()); - auto f_res = field("res", uint32()); - - auto arg0 = TreeExprBuilder::MakeField(f0); - auto arg1 = TreeExprBuilder::MakeField(f1); - auto arg2 = TreeExprBuilder::MakeField(f2); - auto n_groupby = TreeExprBuilder::MakeFunction("action_groupby", {arg0}, uint32()); - auto n_min = TreeExprBuilder::MakeFunction("action_min", {arg1}, uint32()); - auto n_max = TreeExprBuilder::MakeFunction("action_max", {arg1}, uint32()); - auto n_sum_count_merge = - TreeExprBuilder::MakeFunction("action_sum_count_merge", {arg1, arg2}, uint32()); - auto n_schema = TreeExprBuilder::MakeFunction( - "codegen_schema", - {TreeExprBuilder::MakeField(f0), TreeExprBuilder::MakeField(f1), - TreeExprBuilder::MakeField(f2)}, - uint32()); - auto n_aggr = TreeExprBuilder::MakeFunction( - "hashAggregateArrays", {n_groupby, n_min, n_max, n_sum_count_merge}, uint32()); - auto n_codegen_aggr = - TreeExprBuilder::MakeFunction("codegen_withOneInput", {n_aggr, n_schema}, uint32()); - - auto aggr_expr = TreeExprBuilder::MakeExpression(n_codegen_aggr, f_res); - - std::vector> expr_vector = {aggr_expr}; - - auto sch = arrow::schema({f0, f1, f2}); - std::vector> ret_types = {f_unique, f_min, f_max, f_sum, - f_count}; - - /////////////////////// Create Expression Evaluator //////////////////// - std::shared_ptr expr; - arrow::compute::FunctionContext ctx; - ASSERT_NOT_OK(CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)); - std::shared_ptr input_batch; - std::vector> output_batch_list; - - ////////////////////// calculation ///////////////////// - std::vector input_data = { - "[1, 2, 3, 4, 5, null, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]", - "[1, 2, 3, 4, 5, 5, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]", - "[1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1]"}; - MakeInputBatch(input_data, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_2 = { - "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]", - "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]", - "[1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1]"}; - MakeInputBatch(input_data_2, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_3 = { - "[1, 2, 3, 8, 5, 5, 10, 1, 2, 7, 6, 6, 1, 9, 4, 9, null, 8, 5, 5]", - "[1, 2, 3, 8, 5, 5, 10, 1, 2, 7, 6, 6, 1, 9, 4, 9, 5, 8, 5, 5]", - "[1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1]"}; - MakeInputBatch(input_data_3, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - ////////////////////// Finish ////////////////////////// - std::shared_ptr result_batch; - std::shared_ptr> aggr_result_iterator; - std::shared_ptr aggr_result_iterator_base; - ASSERT_NOT_OK(expr->finish(&aggr_result_iterator_base)); - aggr_result_iterator = std::dynamic_pointer_cast>( - aggr_result_iterator_base); - - std::shared_ptr expected_result; - std::vector expected_result_string = { - "[1, 2, 3, 4, 5, null, 6, 7, 8 ,9, 10]", "[1, 2, 3, 4, 5, 5, 6, 7, 8, 9, 10]", - "[1, 2, 3, 4, 5, 5, 6, 7, 8, 9, 10]", "[8, 10, 9, 20, 45, 10, 42, 28, 32, 54, 70]", - "[8, 5, 3, 5, 9, 2, 7, 4, 4, 6, 7]"}; - auto res_sch = arrow::schema(ret_types); - MakeInputBatch(expected_result_string, res_sch, &expected_result); - if (aggr_result_iterator->HasNext()) { - ASSERT_NOT_OK(aggr_result_iterator->Next(&result_batch)); - ASSERT_NOT_OK(Equals(*expected_result.get(), *result_batch.get())); - } -} - -TEST(TestArrowCompute, GroupByHashAggregateWithStringTest) { - ////////////////////// prepare expr_vector /////////////////////// - auto f_unique = field("unique", utf8()); - auto f_sum = field("sum", float64()); - auto f_count = field("count", int64()); - auto f_avg = field("avg", float64()); - auto f_res = field("dummy_res", uint32()); - - auto arg_unique = TreeExprBuilder::MakeField(f_unique); - auto arg_sum = TreeExprBuilder::MakeField(f_sum); - auto arg_count = TreeExprBuilder::MakeField(f_count); - auto n_groupby = - TreeExprBuilder::MakeFunction("action_groupby", {arg_unique}, uint32()); - auto n_avg = - TreeExprBuilder::MakeFunction("action_avgByCount", {arg_sum, arg_count}, uint32()); - auto n_schema = TreeExprBuilder::MakeFunction( - "codegen_schema", {arg_unique, arg_sum, arg_count}, uint32()); - auto n_aggr = - TreeExprBuilder::MakeFunction("hashAggregateArrays", {n_groupby, n_avg}, uint32()); - auto n_codegen_aggr = - TreeExprBuilder::MakeFunction("codegen_withOneInput", {n_aggr, n_schema}, uint32()); - - auto aggr_expr = TreeExprBuilder::MakeExpression(n_codegen_aggr, f_res); - - std::vector> expr_vector = {aggr_expr}; - - auto sch = arrow::schema({f_unique, f_sum, f_count}); - std::vector> ret_types = {f_unique, f_avg}; - - /////////////////////// Create Expression Evaluator //////////////////// - std::shared_ptr expr; - arrow::compute::FunctionContext ctx; - ASSERT_NOT_OK(CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)); - std::shared_ptr input_batch; - std::vector> output_batch_list; - - ////////////////////// calculation ///////////////////// - std::vector input_data = { - R"(["BJ", "SH", "SZ", "HZ", "WH", "WH", "HZ", "BJ", "SH", "SH", "BJ", "BJ", "BJ", -"HZ", "HZ", "SZ", "WH", "WH", "WH", "WH"])", - "[1, 4, 9, 16, 25, 25, 16, 1, 4, 4, 1, 1, 1, 16, 16, 9, 25, 25, 25, 25]", - "[1, 2, 3, 4, 5, 5, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]"}; - MakeInputBatch(input_data, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_2 = { - R"(["CD", "DL", "NY", "LA", "AU", "AU", "LA", "CD", "DL", "DL", "CD", "CD", "CD", -"LA", "LA", "NY", "AU", "AU", "AU", "AU"])", - "[36, 49, 64, 81, 100, 100, 81, 36, 49, 49, 36, 36, 36, 81, 81, 64, 100, 100, 100, " - "100]", - "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]"}; - MakeInputBatch(input_data_2, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_3 = { - R"(["BJ", "SH", "SZ", "NY", "WH", "WH", "AU", "BJ", "SH", "DL", "CD", "CD", "BJ", -"LA", "HZ", "LA", "WH", "NY", "WH", "WH"])", - "[1, 4, 9, 64, 25, 25, 100, 1, 4, 49, 36, 36, 1, 81, 16, 81, 25, 64, 25, 25]", - "[1, 2, 3, 8, 5, 5, 10, 1, 2, 7, 6, 6, 1, 9, 4, 9, 5, 8, 5, 5]"}; - MakeInputBatch(input_data_3, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - ////////////////////// Finish ////////////////////////// - std::shared_ptr result_batch; - std::shared_ptr> aggr_result_iterator; - std::shared_ptr aggr_result_iterator_base; - ASSERT_NOT_OK(expr->finish(&aggr_result_iterator_base)); - aggr_result_iterator = std::dynamic_pointer_cast>( - aggr_result_iterator_base); - - std::shared_ptr expected_result; - std::vector expected_result_string = { - R"(["BJ", "SH", "SZ", "HZ", "WH", "CD", "DL", "NY" ,"LA", "AU"])", - "[1, 2, 3, 4, 5, 6, 7, 8, 9, 10]"}; - auto res_sch = arrow::schema({f_unique, f_avg}); - MakeInputBatch(expected_result_string, res_sch, &expected_result); - if (aggr_result_iterator->HasNext()) { - ASSERT_NOT_OK(aggr_result_iterator->Next(&result_batch)); - ASSERT_NOT_OK(Equals(*expected_result.get(), *result_batch.get())); - } -} - -TEST(TestArrowCompute, GroupByHashAggregateWithProjectionTest) { - ////////////////////// prepare expr_vector /////////////////////// - auto f_0 = field("f0", utf8()); - auto f_1 = field("f1", float64()); - auto f_unique = field("unique", utf8()); - auto f_max = field("max", float64()); - auto f_res = field("dummy_res", uint32()); - - auto arg_0 = TreeExprBuilder::MakeField(f_0); - auto arg_1 = TreeExprBuilder::MakeField(f_1); - auto arg_unique = TreeExprBuilder::MakeField(f_unique); - auto arg_max = TreeExprBuilder::MakeField(f_max); - auto n_groupby = TreeExprBuilder::MakeFunction("action_groupby", {arg_0}, uint32()); - auto n_projection = TreeExprBuilder::MakeFunction( - "multiply", {arg_1, TreeExprBuilder::MakeLiteral(0.3)}, float64()); - auto n_max = TreeExprBuilder::MakeFunction("action_max", {n_projection}, uint32()); - auto n_schema = - TreeExprBuilder::MakeFunction("codegen_schema", {arg_0, arg_1}, uint32()); - auto n_aggr = - TreeExprBuilder::MakeFunction("hashAggregateArrays", {n_groupby, n_max}, uint32()); - auto n_codegen_aggr = - TreeExprBuilder::MakeFunction("codegen_withOneInput", {n_aggr, n_schema}, uint32()); - - auto aggr_expr = TreeExprBuilder::MakeExpression(n_codegen_aggr, f_res); - - std::vector> expr_vector = {aggr_expr}; - - auto sch = arrow::schema({f_0, f_1}); - std::vector> ret_types = {f_unique, f_max}; - - /////////////////////// Create Expression Evaluator //////////////////// - std::shared_ptr expr; - arrow::compute::FunctionContext ctx; - ASSERT_NOT_OK(CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)); - std::shared_ptr input_batch; - std::vector> output_batch_list; - - ////////////////////// calculation ///////////////////// - std::vector input_data = { - R"(["BJ", "SH", "SZ", "HZ", "WH", "WH", "HZ", "BJ", "SH", "SH", "BJ", "BJ", "BJ", -"HZ", "HZ", "SZ", "WH", "WH", "WH", "WH"])", - "[1, 4, 9, 16, 25, 25, 16, 1, 3, 5, 1, 1, 1, 16, 16, 9, 25, 25, 25, 25]"}; - MakeInputBatch(input_data, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_2 = { - R"(["CD", "DL", "NY", "LA", "AU", "AU", "LA", "CD", "DL", "DL", "CD", "CD", "CD", -"LA", "LA", "NY", "AU", "AU", "AU", "AU"])", - "[36, 49, 64, 81, 100, 100, 81, 36, 49, 49, 36, 36, 36, 81, 81, 64, 100, 100, 100, " - "100]"}; - MakeInputBatch(input_data_2, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_3 = { - R"(["BJ", "SH", "SZ", "NY", "WH", "WH", "AU", "BJ", "SH", "DL", "CD", "CD", "BJ", -"LA", "HZ", "LA", "WH", "NY", "WH", "WH"])", - "[1, 4, 9, 64, 25, 25, 100, 1, 4, 49, 36, 36, 1, 81, 16, 81, 25, 64, 25, 25]"}; - MakeInputBatch(input_data_3, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - ////////////////////// Finish ////////////////////////// - std::shared_ptr result_batch; - std::shared_ptr> aggr_result_iterator; - std::shared_ptr aggr_result_iterator_base; - ASSERT_NOT_OK(expr->finish(&aggr_result_iterator_base)); - aggr_result_iterator = std::dynamic_pointer_cast>( - aggr_result_iterator_base); - - std::shared_ptr expected_result; - std::vector expected_result_string = { - R"(["BJ", "SH", "SZ", "HZ", "WH", "CD", "DL", "NY" ,"LA", "AU"])", - "[0.3, 1.5, 2.7, 4.8, 7.5, 10.8, 14.7, 19.2, 24.3, 30]"}; - auto res_sch = arrow::schema(ret_types); - MakeInputBatch(expected_result_string, res_sch, &expected_result); - if (aggr_result_iterator->HasNext()) { - ASSERT_NOT_OK(aggr_result_iterator->Next(&result_batch)); - ASSERT_NOT_OK(Equals(*expected_result.get(), *result_batch.get())); - } -} - -TEST(TestArrowCompute, GroupByHashAggregateWithCaseWhenTest) { - ////////////////////// prepare expr_vector /////////////////////// - auto f_0 = field("f0", utf8()); - auto f_1 = field("f1", float64()); - auto f_unique = field("unique", utf8()); - auto f_sum = field("sum", float64()); - auto f_res = field("dummy_res", uint32()); - - auto arg_0 = TreeExprBuilder::MakeField(f_0); - auto arg_1 = TreeExprBuilder::MakeField(f_1); - auto arg_unique = TreeExprBuilder::MakeField(f_unique); - auto arg_sum = TreeExprBuilder::MakeField(f_sum); - auto n_groupby = TreeExprBuilder::MakeFunction("action_groupby", {arg_0}, uint32()); - - auto n_when = TreeExprBuilder::MakeFunction( - "equal", {arg_0, TreeExprBuilder::MakeStringLiteral("BJ")}, arrow::boolean()); - auto n_then = TreeExprBuilder::MakeFunction( - "multiply", {arg_1, TreeExprBuilder::MakeLiteral((double)0.3)}, float64()); - auto n_else = TreeExprBuilder::MakeFunction( - "multiply", {arg_1, TreeExprBuilder::MakeLiteral((double)1.3)}, float64()); - auto n_projection = TreeExprBuilder::MakeIf(n_when, n_then, n_else, float64()); - - auto n_sum = TreeExprBuilder::MakeFunction("action_sum", {n_projection}, uint32()); - auto n_schema = - TreeExprBuilder::MakeFunction("codegen_schema", {arg_0, arg_1}, uint32()); - auto n_aggr = - TreeExprBuilder::MakeFunction("hashAggregateArrays", {n_groupby, n_sum}, uint32()); - auto n_codegen_aggr = - TreeExprBuilder::MakeFunction("codegen_withOneInput", {n_aggr, n_schema}, uint32()); - - auto aggr_expr = TreeExprBuilder::MakeExpression(n_codegen_aggr, f_res); - - std::vector> expr_vector = {aggr_expr}; - - auto sch = arrow::schema({f_0, f_1}); - std::vector> ret_types = {f_unique, f_sum}; - - /////////////////////// Create Expression Evaluator //////////////////// - std::shared_ptr expr; - arrow::compute::FunctionContext ctx; - ASSERT_NOT_OK(CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)); - std::shared_ptr input_batch; - std::vector> output_batch_list; - - ////////////////////// calculation ///////////////////// - std::vector input_data = { - R"(["BJ", "SH", "SZ", "HZ", "WH", "WH", "HZ", "BJ", "SH", "SH", "BJ", "BJ", "BJ", -"HZ", "HZ", "SZ", "WH", "WH", "WH", "WH"])", - "[1, 4, 9, 16, 25, 25, 16, 1, 3, 5, 1, 1, 1, 16, 16, 9, 25, 25, 25, 25]"}; - MakeInputBatch(input_data, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_2 = { - R"(["CD", "DL", "NY", "LA", "AU", "AU", "LA", "CD", "DL", "DL", "CD", "CD", "CD", -"LA", "LA", "NY", "AU", "AU", "AU", "AU"])", - "[36, 49, 64, 81, 100, 100, 81, 36, 49, 49, 36, 36, 36, 81, 81, 64, 100, 100, 100, " - "100]"}; - MakeInputBatch(input_data_2, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_3 = { - R"(["BJ", "SH", "SZ", "NY", "WH", "WH", "AU", "BJ", "SH", "DL", "CD", "CD", "BJ", -"LA", "HZ", "LA", "WH", "NY", "WH", "WH"])", - "[1, 4, 9, 64, 25, 25, 100, 1, 4, 49, 36, 36, 1, 81, 16, 81, 25, 64, 25, 25]"}; - MakeInputBatch(input_data_3, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - ////////////////////// Finish ////////////////////////// - std::shared_ptr result_batch; - std::shared_ptr> aggr_result_iterator; - std::shared_ptr aggr_result_iterator_base; - ASSERT_NOT_OK(expr->finish(&aggr_result_iterator_base)); - aggr_result_iterator = std::dynamic_pointer_cast>( - aggr_result_iterator_base); - - std::shared_ptr expected_result; - std::vector expected_result_string = { - R"(["BJ", "SH", "SZ", "HZ", "WH", "CD", "DL", "NY" ,"LA", "AU"])", - "[2.4, 26, 35.1, 104, 357.5, 327.6, 254.8, 332.8, 631.8, 910]"}; - auto res_sch = arrow::schema(ret_types); - MakeInputBatch(expected_result_string, res_sch, &expected_result); - if (aggr_result_iterator->HasNext()) { - ASSERT_NOT_OK(aggr_result_iterator->Next(&result_batch)); - ASSERT_NOT_OK(Equals(*expected_result.get(), *result_batch.get())); - } -} - -TEST(TestArrowCompute, GroupByHashAggregateWithNoKeyTest) { - ////////////////////// prepare expr_vector /////////////////////// - auto f_unique = field("unique", utf8()); - auto f_sum = field("sum", float64()); - auto f_count = field("count", int64()); - auto f_avg = field("avg", float64()); - auto f_res = field("dummy_res", uint32()); - - auto arg_unique = TreeExprBuilder::MakeField(f_unique); - auto arg_sum = TreeExprBuilder::MakeField(f_sum); - auto arg_count = TreeExprBuilder::MakeField(f_count); - auto n_groupby = - TreeExprBuilder::MakeFunction("action_groupby_no_keep", {arg_unique}, uint32()); - auto n_avg = - TreeExprBuilder::MakeFunction("action_avgByCount", {arg_sum, arg_count}, uint32()); - auto n_schema = TreeExprBuilder::MakeFunction( - "codegen_schema", {arg_unique, arg_sum, arg_count}, uint32()); - auto n_aggr = - TreeExprBuilder::MakeFunction("hashAggregateArrays", {n_groupby, n_avg}, uint32()); - auto n_codegen_aggr = - TreeExprBuilder::MakeFunction("codegen_withOneInput", {n_aggr, n_schema}, uint32()); - - auto aggr_expr = TreeExprBuilder::MakeExpression(n_codegen_aggr, f_res); - - std::vector> expr_vector = {aggr_expr}; - - auto sch = arrow::schema({f_unique, f_sum, f_count}); - std::vector> ret_types = {f_avg}; - - /////////////////////// Create Expression Evaluator //////////////////// - std::shared_ptr expr; - arrow::compute::FunctionContext ctx; - ASSERT_NOT_OK(CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)); - std::shared_ptr input_batch; - std::vector> output_batch_list; - - ////////////////////// calculation ///////////////////// - std::vector input_data = { - R"(["BJ", "SH", "SZ", "HZ", "WH", "WH", "HZ", "BJ", "SH", "SH", "BJ", "BJ", "BJ", -"HZ", "HZ", "SZ", "WH", "WH", "WH", "WH"])", - "[1, 4, 9, 16, 25, 25, 16, 1, 4, 4, 1, 1, 1, 16, 16, 9, 25, 25, 25, 25]", - "[1, 2, 3, 4, 5, 5, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]"}; - MakeInputBatch(input_data, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_2 = { - R"(["CD", "DL", "NY", "LA", "AU", "AU", "LA", "CD", "DL", "DL", "CD", "CD", "CD", -"LA", "LA", "NY", "AU", "AU", "AU", "AU"])", - "[36, 49, 64, 81, 100, 100, 81, 36, 49, 49, 36, 36, 36, 81, 81, 64, 100, 100, 100, " - "100]", - "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]"}; - MakeInputBatch(input_data_2, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_3 = { - R"(["BJ", "SH", "SZ", "NY", "WH", "WH", "AU", "BJ", "SH", "DL", "CD", "CD", "BJ", -"LA", "HZ", "LA", "WH", "NY", "WH", "WH"])", - "[1, 4, 9, 64, 25, 25, 100, 1, 4, 49, 36, 36, 1, 81, 16, 81, 25, 64, 25, 25]", - "[1, 2, 3, 8, 5, 5, 10, 1, 2, 7, 6, 6, 1, 9, 4, 9, 5, 8, 5, 5]"}; - MakeInputBatch(input_data_3, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - ////////////////////// Finish ////////////////////////// - std::shared_ptr result_batch; - std::shared_ptr> aggr_result_iterator; - std::shared_ptr aggr_result_iterator_base; - ASSERT_NOT_OK(expr->finish(&aggr_result_iterator_base)); - aggr_result_iterator = std::dynamic_pointer_cast>( - aggr_result_iterator_base); - - std::shared_ptr expected_result; - std::vector expected_result_string = {"[1, 2, 3, 4, 5, 6, 7, 8, 9, 10]"}; - auto res_sch = arrow::schema(ret_types); - MakeInputBatch(expected_result_string, res_sch, &expected_result); - if (aggr_result_iterator->HasNext()) { - ASSERT_NOT_OK(aggr_result_iterator->Next(&result_batch)); - ASSERT_NOT_OK(Equals(*expected_result.get(), *result_batch.get())); - } -} - -TEST(TestArrowCompute, GroupByHashAggregateWithTwoStringTest) { - ////////////////////// prepare expr_vector /////////////////////// - auto f_unique_0 = field("unique_0", utf8()); - auto f_unique_1 = field("unique_1", utf8()); - auto f_sum = field("sum", float64()); - auto f_count_all = field("count_all", int64()); - auto f_res = field("dummy_res", uint32()); - - auto arg_unique_0 = TreeExprBuilder::MakeField(f_unique_0); - auto arg_unique_1 = TreeExprBuilder::MakeField(f_unique_1); - auto arg_sum = TreeExprBuilder::MakeField(f_sum); - auto n_groupby_0 = - TreeExprBuilder::MakeFunction("action_groupby", {arg_unique_0}, uint32()); - auto n_groupby_1 = - TreeExprBuilder::MakeFunction("action_groupby", {arg_unique_1}, uint32()); - auto n_sum = TreeExprBuilder::MakeFunction("action_sum", {arg_sum}, uint32()); - auto n_count_all = TreeExprBuilder::MakeFunction("action_countLiteral_1", {}, uint32()); - auto n_schema = TreeExprBuilder::MakeFunction( - "codegen_schema", {arg_unique_0, arg_unique_1, arg_sum}, uint32()); - auto n_aggr = TreeExprBuilder::MakeFunction( - "hashAggregateArrays", {n_groupby_0, n_groupby_1, n_sum, n_count_all}, uint32()); - auto n_codegen_aggr = - TreeExprBuilder::MakeFunction("codegen_withOneInput", {n_aggr, n_schema}, uint32()); - - auto aggr_expr = TreeExprBuilder::MakeExpression(n_codegen_aggr, f_res); - - std::vector> expr_vector = {aggr_expr}; - - auto sch = arrow::schema({f_unique_0, f_unique_1, f_sum}); - std::vector> ret_types = {f_unique_0, f_unique_1, f_sum, - f_count_all}; - - /////////////////////// Create Expression Evaluator //////////////////// - std::shared_ptr expr; - arrow::compute::FunctionContext ctx; - ASSERT_NOT_OK(CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)); - std::shared_ptr input_batch; - std::vector> output_batch_list; - - ////////////////////// calculation ///////////////////// - std::vector input_data = { - R"(["a", "b", "c", "d", "e", "e", "d", "a", "b", "b", "a", "a", "a", "d", "d", "c", -"e", "e", "e", "e"])", - R"(["BJ", "SH", "SZ", "HZ", "WH", "WH", "HZ", "BJ", "SH", "SH", -"BJ", "BJ", "BJ", "HZ", "HZ", "SZ", "WH", "WH", "WH", "WH"])", - R"([1, 2, 3, 4, 5, 5, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5])"}; - MakeInputBatch(input_data, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_2 = { - R"(["f", "g", "h", "i", "j", "j", "i", "g", "h", "i", "g", "g", "g", "j", "i", "f", -"f", "i", "j", "j"])", - R"(["CD", "DL", "NY", "LA", "AU", "AU", "LA", "DL", "NY", "LA", -"DL", "DL", "DL", "AU", "LA", "CD", "CD", "LA", "AU", "AU"])", - "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]"}; - MakeInputBatch(input_data_2, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - ////////////////////// Finish ////////////////////////// - std::shared_ptr result_batch; - std::shared_ptr> aggr_result_iterator; - std::shared_ptr aggr_result_iterator_base; - ASSERT_NOT_OK(expr->finish(&aggr_result_iterator_base)); - aggr_result_iterator = std::dynamic_pointer_cast>( - aggr_result_iterator_base); - - std::shared_ptr expected_result; - std::vector expected_result_string = { - R"(["a", "b", "c", "d", "e", "f", "g", "h", "i", "j"])", - R"(["BJ", "SH", "SZ", "HZ", "WH", "CD", "DL", "NY", "LA", "AU"])", - "[5, 6, 6, 16, 30, 24, 31, 15, 44, 49]", "[5, 3, 2, 4, 6, 3, 5, 2, 5, 5]"}; - auto res_sch = arrow::schema(ret_types); - MakeInputBatch(expected_result_string, res_sch, &expected_result); - if (aggr_result_iterator->HasNext()) { - ASSERT_NOT_OK(aggr_result_iterator->Next(&result_batch)); - ASSERT_NOT_OK(Equals(*expected_result.get(), *result_batch.get())); - } -} - -TEST(TestArrowCompute, GroupByHashAggregateWithProjectedKeyTest) { - ////////////////////// prepare expr_vector /////////////////////// - auto f_unique_0 = field("unique_0", utf8()); - auto f_unique_1 = field("unique_1", utf8()); - auto f_sum = field("sum", float64()); - auto f_count_all = field("count_all", int64()); - auto f_res = field("dummy_res", uint32()); - - auto arg_unique_0 = TreeExprBuilder::MakeField(f_unique_0); - auto arg_unique_1 = TreeExprBuilder::MakeField(f_unique_1); - auto arg_sum = TreeExprBuilder::MakeField(f_sum); - auto n_projection = TreeExprBuilder::MakeFunction( - "substr", - {arg_unique_1, TreeExprBuilder::MakeLiteral((int64_t)0), - TreeExprBuilder::MakeLiteral((int64_t)2)}, - utf8()); - auto n_groupby_0 = - TreeExprBuilder::MakeFunction("action_groupby", {arg_unique_0}, uint32()); - auto n_groupby_1 = - TreeExprBuilder::MakeFunction("action_groupby", {n_projection}, uint32()); - auto n_sum = TreeExprBuilder::MakeFunction("action_sum", {arg_sum}, uint32()); - auto n_count_all = TreeExprBuilder::MakeFunction("action_countLiteral_1", {}, uint32()); - auto n_schema = TreeExprBuilder::MakeFunction( - "codegen_schema", {arg_unique_0, arg_unique_1, arg_sum}, uint32()); - auto n_aggr = TreeExprBuilder::MakeFunction( - "hashAggregateArrays", {n_groupby_0, n_groupby_1, n_sum, n_count_all}, uint32()); - auto n_codegen_aggr = - TreeExprBuilder::MakeFunction("codegen_withOneInput", {n_aggr, n_schema}, uint32()); - - auto aggr_expr = TreeExprBuilder::MakeExpression(n_codegen_aggr, f_res); - - std::vector> expr_vector = {aggr_expr}; - - auto sch = arrow::schema({f_unique_0, f_unique_1, f_sum}); - std::vector> ret_types = {f_unique_0, f_unique_1, f_sum, - f_count_all}; - - /////////////////////// Create Expression Evaluator //////////////////// - std::shared_ptr expr; - arrow::compute::FunctionContext ctx; - ASSERT_NOT_OK(CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)); - std::shared_ptr input_batch; - std::vector> output_batch_list; - - ////////////////////// calculation ///////////////////// - std::vector input_data = { - R"(["a", "b", "c", "d", "e", "e", "d", "a", "b", "b", "a", "a", "a", "d", "d", "c", -"e", "e", "e", "e"])", - R"(["BJH", "SHH", "SZN", "HZK", "WHK", "WHZ", "HZJ", "BJJ", "SHZ", "SHJ", -"BJJ", "BJH", "BJH", "HZJ", "HZJ", "SZZ", "WHH", "WHJ", "WHK", "WHK"])", - R"([1, 2, 3, 4, 5, 5, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5])"}; - MakeInputBatch(input_data, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_2 = { - R"(["f", "g", "h", "i", "j", "j", "i", "g", "h", "i", "g", "g", "g", "j", "i", "f", -"f", "i", "j", "j"])", - R"(["CD", "DL", "NY", "LA", "AU", "AU", "LA", "DL", "NY", "LA", -"DL", "DL", "DL", "AU", "LA", "CD", "CD", "LA", "AU", "AU"])", - "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]"}; - MakeInputBatch(input_data_2, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - ////////////////////// Finish ////////////////////////// - std::shared_ptr result_batch; - std::shared_ptr> aggr_result_iterator; - std::shared_ptr aggr_result_iterator_base; - ASSERT_NOT_OK(expr->finish(&aggr_result_iterator_base)); - aggr_result_iterator = std::dynamic_pointer_cast>( - aggr_result_iterator_base); - - std::shared_ptr expected_result; - std::vector expected_result_string = { - R"(["a", "b", "c", "d", "e", "f", "g", "h", "i", "j"])", - R"(["BJ", "SH", "SZ", "HZ", "WH", "CD", "DL", "NY", "LA", "AU"])", - "[5, 6, 6, 16, 30, 24, 31, 15, 44, 49]", "[5, 3, 2, 4, 6, 3, 5, 2, 5, 5]"}; - auto res_sch = arrow::schema(ret_types); - MakeInputBatch(expected_result_string, res_sch, &expected_result); - if (aggr_result_iterator->HasNext()) { - ASSERT_NOT_OK(aggr_result_iterator->Next(&result_batch)); - ASSERT_NOT_OK(Equals(*expected_result.get(), *result_batch.get())); - } -} - -TEST(TestArrowCompute, GroupByStddevSampPartialHashAggregateTest) { - ////////////////////// prepare expr_vector /////////////////////// - auto f0 = field("f0", uint32()); - auto f1 = field("f1", float64()); - auto f2 = field("f2", float64()); - auto f_unique = field("unique", uint32()); - auto f_n = field("count", float64()); - auto f_avg = field("avg", float64()); - auto f_m2 = field("m2", float64()); - auto f_res = field("res", uint32()); - - auto arg0 = TreeExprBuilder::MakeField(f0); - auto arg1 = TreeExprBuilder::MakeField(f1); - auto arg2 = TreeExprBuilder::MakeField(f2); - auto n_groupby = TreeExprBuilder::MakeFunction("action_groupby", {arg0}, uint32()); - auto n_stddev_1 = - TreeExprBuilder::MakeFunction("action_stddev_samp_partial", {arg1}, uint32()); - auto n_stddev_2 = - TreeExprBuilder::MakeFunction("action_stddev_samp_partial", {arg2}, uint32()); - auto n_schema = TreeExprBuilder::MakeFunction( - "codegen_schema", - {TreeExprBuilder::MakeField(f0), TreeExprBuilder::MakeField(f1), - TreeExprBuilder::MakeField(f2)}, - uint32()); - auto n_aggr = TreeExprBuilder::MakeFunction( - "hashAggregateArrays", {n_groupby, n_stddev_1, n_stddev_2}, uint32()); - auto n_codegen_aggr = - TreeExprBuilder::MakeFunction("codegen_withOneInput", {n_aggr, n_schema}, uint32()); - - auto aggr_expr = TreeExprBuilder::MakeExpression(n_codegen_aggr, f_res); - - std::vector> expr_vector = {aggr_expr}; - - auto sch = arrow::schema({f0, f1, f2}); - std::vector> ret_types = {f_unique, f_n, f_avg, f_m2, - f_n, f_avg, f_m2}; - - /////////////////////// Create Expression Evaluator //////////////////// - std::shared_ptr expr; - arrow::compute::FunctionContext ctx; - ASSERT_NOT_OK(CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)); - std::shared_ptr input_batch; - std::vector> output_batch_list; - - ////////////////////// calculation ///////////////////// - std::vector input_data = { - "[1, 2, 3, 4, 5, null, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]", - "[2, 4, 5, 7, 8, 2, 45, 32, 23, 12, 14, 16, 18, 19, 23, 25, 57, 59, 12, 1]", - "[2, 4, 5, 7, 8, 2, 45, 32, 23, 12, 14, 16, 18, 19, 23, 25, 57, 59, 12, 1]"}; - MakeInputBatch(input_data, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_2 = { - "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]", - "[7, 8, 4, 5, 6, 1, 34, 54, 65, 66, 78, 12, 32, 24, 32, 45, 12, 24, 35, 46]", - "[7, 8, 4, 5, 6, 1, 34, 54, 65, 66, 78, 12, 32, 24, 32, 45, 12, 24, 35, 46]"}; - MakeInputBatch(input_data_2, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_3 = { - "[1, 2, 3, 8, 5, 5, 10, 1, 2, 7, 6, 6, 1, 9, 4, 9, null, 8, 5, 5]", - "[23, 34, 56, 78, 98, 12, 23, 26, 28, 29, 21, 31, 12, 4, 5, 6, 9, 65, 45, 12]", - "[23, 34, 56, 78, 98, 12, 23, 26, 28, 29, 21, 31, 12, 4, 5, 6, 9, 65, 45, 12]"}; - MakeInputBatch(input_data_3, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - ////////////////////// Finish ////////////////////////// - std::shared_ptr result_batch; - std::shared_ptr> aggr_result_iterator; - std::shared_ptr aggr_result_iterator_base; - ASSERT_NOT_OK(expr->finish(&aggr_result_iterator_base)); - aggr_result_iterator = std::dynamic_pointer_cast>( - aggr_result_iterator_base); - - std::shared_ptr expected_result; - std::vector expected_result_string = { - "[1, 2, 3, 4, 5, null, 6, 7, 8 ,9, 10]", - "[8, 5, 3, 5, 9, 2, 7, 4, 4, 6, 7]", - "[17.875, 20.2, 28.6667, 19.8, 33.7778, 5.5, 33.5714, 42, 48 ,17.5, 21]", - "[596.875, 588.8, 1320.67, 1028.8, 8587.56, 24.5, 3729.71, 2430, 3134, 995.5, " - "1540]", - "[8, 5, 3, 5, 9, 2, 7, 4, 4, 6, 7]", - "[17.875, 20.2, 28.6667, 19.8, 33.7778, 5.5, 33.5714, 42, 48 ,17.5, 21]", - "[596.875, 588.8, 1320.67, 1028.8, 8587.56, 24.5, 3729.71, 2430, 3134, 995.5, " - "1540]"}; - auto res_sch = arrow::schema({f_unique, f_n, f_avg, f_m2, f_n, f_avg, f_m2}); - MakeInputBatch(expected_result_string, res_sch, &expected_result); - if (aggr_result_iterator->HasNext()) { - ASSERT_NOT_OK(aggr_result_iterator->Next(&result_batch)); - ASSERT_NOT_OK(Equals(*expected_result.get(), *result_batch.get())); - } -} - -TEST(TestArrowCompute, GroupByStddevSampFinalHashAggregateTest) { - ////////////////////// prepare expr_vector /////////////////////// - auto f0 = field("f0", uint32()); - auto f1 = field("f1", float64()); - auto f2 = field("f2", float64()); - auto f3 = field("f3", float64()); - auto f_unique = field("unique", uint32()); - auto f_stddev = field("stddev", float64()); - auto f_res = field("res", uint32()); - - auto arg0 = TreeExprBuilder::MakeField(f0); - auto arg1 = TreeExprBuilder::MakeField(f1); - auto arg2 = TreeExprBuilder::MakeField(f2); - auto arg3 = TreeExprBuilder::MakeField(f3); - auto n_groupby = TreeExprBuilder::MakeFunction("action_groupby", {arg0}, uint32()); - auto n_stddev = TreeExprBuilder::MakeFunction("action_stddev_samp_final", - {arg1, arg2, arg3}, uint32()); - auto n_schema = TreeExprBuilder::MakeFunction( - "codegen_schema", - {TreeExprBuilder::MakeField(f0), TreeExprBuilder::MakeField(f1), - TreeExprBuilder::MakeField(f2), TreeExprBuilder::MakeField(f3)}, - uint32()); - auto n_aggr = TreeExprBuilder::MakeFunction("hashAggregateArrays", - {n_groupby, n_stddev}, uint32()); - auto n_codegen_aggr = - TreeExprBuilder::MakeFunction("codegen_withOneInput", {n_aggr, n_schema}, uint32()); - - auto aggr_expr = TreeExprBuilder::MakeExpression(n_codegen_aggr, f_res); - - std::vector> expr_vector = {aggr_expr}; - - auto sch = arrow::schema({f0, f1, f2, f3}); - std::vector> ret_types = {f_unique, f_stddev}; - - /////////////////////// Create Expression Evaluator //////////////////// - std::shared_ptr expr; - arrow::compute::FunctionContext ctx; - ASSERT_NOT_OK(CreateCodeGenerator(ctx.memory_pool(), sch, expr_vector, ret_types, &expr, true)); - std::shared_ptr input_batch; - std::vector> output_batch_list; - - ////////////////////// calculation ///////////////////// - std::vector input_data = { - "[1, 2, 3, 4, 5, null, 4, 1, 2, 2, 1, 1, 1, 4, 4, 3, 5, 5, 5, 5]", - "[2, 4, 5, 7, 8, 2, 45, 32, 23, 12, 14, 16, 18, 19, 23, 25, 57, 59, 12, 1]", - "[2, 4, 5, 7, 8, 2, 45, 32, 23, 12, 14, 16, 18, 19, 23, 25, 57, 59, 12, 1]", - "[2, 4, 5, 7, 8, 2, 45, 32, 23, 12, 14, 16, 18, 19, 23, 25, 57, 59, 12, 1]"}; - MakeInputBatch(input_data, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - std::vector input_data_2 = { - "[6, 7, 8, 9, 10, 10, 9, 6, 7, 7, 6, 6, 6, 9, 9, 8, 10, 10, 10, 10]", - "[7, 8, 4, 5, 6, 1, 34, 54, 65, 66, 78, 12, 32, 24, 32, 45, 12, 24, 35, 46]", - "[2, 4, 5, 7, 8, 2, 45, 32, 23, 12, 14, 16, 18, 19, 23, 25, 57, 59, 12, 1]", - "[2, 4, 5, 7, 8, 2, 45, 32, 23, 12, 14, 16, 18, 19, 23, 25, 57, 59, 12, 1]"}; - MakeInputBatch(input_data_2, sch, &input_batch); - ASSERT_NOT_OK(expr->evaluate(input_batch, &output_batch_list)); - - ////////////////////// Finish ////////////////////////// - std::shared_ptr result_batch; - std::shared_ptr> aggr_result_iterator; - std::shared_ptr aggr_result_iterator_base; - ASSERT_NOT_OK(expr->finish(&aggr_result_iterator_base)); - aggr_result_iterator = std::dynamic_pointer_cast>( - aggr_result_iterator_base); - - std::shared_ptr expected_result; - std::vector expected_result_string = { - "[1, 2, 3, 4, 5, null, 6, 7, 8 ,9, 10]", - "[8.49255, 6.93137, 7.6489, 13.5708, 17.4668, 1.41421, 8.52779, 6.23633, " - "5.58903, 12.535, 24.3544]"}; - auto res_sch = arrow::schema({f_unique, f_stddev}); - MakeInputBatch(expected_result_string, res_sch, &expected_result); - if (aggr_result_iterator->HasNext()) { - ASSERT_NOT_OK(aggr_result_iterator->Next(&result_batch)); - ASSERT_NOT_OK(Equals(*expected_result.get(), *result_batch.get())); - } -} - -} // namespace codegen -} // namespace sparkcolumnarplugin diff --git a/cpp/src/tests/arrow_compute_test_wscg.cc b/cpp/src/tests/arrow_compute_test_wscg.cc index f368a48ce..d2eaf58e9 100644 --- a/cpp/src/tests/arrow_compute_test_wscg.cc +++ b/cpp/src/tests/arrow_compute_test_wscg.cc @@ -3857,8 +3857,8 @@ TEST(TestArrowComputeWSCG, WSCGTestGroupbyHashAggregateTwoKeys) { auto n_avg = TreeExprBuilder::MakeFunction("action_avgByCount", {arg2, arg0}, uint32()); auto n_stddev = TreeExprBuilder::MakeFunction("action_stddev_samp_final", {arg2, arg3, arg4}, uint32()); - auto n_proj = TreeExprBuilder::MakeFunction("aggregateExpressions", - {arg0, arg1, arg2, arg3, arg4}, uint32()); + auto n_proj = TreeExprBuilder::MakeFunction( + "aggregateExpressions", {arg0, arg1, arg2, arg3, arg4, arg5}, uint32()); auto n_action = TreeExprBuilder::MakeFunction( "aggregateActions", {n_groupby, n_groupby_5, n_sum_count, n_min, n_max, n_avg, n_stddev}, uint32());