This repository has been archived by the owner on Sep 18, 2023. It is now read-only.
-
Notifications
You must be signed in to change notification settings - Fork 77
[NSE-400] Native Arrow Row to columnar support #637
Merged
Merged
Changes from all commits
Commits
Show all changes
24 commits
Select commit
Hold shift + click to select a range
cfa49bd
Support ArrowRowToColumnar Optimization
haojinIntel 9763bf2
Replace expired code
haojinIntel db1165c
Add the code to convert recordbatch to columnarBatch
haojinIntel 44f36f2
Add unit test on java size
haojinIntel 5061c2d
Update the unit tests
haojinIntel a9e9774
Fix the bug when reading decimal value from unsafeRow
haojinIntel df7df67
Use ArrowRowToColumnarExec instead of RowToArrowColumnarExec
haojinIntel e580666
Use clang-format to standardize the CPP code format
haojinIntel f2418a7
enable arrowRowToColumnarExec
haojinIntel 43afdfb
Add the metrics for ArrowRowToColumnarExec
haojinIntel b03668d
Add the metrics for ArrowRowToColumnarExec and unsupport Codegen
haojinIntel 723e739
Add parameter 'spark.oap.sql.columnar.rowtocolumnar' to control Arrow…
haojinIntel b1b267a
Remove useless code
haojinIntel cc6c9eb
Release arrowbuf after return recordbatch
haojinIntel 7221c8f
Fix the processTime metric for ArrowRowToColumnarExec
haojinIntel aa84c18
Merge branch 'master' into rowToColumnSupport
haojinIntel 8cbe0a9
Refine the code of ArrowRowToColumnar operator
haojinIntel ba76a11
Add more metrics to detect the elapse time of each action
haojinIntel 0dbf08b
Small fix about allocating buffer for unsafeRow
haojinIntel 7c0c372
Remove useless code
haojinIntel 9373bb7
Remove useless metrics for ArrowRowToColumnarExec
haojinIntel 9370401
Fall back to use java RowToColumnarExec when the row is not unsafeRow…
haojinIntel ef1d109
Fix the bug for decimal format
haojinIntel f1f2f2c
fix format
zhouyuan File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
31 changes: 31 additions & 0 deletions
31
...-sql-engine/core/src/main/java/com/intel/oap/vectorized/ArrowRowToColumnarJniWrapper.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,31 @@ | ||
/* | ||
* 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.arrow.vector.ipc.message.ArrowRecordBatch; | ||
|
||
public class ArrowRowToColumnarJniWrapper { | ||
public ArrowRowToColumnarJniWrapper() throws Exception { | ||
JniUtils.getInstance(); | ||
} | ||
|
||
public native byte[] nativeConvertRowToColumnar( | ||
byte[] schema, long[] rowLength, | ||
long bufferAddress, long memoryPollID) throws RuntimeException; | ||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
203 changes: 203 additions & 0 deletions
203
native-sql-engine/core/src/main/scala/com/intel/oap/execution/ArrowRowToColumnarExec.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,203 @@ | ||
/* | ||
* 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 java.util.concurrent.TimeUnit._ | ||
|
||
import scala.collection.mutable.ListBuffer | ||
import com.intel.oap.expression.ConverterUtils | ||
import com.intel.oap.sql.execution.RowToColumnConverter | ||
import com.intel.oap.vectorized.{ArrowRowToColumnarJniWrapper, ArrowWritableColumnVector, CloseableColumnBatchIterator} | ||
import org.apache.arrow.dataset.jni.UnsafeRecordBatchSerializer | ||
import org.apache.arrow.memory.ArrowBuf | ||
import org.apache.arrow.vector.ipc.message.ArrowRecordBatch | ||
import org.apache.arrow.vector.types.pojo.Schema | ||
import org.apache.spark.rdd.RDD | ||
import org.apache.spark.sql.catalyst.InternalRow | ||
import org.apache.spark.sql.catalyst.expressions.UnsafeRow | ||
import org.apache.spark.sql.execution.{RowToColumnarExec, SparkPlan} | ||
import org.apache.spark.sql.execution.datasources.v2.arrow.{SparkMemoryUtils, SparkSchemaUtils} | ||
import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils.UnsafeItr | ||
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} | ||
import org.apache.spark.sql.execution.vectorized.WritableColumnVector | ||
import org.apache.spark.sql.types._ | ||
import org.apache.spark.sql.util.ArrowUtils | ||
import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} | ||
import org.apache.spark.unsafe.Platform | ||
|
||
|
||
class ArrowRowToColumnarExec(child: SparkPlan) extends RowToColumnarExec(child = child) { | ||
override def nodeName: String = "ArrowRowToColumnarExec" | ||
|
||
buildCheck() | ||
|
||
def buildCheck(): Unit = { | ||
val schema = child.schema | ||
for (field <- schema.fields) { | ||
field.dataType match { | ||
case d: BooleanType => | ||
case d: ByteType => | ||
case d: ShortType => | ||
case d: IntegerType => | ||
case d: LongType => | ||
case d: FloatType => | ||
case d: DoubleType => | ||
case d: StringType => | ||
case d: DateType => | ||
case d: DecimalType => | ||
case d: TimestampType => | ||
case d: BinaryType => | ||
case _ => | ||
throw new UnsupportedOperationException(s"${field.dataType} " + | ||
s"is not supported in ArrowColumnarToRowExec.") | ||
} | ||
} | ||
} | ||
|
||
override lazy val metrics: Map[String, SQLMetric] = Map( | ||
"numInputRows" -> SQLMetrics.createMetric(sparkContext, "number of input rows"), | ||
"numOutputBatches" -> SQLMetrics.createMetric(sparkContext, "number of output batches"), | ||
"processTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to convert") | ||
) | ||
|
||
override def doExecuteColumnar(): RDD[ColumnarBatch] = { | ||
val numInputRows = longMetric("numInputRows") | ||
val numOutputBatches = longMetric("numOutputBatches") | ||
val processTime = longMetric("processTime") | ||
// Instead of creating a new config we are reusing columnBatchSize. In the future if we do | ||
// combine with some of the Arrow conversion tools we will need to unify some of the configs. | ||
val numRows = conf.columnBatchSize | ||
// This avoids calling `schema` in the RDD closure, so that we don't need to include the entire | ||
// plan (this) in the closure. | ||
val localSchema = this.schema | ||
child.execute().mapPartitions { rowIterator => | ||
|
||
val jniWrapper = new ArrowRowToColumnarJniWrapper() | ||
val timeZoneId = SparkSchemaUtils.getLocalTimezoneID() | ||
val arrowSchema = ArrowUtils.toArrowSchema(schema, timeZoneId) | ||
var schemaBytes: Array[Byte] = null | ||
|
||
if (rowIterator.hasNext) { | ||
val res = new Iterator[ColumnarBatch] { | ||
private val converters = new RowToColumnConverter(localSchema) | ||
private var last_cb: ColumnarBatch = null | ||
private var elapse: Long = 0 | ||
// Allocate large buffer to store the numRows rows | ||
val bufferSize = 134217728 // 128M can estimator the buffer size based on the data type | ||
val allocator = SparkMemoryUtils.contextAllocator() | ||
val arrowBuf: ArrowBuf = allocator.buffer(bufferSize) | ||
override def hasNext: Boolean = { | ||
rowIterator.hasNext | ||
} | ||
|
||
override def next(): ColumnarBatch = { | ||
var isUnsafeRow = true | ||
var firstRow = InternalRow.apply() | ||
var hasNextRow = false | ||
if (rowIterator.hasNext) { | ||
firstRow = rowIterator.next() | ||
hasNextRow = true | ||
} | ||
if (!firstRow.isInstanceOf[UnsafeRow]) { | ||
isUnsafeRow = false | ||
} | ||
|
||
if (arrowBuf != null && isUnsafeRow) { | ||
val rowLength = new ListBuffer[Long]() | ||
var rowCount = 0 | ||
var offset = 0 | ||
val start = System.nanoTime() | ||
|
||
assert(firstRow.isInstanceOf[UnsafeRow]) | ||
val unsafeRow = firstRow.asInstanceOf[UnsafeRow] | ||
val sizeInBytes = unsafeRow.getSizeInBytes | ||
Platform.copyMemory(unsafeRow.getBaseObject, unsafeRow.getBaseOffset, | ||
null, arrowBuf.memoryAddress() + offset, sizeInBytes) | ||
offset += sizeInBytes | ||
rowLength += sizeInBytes.toLong | ||
rowCount += 1 | ||
|
||
while (rowCount < numRows && rowIterator.hasNext) { | ||
val row = rowIterator.next() // UnsafeRow | ||
assert(row.isInstanceOf[UnsafeRow]) | ||
val unsafeRow = row.asInstanceOf[UnsafeRow] | ||
val sizeInBytes = unsafeRow.getSizeInBytes | ||
Platform.copyMemory(unsafeRow.getBaseObject, unsafeRow.getBaseOffset, | ||
null, arrowBuf.memoryAddress() + offset, sizeInBytes) | ||
offset += sizeInBytes | ||
rowLength += sizeInBytes.toLong | ||
rowCount += 1 | ||
} | ||
if (schemaBytes == null) { | ||
schemaBytes = ConverterUtils.getSchemaBytesBuf(arrowSchema) | ||
} | ||
val serializedRecordBatch = jniWrapper.nativeConvertRowToColumnar(schemaBytes, rowLength.toArray, | ||
arrowBuf.memoryAddress(), SparkMemoryUtils.contextMemoryPool().getNativeInstanceId) | ||
numInputRows += rowCount | ||
numOutputBatches += 1 | ||
val rb = UnsafeRecordBatchSerializer.deserializeUnsafe(allocator, serializedRecordBatch) | ||
val output = ConverterUtils.fromArrowRecordBatch(arrowSchema, rb) | ||
val outputNumRows = rb.getLength | ||
ConverterUtils.releaseArrowRecordBatch(rb) | ||
last_cb = new ColumnarBatch(output.map(v => v.asInstanceOf[ColumnVector]).toArray, outputNumRows) | ||
elapse = System.nanoTime() - start | ||
processTime.set(NANOSECONDS.toMillis(elapse)) | ||
last_cb | ||
} else { | ||
logInfo("the buffer allocated failed and will fall back to non arrow optimization") | ||
val vectors: Seq[WritableColumnVector] = | ||
ArrowWritableColumnVector.allocateColumns(numRows, schema) | ||
var rowCount = 0 | ||
|
||
val start = System.nanoTime() | ||
converters.convert(firstRow, vectors.toArray) | ||
elapse += System.nanoTime() - start | ||
rowCount += 1 | ||
|
||
while (rowCount < numRows && rowIterator.hasNext) { | ||
val row = rowIterator.next() | ||
val start = System.nanoTime() | ||
converters.convert(row, vectors.toArray) | ||
elapse += System.nanoTime() - start | ||
rowCount += 1 | ||
} | ||
vectors.foreach(v => v.asInstanceOf[ArrowWritableColumnVector].setValueCount(rowCount)) | ||
processTime.set(NANOSECONDS.toMillis(elapse)) | ||
numInputRows += rowCount | ||
numOutputBatches += 1 | ||
last_cb = new ColumnarBatch(vectors.toArray, rowCount) | ||
last_cb | ||
} | ||
} | ||
} | ||
new CloseableColumnBatchIterator(res) | ||
} else { | ||
Iterator.empty | ||
} | ||
} | ||
} | ||
|
||
override def canEqual(other: Any): Boolean = other.isInstanceOf[ArrowRowToColumnarExec] | ||
|
||
override def equals(other: Any): Boolean = other match { | ||
case that: ArrowRowToColumnarExec => | ||
(that canEqual this) && super.equals(that) | ||
case _ => false | ||
} | ||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
new buf here