Skip to content
This repository has been archived by the owner on Sep 18, 2023. It is now read-only.

[NSE-400] Optimize ColumnarToRow Operator in NSE. #401

Merged
merged 20 commits into from
Jul 22, 2021
Merged
Show file tree
Hide file tree
Changes from 18 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
/*
* 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.vectorized;

import org.apache.spark.sql.catalyst.expressions.UnsafeRow;

import java.io.IOException;

public class ArrowColumnarToRowJniWrapper {

public ArrowColumnarToRowJniWrapper() throws IOException {
JniUtils.getInstance();
}

public native long nativeConvertColumnarToRow(
byte[] schema, int numRows, long[] bufAddrs, long[] bufSizes, long memory_pool_id) throws RuntimeException;

public native boolean nativeHasNext(long instanceID);

public native UnsafeRow nativeNext(long instanceID);
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ package com.intel.oap

import com.intel.oap.execution._
import com.intel.oap.sql.execution.RowToArrowColumnarExec

import org.apache.spark.internal.config._
import org.apache.spark.internal.Logging
import org.apache.spark.sql.{SparkSession, SparkSessionExtensions}
Expand All @@ -35,6 +34,7 @@ import org.apache.spark.sql.execution.joins._
import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, ColumnarArrowEvalPythonExec}
import org.apache.spark.sql.execution.window.WindowExec
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.CalendarIntervalType

case class ColumnarPreOverrides() extends Rule[SparkPlan] {
val columnarConf: ColumnarPluginConfig = ColumnarPluginConfig.getSessionConf
Expand Down Expand Up @@ -305,7 +305,18 @@ case class ColumnarPostOverrides() extends Rule[SparkPlan] {
val children = r.children.map(c =>
c match {
case c: ColumnarToRowExec =>
c.withNewChildren(c.children.map(replaceWithColumnarPlan))
if (columnarConf.enableArrowColumnarToRow) {
try {
val child = replaceWithColumnarPlan(c.child)
ArrowColumnarToRowExec(child)
} catch {
case _: Throwable =>
logInfo("ArrowColumnarToRow : Falling back to ColumnarToRow...")
c.withNewChildren(c.children.map(replaceWithColumnarPlan))
}
} else {
c.withNewChildren(c.children.map(replaceWithColumnarPlan))
}
case other =>
replaceWithColumnarPlan(other)
})
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,9 @@ class ColumnarPluginConfig(conf: SQLConf) extends Logging {
val enableColumnarShuffledHashJoin: Boolean =
conf.getConfString("spark.oap.sql.columnar.shuffledhashjoin", "true").toBoolean && enableCpu

val enableArrowColumnarToRow: Boolean =
conf.getConfString("spark.oap.sql.columnar.columnartorow", "false").toBoolean && enableCpu

// enable or disable columnar sortmergejoin
// this should be set with preferSortMergeJoin=false
val enableColumnarSortMergeJoin: Boolean =
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,149 @@
/*
* 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.execution

import com.intel.oap.expression.ConverterUtils
import com.intel.oap.vectorized.{ArrowColumnarToRowJniWrapper, ArrowWritableColumnVector}
import org.apache.arrow.vector.types.pojo.{Field, Schema}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder, UnsafeRow}
import org.apache.spark.sql.catalyst.plans.physical.Partitioning
import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}

import scala.collection.JavaConverters._
import scala.collection.mutable.ListBuffer
import scala.concurrent.duration._

case class ArrowColumnarToRowExec(child: SparkPlan) extends UnaryExecNode {
override def nodeName: String = "ArrowColumnarToRow"

assert(child.supportsColumnar)

override def output: Seq[Attribute] = child.output

override def outputPartitioning: Partitioning = child.outputPartitioning

override def outputOrdering: Seq[SortOrder] = child.outputOrdering

buildCheck()

def buildCheck(): Unit = {
val schema = child.schema
for (field <- schema.fields) {
try {
ConverterUtils.checkIfTypeSupported(field.dataType)
} catch {
case e: UnsupportedOperationException =>
throw new UnsupportedOperationException(
s"${field.dataType} is not supported in ArrowColumnarToRowExec.")
}
}
}

override lazy val metrics: Map[String, SQLMetric] = Map(
"numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"),
"numInputBatches" -> SQLMetrics.createMetric(sparkContext, "number of input batches"),
"convertTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to convert"),
"hasNextTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to has next"),
"nextTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to next")
)

override def doExecute(): RDD[InternalRow] = {
val numOutputRows = longMetric("numOutputRows")
val numInputBatches = longMetric("numInputBatches")
val convertTime = longMetric("convertTime")
val hasNextTime = longMetric("hasNextTime")
val nextTime = longMetric("nextTime")

child.executeColumnar().mapPartitions { batches =>
// TODO:: pass the jni jniWrapper and arrowSchema and serializeSchema method by broadcast
val jniWrapper = new ArrowColumnarToRowJniWrapper()
var arrowSchema: Array[Byte] = null

def serializeSchema(fields: Seq[Field]): Array[Byte] = {
val schema = new Schema(fields.asJava)
ConverterUtils.getSchemaBytesBuf(schema)
}

batches.flatMap { batch =>
numInputBatches += 1
numOutputRows += batch.numRows()

if (batch.numRows == 0 || batch.numCols == 0) {
logInfo(s"Skip ColumnarBatch of ${batch.numRows} rows, ${batch.numCols} cols")
Iterator.empty
} else {
val bufAddrs = new ListBuffer[Long]()
val bufSizes = new ListBuffer[Long]()
val fields = new ListBuffer[Field]()
(0 until batch.numCols).foreach { idx =>
val column = batch.column(idx).asInstanceOf[ArrowWritableColumnVector]
fields += column.getValueVector.getField
column.getValueVector
.getBuffers(false)
.foreach { buffer =>
bufAddrs += buffer.memoryAddress()
bufSizes += buffer.readableBytes()
}
}

if (arrowSchema == null) {
arrowSchema = serializeSchema(fields)
}

val beforeConvert = System.nanoTime()

val instanceID = jniWrapper.nativeConvertColumnarToRow(
arrowSchema, batch.numRows, bufAddrs.toArray, bufSizes.toArray,
SparkMemoryUtils.contextMemoryPool().getNativeInstanceId)

convertTime += NANOSECONDS.toMillis(System.nanoTime() - beforeConvert)

new Iterator[InternalRow] {
override def hasNext: Boolean = {
val beforeHasNext = System.nanoTime()
val result = jniWrapper.nativeHasNext(instanceID)

hasNextTime += NANOSECONDS.toMillis(System.nanoTime() - beforeHasNext)
result

}
override def next: UnsafeRow = {
val beforeNext = System.nanoTime()
val row = jniWrapper.nativeNext(instanceID)
nextTime += NANOSECONDS.toMillis(System.nanoTime() - beforeNext)
row
}
}
}
}
}
}

override def canEqual(other: Any): Boolean = other.isInstanceOf[ArrowColumnarToRowExec]

override def equals(other: Any): Boolean = other match {
case that: ArrowColumnarToRowExec =>
(that canEqual this)
case _ => false
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -279,6 +279,9 @@ case class ColumnarHashAggregateExec(
def getResForAggregateAndGroupingLiteral: ColumnarBatch = {
val resultColumnVectors =
ArrowWritableColumnVector.allocateColumns(1, resultStructType)
for (vector <- resultColumnVectors) {
vector.getValueVector.setValueCount(1)
}
var idx = 0
for (exp <- groupingExpressions) {
val out_res = exp.children.head.asInstanceOf[Literal].value
Expand Down Expand Up @@ -376,6 +379,9 @@ case class ColumnarHashAggregateExec(
}
val resultColumnVectors =
ArrowWritableColumnVector.allocateColumns(1, resultStructType)
for (vector <- resultColumnVectors) {
vector.getValueVector.setValueCount(1)
}
// If groupby is not required, for Final mode, a default value will be
// returned if input is empty.
var idx = 0
Expand Down
1 change: 1 addition & 0 deletions native-sql-engine/cpp/src/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -467,6 +467,7 @@ set(SPARK_COLUMNAR_PLUGIN_SRCS
codegen/arrow_compute/ext/sort_kernel.cc
codegen/arrow_compute/ext/kernels_ext.cc
shuffle/splitter.cc
operators/unsafe_row_writer_and_reader.cc
precompile/hash_map.cc
precompile/sparse_hash_map.cc
precompile/builder.cc
Expand Down
Loading