From 0a0d7ad0a6865599c8c4b332aa7bbb7193cc6ee1 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 6 Dec 2017 00:12:02 -0800 Subject: [PATCH 01/23] [SPARK-16060][SQL] Support Vectorized ORC Reader --- .../apache/spark/sql/internal/SQLConf.scala | 7 + .../sql/execution/DataSourceScanExec.scala | 3 + .../orc/OrcColumnarBatchReader.scala | 442 ++++++++++++++++++ .../datasources/orc/OrcFileFormat.scala | 65 ++- .../execution/datasources/orc/OrcUtils.scala | 24 +- .../spark/sql/hive/orc/OrcReadBenchmark.scala | 357 ++++++++++++++ 6 files changed, 876 insertions(+), 22 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 5c61f10bb71ad..955cddedda4c0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -386,6 +386,11 @@ object SQLConf { .checkValues(Set("hive", "native")) .createWithDefault("native") + val ORC_VECTORIZED_READER_ENABLED = buildConf("spark.sql.orc.vectorizedReader.enabled") + .doc("Enables vectorized orc decoding.") + .booleanConf + .createWithDefault(true) + val ORC_FILTER_PUSHDOWN_ENABLED = buildConf("spark.sql.orc.filterPushdown") .doc("When true, enable filter pushdown for ORC files.") .booleanConf @@ -1183,6 +1188,8 @@ class SQLConf extends Serializable with Logging { def orcCompressionCodec: String = getConf(ORC_COMPRESSION) + def orcVectorizedReaderEnabled: Boolean = getConf(ORC_VECTORIZED_READER_ENABLED) + def parquetCompressionCodec: String = getConf(PARQUET_COMPRESSION) def parquetVectorizedReaderEnabled: Boolean = getConf(PARQUET_VECTORIZED_READER_ENABLED) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index d1ff82c7c06bc..664edf05dbdda 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning} import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat => ParquetSource} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.sources.{BaseRelation, Filter} @@ -169,6 +170,8 @@ case class FileSourceScanExec( val needsUnsafeRowConversion: Boolean = if (relation.fileFormat.isInstanceOf[ParquetSource]) { SparkSession.getActiveSession.get.sessionState.conf.parquetVectorizedReaderEnabled + } else if (relation.fileFormat.isInstanceOf[OrcFileFormat]) { + SparkSession.getActiveSession.get.sessionState.conf.orcVectorizedReaderEnabled } else { false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala new file mode 100644 index 0000000000000..e99aef2800d85 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala @@ -0,0 +1,442 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.orc + +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext} +import org.apache.hadoop.mapreduce.lib.input.FileSplit +import org.apache.orc._ +import org.apache.orc.mapred.OrcInputFormat +import org.apache.orc.storage.ql.exec.vector._ + +import org.apache.spark.internal.Logging +import org.apache.spark.memory.MemoryMode +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.vectorized._ +import org.apache.spark.sql.types._ + + +/** + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch. + */ +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging { + /** + * ORC File Reader. + */ + private var reader: Reader = _ + + /** + * Vectorized Row Batch. + */ + private var batch: VectorizedRowBatch = _ + + /** + * Requested Column IDs. + */ + private var requestedColIds: Array[Int] = _ + + /** + * Record reader from row batch. + */ + private var rows: org.apache.orc.RecordReader = _ + + /** + * Required Schema. + */ + private var requiredSchema: StructType = _ + + /** + * ColumnarBatch for vectorized execution by whole-stage codegen. + */ + private var columnarBatch: ColumnarBatch = _ + + /** + * Writable columnVectors of ColumnarBatch. + */ + private var columnVectors: Seq[WritableColumnVector] = _ + + /** + * The number of rows read and considered to be returned. + */ + private var rowsReturned: Long = 0L + + /** + * Total number of rows. + */ + private var totalRowCount: Long = 0L + + override def getCurrentKey: Void = null + + override def getCurrentValue: ColumnarBatch = columnarBatch + + override def getProgress: Float = rowsReturned.toFloat / totalRowCount + + override def nextKeyValue(): Boolean = nextBatch() + + override def close(): Unit = { + if (columnarBatch != null) { + columnarBatch.close() + columnarBatch = null + } + if (rows != null) { + rows.close() + rows = null + } + } + + /** + * Initialize ORC file reader and batch record reader. + * Please note that `setRequiredSchema` is needed to be called after this. + */ + override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = { + val fileSplit = inputSplit.asInstanceOf[FileSplit] + val conf = taskAttemptContext.getConfiguration + reader = OrcFile.createReader( + fileSplit.getPath, + OrcFile.readerOptions(conf) + .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf)) + .filesystem(fileSplit.getPath.getFileSystem(conf))) + + val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength) + rows = reader.rows(options) + } + + /** + * Set required schema and partition information. + * With this information, this creates ColumnarBatch with the full schema. + */ + def setRequiredSchema( + orcSchema: TypeDescription, + requestedColIds: Array[Int], + resultSchema: StructType, + requiredSchema: StructType, + partitionValues: InternalRow): Unit = { + batch = orcSchema.createRowBatch(OrcColumnarBatchReader.DEFAULT_SIZE) + totalRowCount = reader.getNumberOfRows + logDebug(s"totalRowCount = $totalRowCount") + + this.requiredSchema = requiredSchema + this.requestedColIds = requestedColIds + + val memMode = OrcColumnarBatchReader.DEFAULT_MEMORY_MODE + val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE + if (memMode == MemoryMode.OFF_HEAP) { + columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema) + } else { + columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema) + } + columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity) + + if (partitionValues.numFields > 0) { + val partitionIdx = requiredSchema.fields.length + for (i <- 0 until partitionValues.numFields) { + ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i) + columnVectors(i + partitionIdx).setIsConstant() + } + } + } + + /** + * Return true if there exists more data in the next batch. If exists, prepare the next batch + * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns. + */ + private def nextBatch(): Boolean = { + if (rowsReturned >= totalRowCount) { + return false + } + + rows.nextBatch(batch) + val batchSize = batch.size + if (batchSize == 0) { + return false + } + rowsReturned += batchSize + columnarBatch.reset() + columnarBatch.setNumRows(batchSize) + + var i = 0 + while (i < requiredSchema.length) { + val field = requiredSchema(i) + val toColumn = columnVectors(i) + + if (requestedColIds(i) < 0) { + toColumn.appendNulls(batchSize) + } else { + val fromColumn = batch.cols(requestedColIds(i)) + + if (fromColumn.isRepeating) { + if (fromColumn.isNull(0)) { + toColumn.appendNulls(batchSize) + } else { + field.dataType match { + case BooleanType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1 + toColumn.appendBooleans(batchSize, data) + + case ByteType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte + toColumn.appendBytes(batchSize, data) + case ShortType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort + toColumn.appendShorts(batchSize, data) + case IntegerType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt + toColumn.appendInts(batchSize, data) + case LongType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) + toColumn.appendLongs(batchSize, data) + + case DateType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt + toColumn.appendInts(batchSize, data) + + case TimestampType => + val data = fromColumn.asInstanceOf[TimestampColumnVector] + toColumn.appendLongs(batchSize, data.time(0) * 1000L + data.nanos(0) / 1000L) + + case FloatType => + val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0).toFloat + toColumn.appendFloats(batchSize, data) + case DoubleType => + val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0) + toColumn.appendDoubles(batchSize, data) + + case StringType => + val data = fromColumn.asInstanceOf[BytesColumnVector] + var index = 0 + while (index < batchSize) { + toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0)) + index += 1 + } + case BinaryType => + val data = fromColumn.asInstanceOf[BytesColumnVector] + var index = 0 + while (index < batchSize) { + toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0)) + index += 1 + } + + case DecimalType.Fixed(precision, scale) => + val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(0) + val value = Decimal(d.getHiveDecimal.bigDecimalValue, d.precision(), d.scale) + value.changePrecision(precision, scale) + if (precision <= Decimal.MAX_INT_DIGITS) { + toColumn.appendInts(batchSize, value.toUnscaledLong.toInt) + } else if (precision <= Decimal.MAX_LONG_DIGITS) { + toColumn.appendLongs(batchSize, value.toUnscaledLong) + } else { + val bytes = value.toJavaBigDecimal.unscaledValue.toByteArray + var index = 0 + while (index < batchSize) { + toColumn.appendByteArray(bytes, 0, bytes.length) + index += 1 + } + } + + case dt => + throw new UnsupportedOperationException(s"Unsupported Data Type: $dt") + } + } + } else if (!field.nullable || fromColumn.noNulls) { + field.dataType match { + case BooleanType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector + var index = 0 + while (index < batchSize) { + toColumn.appendBoolean(data(index) == 1) + index += 1 + } + + case ByteType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector + var index = 0 + while (index < batchSize) { + toColumn.appendByte(data(index).toByte) + index += 1 + } + case ShortType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector + var index = 0 + while (index < batchSize) { + toColumn.appendShort(data(index).toShort) + index += 1 + } + case IntegerType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector + var index = 0 + while (index < batchSize) { + toColumn.appendInt(data(index).toInt) + index += 1 + } + case LongType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector + toColumn.appendLongs(batchSize, data, 0) + + case DateType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector + var index = 0 + while (index < batchSize) { + toColumn.appendInt(data(index).toInt) + index += 1 + } + + case TimestampType => + val data = fromColumn.asInstanceOf[TimestampColumnVector] + var index = 0 + while (index < batchSize) { + toColumn.appendLong(data.time(index) * 1000L + data.nanos(index) / 1000L) + index += 1 + } + + case FloatType => + val data = fromColumn.asInstanceOf[DoubleColumnVector].vector + var index = 0 + while (index < batchSize) { + toColumn.appendFloat(data(index).toFloat) + index += 1 + } + case DoubleType => + val data = fromColumn.asInstanceOf[DoubleColumnVector].vector + toColumn.appendDoubles(batchSize, data, 0) + + case StringType => + val data = fromColumn.asInstanceOf[BytesColumnVector] + var index = 0 + while (index < batchSize) { + toColumn.appendByteArray( + data.vector(index), data.start(index), data.length(index)) + index += 1 + } + case BinaryType => + val data = fromColumn.asInstanceOf[BytesColumnVector] + var index = 0 + while (index < batchSize) { + toColumn.appendByteArray(data.vector(index), data.start(index), data.length(index)) + index += 1 + } + + case DecimalType.Fixed(precision, scale) => + val data = fromColumn.asInstanceOf[DecimalColumnVector] + var index = 0 + while (index < batchSize) { + val d = data.vector(index) + val value = Decimal(d.getHiveDecimal.bigDecimalValue, d.precision(), d.scale) + value.changePrecision(precision, scale) + if (precision <= Decimal.MAX_INT_DIGITS) { + toColumn.appendInt(value.toUnscaledLong.toInt) + } else if (precision <= Decimal.MAX_LONG_DIGITS) { + toColumn.appendLong(value.toUnscaledLong) + } else { + val bytes = value.toJavaBigDecimal.unscaledValue.toByteArray + toColumn.appendByteArray(bytes, 0, bytes.length) + } + index += 1 + } + + case dt => + throw new UnsupportedOperationException(s"Unsupported Data Type: $dt") + } + } else { + for (index <- 0 until batchSize) { + if (fromColumn.isNull(index)) { + toColumn.appendNull() + } else { + field.dataType match { + case BooleanType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector(index) == 1 + toColumn.appendBoolean(data) + case ByteType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector(index).toByte + toColumn.appendByte(data) + case ShortType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector(index).toShort + toColumn.appendShort(data) + case IntegerType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector(index).toInt + toColumn.appendInt(data) + case LongType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector(index) + toColumn.appendLong(data) + + case DateType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector(index).toInt + toColumn.appendInt(data) + + case TimestampType => + val data = fromColumn.asInstanceOf[TimestampColumnVector] + toColumn.appendLong(data.time(index) * 1000L + data.nanos(index) / 1000L) + + case FloatType => + val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(index).toFloat + toColumn.appendFloat(data) + case DoubleType => + val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(index) + toColumn.appendDouble(data) + + case StringType => + val v = fromColumn.asInstanceOf[BytesColumnVector] + toColumn.appendByteArray(v.vector(index), v.start(index), v.length(index)) + + case BinaryType => + val v = fromColumn.asInstanceOf[BytesColumnVector] + toColumn.appendByteArray(v.vector(index), v.start(index), v.length(index)) + + case DecimalType.Fixed(precision, scale) => + val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(index) + val value = Decimal(d.getHiveDecimal.bigDecimalValue, d.precision(), d.scale) + value.changePrecision(precision, scale) + if (precision <= Decimal.MAX_INT_DIGITS) { + toColumn.appendInt(value.toUnscaledLong.toInt) + } else if (precision <= Decimal.MAX_LONG_DIGITS) { + toColumn.appendLong(value.toUnscaledLong) + } else { + val bytes = value.toJavaBigDecimal.unscaledValue.toByteArray + toColumn.appendByteArray(bytes, 0, bytes.length) + } + + case dt => + throw new UnsupportedOperationException(s"Unsupported Data Type: $dt") + } + } + } + } + } + i += 1 + } + true + } +} + +/** + * Constants for OrcColumnarBatchReader. + */ +object OrcColumnarBatchReader { + /** + * Default memory mode for ColumnarBatch. + */ + val DEFAULT_MEMORY_MODE = MemoryMode.ON_HEAP + + /** + * The default size of batch. We use this value for both ORC and Spark consistently + * because they have different default values like the following. + * + * - ORC's VectorizedRowBatch.DEFAULT_SIZE = 1024 + * - Spark's ColumnarBatch.DEFAULT_BATCH_SIZE = 4 * 1024 + */ + val DEFAULT_SIZE: Int = 4 * 1024 +} + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala index f7471cd7debce..582ca4e048357 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala @@ -118,6 +118,13 @@ class OrcFileFormat } } + override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = { + val conf = sparkSession.sessionState.conf + conf.orcVectorizedReaderEnabled && conf.wholeStageEnabled && + schema.length <= conf.wholeStageMaxNumFields && + schema.forall(_.dataType.isInstanceOf[AtomicType]) + } + override def isSplitable( sparkSession: SparkSession, options: Map[String, String], @@ -139,6 +146,10 @@ class OrcFileFormat } } + val resultSchema = StructType(requiredSchema.fields ++ partitionSchema.fields) + val enableVectorizedReader = sparkSession.sessionState.conf.orcVectorizedReaderEnabled && + supportBatch(sparkSession, resultSchema) + val broadcastedConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) val isCaseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis @@ -146,8 +157,14 @@ class OrcFileFormat (file: PartitionedFile) => { val conf = broadcastedConf.value.value + val filePath = new Path(new URI(file.filePath)) + + val fs = filePath.getFileSystem(conf) + val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) + val reader = OrcFile.createReader(filePath, readerOptions) + val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( - isCaseSensitive, dataSchema, requiredSchema, new Path(new URI(file.filePath)), conf) + isCaseSensitive, dataSchema, requiredSchema, reader, conf) if (requestedColIdsOrEmptyFile.isEmpty) { Iterator.empty @@ -158,26 +175,40 @@ class OrcFileFormat conf.set(OrcConf.INCLUDE_COLUMNS.getAttribute, requestedColIds.filter(_ != -1).sorted.mkString(",")) - val fileSplit = - new FileSplit(new Path(new URI(file.filePath)), file.start, file.length, Array.empty) + val fileSplit = new FileSplit(filePath, file.start, file.length, Array.empty) val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) val taskAttemptContext = new TaskAttemptContextImpl(conf, attemptId) - val orcRecordReader = new OrcInputFormat[OrcStruct] - .createRecordReader(fileSplit, taskAttemptContext) - val iter = new RecordReaderIterator[OrcStruct](orcRecordReader) - Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => iter.close())) - - val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes - val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) - val deserializer = new OrcDeserializer(dataSchema, requiredSchema, requestedColIds) - - if (partitionSchema.length == 0) { - iter.map(value => unsafeProjection(deserializer.deserialize(value))) + if (enableVectorizedReader) { + val batchReader = new OrcColumnarBatchReader + batchReader.initialize(fileSplit, taskAttemptContext) + batchReader.setRequiredSchema( + OrcUtils.getFixedTypeDescription(reader.getSchema, dataSchema), + requestedColIds, + resultSchema, + requiredSchema, + file.partitionValues) + + val iter = new RecordReaderIterator(batchReader) + Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => iter.close())) + iter.asInstanceOf[Iterator[InternalRow]] } else { - val joinedRow = new JoinedRow() - iter.map(value => - unsafeProjection(joinedRow(deserializer.deserialize(value), file.partitionValues))) + val orcRecordReader = new OrcInputFormat[OrcStruct] + .createRecordReader(fileSplit, taskAttemptContext) + val iter = new RecordReaderIterator[OrcStruct](orcRecordReader) + Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => iter.close())) + + val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes + val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) + val deserializer = new OrcDeserializer(dataSchema, requiredSchema, requestedColIds) + + if (partitionSchema.length == 0) { + iter.map(value => unsafeProjection(deserializer.deserialize(value))) + } else { + val joinedRow = new JoinedRow() + iter.map(value => + unsafeProjection(joinedRow(deserializer.deserialize(value), file.partitionValues))) + } } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala index b03ee06d04a16..be04ed1aaaa7f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala @@ -21,7 +21,7 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} -import org.apache.orc.{OrcFile, TypeDescription} +import org.apache.orc.{OrcFile, Reader, TypeDescription} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging @@ -80,11 +80,8 @@ object OrcUtils extends Logging { isCaseSensitive: Boolean, dataSchema: StructType, requiredSchema: StructType, - file: Path, + reader: Reader, conf: Configuration): Option[Array[Int]] = { - val fs = file.getFileSystem(conf) - val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) - val reader = OrcFile.createReader(file, readerOptions) val orcFieldNames = reader.getSchema.getFieldNames.asScala if (orcFieldNames.isEmpty) { // SPARK-8501: Some old empty ORC files always have an empty schema stored in their footer. @@ -110,4 +107,21 @@ object OrcUtils extends Logging { } } } + + /** + * Return a fixed ORC schema with data schema information, if needed. + */ + def getFixedTypeDescription( + schema: TypeDescription, + dataSchema: StructType): TypeDescription = { + if (schema.getFieldNames.asScala.forall(_.startsWith("_col"))) { + var schemaString = schema.toString + dataSchema.zipWithIndex.foreach { case (field: StructField, index: Int) => + schemaString = schemaString.replace(s"_col$index:", s"${field.name}:") + } + TypeDescription.fromString(schemaString) + } else { + schema + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala new file mode 100644 index 0000000000000..59cfcb38f58b8 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala @@ -0,0 +1,357 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.orc + +import java.io.File + +import scala.util.Random + +import org.apache.spark.SparkConf +import org.apache.spark.sql.{DataFrame, SparkSession} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.util.{Benchmark, Utils} + + +/** + * Benchmark to measure ORC read performance. + * + * This is in `sql/hive` module in order to compare `sql/core` and `sql/hive` ORC data sources. + */ +// scalastyle:off line.size.limit +object OrcReadBenchmark { + val conf = new SparkConf() + conf.set("orc.compression", "snappy") + + private val spark = SparkSession.builder() + .master("local[1]") + .appName("OrcReadBenchmark") + .config(conf) + .getOrCreate() + + // Set default configs. Individual cases will change them if necessary. + spark.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true") + + def withTempPath(f: File => Unit): Unit = { + val path = Utils.createTempDir() + path.delete() + try f(path) finally Utils.deleteRecursively(path) + } + + def withTempTable(tableNames: String*)(f: => Unit): Unit = { + try f finally tableNames.foreach(spark.catalog.dropTempView) + } + + private val NATIVE_ORC_FORMAT = "org.apache.spark.sql.execution.datasources.orc.OrcFileFormat" + private val HIVE_ORC_FORMAT = "org.apache.spark.sql.hive.orc.OrcFileFormat" + + private def prepareTable(dir: File, df: DataFrame, partition: Option[String] = None): Unit = { + val dirORC = dir.getCanonicalPath + + if (partition.isDefined) { + df.write.partitionBy(partition.get).orc(dirORC) + } else { + df.write.orc(dirORC) + } + + spark.read.format(NATIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("nativeOrcTable") + spark.read.format(HIVE_ORC_FORMAT).load(dirORC).createOrReplaceTempView("hiveOrcTable") + } + + def numericScanBenchmark(values: Int, dataType: DataType): Unit = { + val sqlBenchmark = new Benchmark(s"SQL Single ${dataType.sql} Column Scan", values) + + withTempPath { dir => + withTempTable("t1", "nativeOrcTable", "hiveOrcTable") { + import spark.implicits._ + spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1") + + prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1")) + + sqlBenchmark.addCase("Native ORC") { _ => + spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() + } + + sqlBenchmark.addCase("Hive built-in ORC") { _ => + spark.sql("SELECT sum(id) FROM hiveOrcTable").collect() + } + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 + Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz + + SQL Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Native ORC 132 / 138 119.4 8.4 1.0X + Hive built-in ORC 1328 / 1333 11.8 84.5 0.1X + + SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Native ORC 178 / 188 88.2 11.3 1.0X + Hive built-in ORC 1541 / 1560 10.2 98.0 0.1X + + SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Native ORC 242 / 256 64.9 15.4 1.0X + Hive built-in ORC 1650 / 1676 9.5 104.9 0.1X + + SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Native ORC 297 / 309 53.0 18.9 1.0X + Hive built-in ORC 1750 / 1766 9.0 111.3 0.2X + + SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Native ORC 352 / 363 44.7 22.4 1.0X + Hive built-in ORC 1749 / 1764 9.0 111.2 0.2X + + SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Native ORC 436 / 456 36.1 27.7 1.0X + Hive built-in ORC 1852 / 1860 8.5 117.8 0.2X + */ + sqlBenchmark.run() + } + } + } + + def intStringScanBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Int and String Scan", values) + + withTempPath { dir => + withTempTable("t1", "nativeOrcTable", "hiveOrcTable") { + import spark.implicits._ + spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1") + + prepareTable( + dir, + spark.sql("SELECT CAST(value AS INT) AS c1, CAST(value as STRING) AS c2 FROM t1")) + + benchmark.addCase("Native ORC") { _ => + spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect() + } + + benchmark.addCase("Hive built-in ORC") { _ => + spark.sql("SELECT sum(c1), sum(length(c2)) FROM hiveOrcTable").collect() + } + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 + Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz + + Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Native ORC 1374 / 1376 7.6 131.0 1.0X + Hive built-in ORC 3653 / 3664 2.9 348.4 0.4X + */ + benchmark.run() + } + } + } + + def partitionTableScanBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Partitioned Table", values) + + withTempPath { dir => + withTempTable("t1", "nativeOrcTable", "hiveOrcTable") { + import spark.implicits._ + spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1") + + prepareTable(dir, spark.sql("SELECT value % 2 AS p, value AS id FROM t1"), Some("p")) + + benchmark.addCase("Read data column - Native ORC") { _ => + spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() + } + + benchmark.addCase("Read data column - Hive built-in ORC") { _ => + spark.sql("SELECT sum(id) FROM hiveOrcTable").collect() + } + + benchmark.addCase("Read partition column - Native ORC") { _ => + spark.sql("SELECT sum(p) FROM nativeOrcTable").collect() + } + + benchmark.addCase("Read partition column - Hive built-in ORC") { _ => + spark.sql("SELECT sum(p) FROM hiveOrcTable").collect() + } + + benchmark.addCase("Read both columns - Native ORC") { _ => + spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect() + } + + benchmark.addCase("Read both columns - Hive built-in ORC") { _ => + spark.sql("SELECT sum(p), sum(id) FROM hiveOrcTable").collect() + } + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 + Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz + + Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Read data column - Native ORC 321 / 327 49.0 20.4 1.0X + Read data column - Hive built-in ORC 2041 / 2176 7.7 129.8 0.2X + Read partition column - Native ORC 53 / 57 298.2 3.4 6.1X + Read partition column - Hive built-in ORC 1176 / 1183 13.4 74.7 0.3X + Read both columns - Native ORC 335 / 340 47.0 21.3 1.0X + Read both columns - Hive built-in ORC 1970 / 1974 8.0 125.2 0.2X + */ + benchmark.run() + } + } + } + + def stringDictionaryScanBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("String Dictionary", values) + + withTempPath { dir => + withTempTable("t1", "nativeOrcTable", "hiveOrcTable") { + spark.range(values).createOrReplaceTempView("t1") + + prepareTable(dir, spark.sql("SELECT CAST((id % 200) + 10000 as STRING) AS c1 FROM t1")) + + benchmark.addCase("Native ORC") { _ => + spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect() + } + + benchmark.addCase("Hive built-in ORC") { _ => + spark.sql("SELECT sum(length(c1)) FROM hiveOrcTable").collect() + } + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 + Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz + + String Dictionary: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Native ORC 363 / 382 28.9 34.7 1.0X + Hive built-in ORC 2012 / 2080 5.2 191.9 0.2X + */ + benchmark.run() + } + } + } + + def stringWithNullsScanBenchmark(values: Int, fractionOfNulls: Double): Unit = { + withTempPath { dir => + withTempTable("t1", "nativeOrcTable", "hiveOrcTable") { + spark.range(values).createOrReplaceTempView("t1") + + prepareTable( + dir, + spark.sql( + s"SELECT IF(RAND(1) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c1, " + + s"IF(RAND(2) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c2 FROM t1")) + + val benchmark = new Benchmark("String with Nulls Scan", values) + + benchmark.addCase(s"Native ORC ($fractionOfNulls%)") { iter => + spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " + + "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() + } + + benchmark.addCase(s"Hive built-in ORC ($fractionOfNulls%)") { iter => + spark.sql("SELECT SUM(LENGTH(c2)) FROM hiveOrcTable " + + "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() + } + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 + Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz + + String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Native ORC (0.0%) 1120 / 1142 9.4 106.8 1.0X + Hive built-in ORC (0.0%) 4232 / 4284 2.5 403.6 0.3X + + String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Native ORC (0.5%) 1474 / 1512 7.1 140.5 1.0X + Hive built-in ORC (0.5%) 3114 / 3140 3.4 297.0 0.5X + + String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Native ORC (0.95%) 568 / 589 18.5 54.1 1.0X + Hive built-in ORC (0.95%) 1548 / 1549 6.8 147.6 0.4X + */ + benchmark.run() + } + } + } + + def columnsBenchmark(values: Int, width: Int): Unit = { + val sqlBenchmark = new Benchmark(s"SQL Single Column Scan FROM $width-Column Rows", values) + + withTempPath { dir => + withTempTable("t1", "nativeOrcTable", "hiveOrcTable") { + import spark.implicits._ + val middle = width / 2 + val selectExpr = (1 to width).map(i => s"value as c$i") + spark.range(values).map(_ => Random.nextLong).toDF() + .selectExpr(selectExpr: _*).createOrReplaceTempView("t1") + + prepareTable(dir, spark.sql(s"SELECT * FROM t1")) + + sqlBenchmark.addCase("Native ORC") { _ => + spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect() + } + + sqlBenchmark.addCase("Hive built-in ORC") { _ => + spark.sql(s"SELECT sum(c$middle) FROM hiveOrcTable").collect() + } + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 + Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz + + SQL Single Column Scan FROM 100-Column Rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Native ORC 101 / 109 10.4 95.9 1.0X + Hive built-in ORC 372 / 387 2.8 355.1 0.3X + + SQL Single Column Scan FROM 200-Column Rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Native ORC 187 / 203 5.6 178.7 1.0X + Hive built-in ORC 635 / 648 1.7 605.4 0.3X + + SQL Single Column Scan FROM 300-Column Rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Native ORC 357 / 369 2.9 340.6 1.0X + Hive built-in ORC 960 / 967 1.1 915.3 0.4X + */ + sqlBenchmark.run() + } + } + } + + def main(args: Array[String]): Unit = { + Seq(ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType).foreach { dataType => + numericScanBenchmark(1024 * 1024 * 15, dataType) + } + intStringScanBenchmark(1024 * 1024 * 10) + partitionTableScanBenchmark(1024 * 1024 * 15) + stringDictionaryScanBenchmark(1024 * 1024 * 10) + for (fractionOfNulls <- List(0.0, 0.50, 0.95)) { + stringWithNullsScanBenchmark(1024 * 1024 * 10, fractionOfNulls) + } + columnsBenchmark(1024 * 1024 * 1, 100) + columnsBenchmark(1024 * 1024 * 1, 200) + columnsBenchmark(1024 * 1024 * 1, 300) + } +} +// scalastyle:on line.size.limit From 42048cc9f3e757de2864db2b23f3bbadf9f5f729 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 12 Dec 2017 08:52:40 -0800 Subject: [PATCH 02/23] Address comments --- .../apache/spark/sql/internal/SQLConf.scala | 2 +- .../orc/OrcColumnarBatchReader.scala | 95 ++++++++----------- 2 files changed, 43 insertions(+), 54 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 955cddedda4c0..74949db883f7a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -386,7 +386,7 @@ object SQLConf { .checkValues(Set("hive", "native")) .createWithDefault("native") - val ORC_VECTORIZED_READER_ENABLED = buildConf("spark.sql.orc.vectorizedReader.enabled") + val ORC_VECTORIZED_READER_ENABLED = buildConf("spark.sql.orc.enableVectorizedReader") .doc("Enables vectorized orc decoding.") .booleanConf .createWithDefault(true) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala index e99aef2800d85..009e7e44c37c6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala @@ -22,6 +22,7 @@ import org.apache.hadoop.mapreduce.lib.input.FileSplit import org.apache.orc._ import org.apache.orc.mapred.OrcInputFormat import org.apache.orc.storage.ql.exec.vector._ +import org.apache.orc.storage.serde2.io.HiveDecimalWritable import org.apache.spark.internal.Logging import org.apache.spark.memory.MemoryMode @@ -34,6 +35,8 @@ import org.apache.spark.sql.types._ * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch. */ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging { + import OrcColumnarBatchReader._ + /** * ORC File Reader. */ @@ -125,14 +128,14 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat resultSchema: StructType, requiredSchema: StructType, partitionValues: InternalRow): Unit = { - batch = orcSchema.createRowBatch(OrcColumnarBatchReader.DEFAULT_SIZE) + batch = orcSchema.createRowBatch(DEFAULT_SIZE) totalRowCount = reader.getNumberOfRows logDebug(s"totalRowCount = $totalRowCount") this.requiredSchema = requiredSchema this.requestedColIds = requestedColIds - val memMode = OrcColumnarBatchReader.DEFAULT_MEMORY_MODE + val memMode = DEFAULT_MEMORY_MODE val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE if (memMode == MemoryMode.OFF_HEAP) { columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema) @@ -206,7 +209,7 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat case TimestampType => val data = fromColumn.asInstanceOf[TimestampColumnVector] - toColumn.appendLongs(batchSize, data.time(0) * 1000L + data.nanos(0) / 1000L) + toColumn.appendLongs(batchSize, fromTimestampColumnVector(data, 0)) case FloatType => val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0).toFloat @@ -232,20 +235,7 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat case DecimalType.Fixed(precision, scale) => val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(0) - val value = Decimal(d.getHiveDecimal.bigDecimalValue, d.precision(), d.scale) - value.changePrecision(precision, scale) - if (precision <= Decimal.MAX_INT_DIGITS) { - toColumn.appendInts(batchSize, value.toUnscaledLong.toInt) - } else if (precision <= Decimal.MAX_LONG_DIGITS) { - toColumn.appendLongs(batchSize, value.toUnscaledLong) - } else { - val bytes = value.toJavaBigDecimal.unscaledValue.toByteArray - var index = 0 - while (index < batchSize) { - toColumn.appendByteArray(bytes, 0, bytes.length) - index += 1 - } - } + appendDecimalWritable(toColumn, precision, scale, d) case dt => throw new UnsupportedOperationException(s"Unsupported Data Type: $dt") @@ -275,7 +265,7 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat toColumn.appendShort(data(index).toShort) index += 1 } - case IntegerType => + case IntegerType | DateType => val data = fromColumn.asInstanceOf[LongColumnVector].vector var index = 0 while (index < batchSize) { @@ -286,19 +276,11 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat val data = fromColumn.asInstanceOf[LongColumnVector].vector toColumn.appendLongs(batchSize, data, 0) - case DateType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector - var index = 0 - while (index < batchSize) { - toColumn.appendInt(data(index).toInt) - index += 1 - } - case TimestampType => val data = fromColumn.asInstanceOf[TimestampColumnVector] var index = 0 while (index < batchSize) { - toColumn.appendLong(data.time(index) * 1000L + data.nanos(index) / 1000L) + toColumn.appendLong(fromTimestampColumnVector(data, index)) index += 1 } @@ -334,16 +316,7 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat var index = 0 while (index < batchSize) { val d = data.vector(index) - val value = Decimal(d.getHiveDecimal.bigDecimalValue, d.precision(), d.scale) - value.changePrecision(precision, scale) - if (precision <= Decimal.MAX_INT_DIGITS) { - toColumn.appendInt(value.toUnscaledLong.toInt) - } else if (precision <= Decimal.MAX_LONG_DIGITS) { - toColumn.appendLong(value.toUnscaledLong) - } else { - val bytes = value.toJavaBigDecimal.unscaledValue.toByteArray - toColumn.appendByteArray(bytes, 0, bytes.length) - } + appendDecimalWritable(toColumn, precision, scale, d) index += 1 } @@ -359,26 +332,23 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat case BooleanType => val data = fromColumn.asInstanceOf[LongColumnVector].vector(index) == 1 toColumn.appendBoolean(data) + case ByteType => val data = fromColumn.asInstanceOf[LongColumnVector].vector(index).toByte toColumn.appendByte(data) case ShortType => val data = fromColumn.asInstanceOf[LongColumnVector].vector(index).toShort toColumn.appendShort(data) - case IntegerType => + case IntegerType | DateType => val data = fromColumn.asInstanceOf[LongColumnVector].vector(index).toInt toColumn.appendInt(data) case LongType => val data = fromColumn.asInstanceOf[LongColumnVector].vector(index) toColumn.appendLong(data) - case DateType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector(index).toInt - toColumn.appendInt(data) - case TimestampType => val data = fromColumn.asInstanceOf[TimestampColumnVector] - toColumn.appendLong(data.time(index) * 1000L + data.nanos(index) / 1000L) + toColumn.appendLong(fromTimestampColumnVector(data, index)) case FloatType => val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(index).toFloat @@ -397,16 +367,7 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat case DecimalType.Fixed(precision, scale) => val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(index) - val value = Decimal(d.getHiveDecimal.bigDecimalValue, d.precision(), d.scale) - value.changePrecision(precision, scale) - if (precision <= Decimal.MAX_INT_DIGITS) { - toColumn.appendInt(value.toUnscaledLong.toInt) - } else if (precision <= Decimal.MAX_LONG_DIGITS) { - toColumn.appendLong(value.toUnscaledLong) - } else { - val bytes = value.toJavaBigDecimal.unscaledValue.toByteArray - toColumn.appendByteArray(bytes, 0, bytes.length) - } + appendDecimalWritable(toColumn, precision, scale, d) case dt => throw new UnsupportedOperationException(s"Unsupported Data Type: $dt") @@ -438,5 +399,33 @@ object OrcColumnarBatchReader { * - Spark's ColumnarBatch.DEFAULT_BATCH_SIZE = 4 * 1024 */ val DEFAULT_SIZE: Int = 4 * 1024 + + /** + * Returns the number of micros since epoch from an element of TimestampColumnVector. + */ + private def fromTimestampColumnVector(vector: TimestampColumnVector, index: Int): Long = + vector.time(index) * 1000L + vector.nanos(index) / 1000L + + /** + * Append a decimalWritable to a writableColumnVector. + */ + private def appendDecimalWritable( + toColumn: WritableColumnVector, + precision: Int, + scale: Int, + decimalWritable: HiveDecimalWritable) = { + val decimal = decimalWritable.getHiveDecimal() + val value = Decimal(decimal.bigDecimalValue, decimal.precision(), decimal.scale()) + value.changePrecision(precision, scale) + + if (precision <= Decimal.MAX_INT_DIGITS) { + toColumn.appendInt(value.toUnscaledLong.toInt) + } else if (precision <= Decimal.MAX_LONG_DIGITS) { + toColumn.appendLong(value.toUnscaledLong) + } else { + val bytes = value.toJavaBigDecimal.unscaledValue.toByteArray + toColumn.appendByteArray(bytes, 0, bytes.length) + } + } } From c0b9aeabf6f459a46161ed2ce78fb92a4f1951b1 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 12 Dec 2017 11:41:15 -0800 Subject: [PATCH 03/23] Address more comments. --- .../sql/execution/datasources/orc/OrcColumnarBatchReader.scala | 3 ++- .../apache/spark/sql/execution/datasources/orc/OrcUtils.scala | 1 + 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala index 009e7e44c37c6..aec79bcec5f75 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala @@ -129,6 +129,7 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat requiredSchema: StructType, partitionValues: InternalRow): Unit = { batch = orcSchema.createRowBatch(DEFAULT_SIZE) + assert(!batch.selectedInUse) totalRowCount = reader.getNumberOfRows logDebug(s"totalRowCount = $totalRowCount") @@ -413,7 +414,7 @@ object OrcColumnarBatchReader { toColumn: WritableColumnVector, precision: Int, scale: Int, - decimalWritable: HiveDecimalWritable) = { + decimalWritable: HiveDecimalWritable): Unit = { val decimal = decimalWritable.getHiveDecimal() val value = Decimal(decimal.bigDecimalValue, decimal.precision(), decimal.scale()) value.changePrecision(precision, scale) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala index be04ed1aaaa7f..daa3ac588dc28 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala @@ -110,6 +110,7 @@ object OrcUtils extends Logging { /** * Return a fixed ORC schema with data schema information, if needed. + * The schema inside old ORC files might consist of invalid column names like '_col0'. */ def getFixedTypeDescription( schema: TypeDescription, From a65ea04d3d57d6c9e90c0f215b1ce8f444c27310 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 27 Dec 2017 20:50:07 -0800 Subject: [PATCH 04/23] Address comments. --- .../orc/OrcColumnarBatchReader.scala | 54 ++++++------------- .../spark/sql/hive/orc/OrcReadBenchmark.scala | 2 +- 2 files changed, 16 insertions(+), 40 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala index aec79bcec5f75..bb727a870e456 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.types._ /** * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch. + * After creating, `initialize` and `setRequiredSchema` should be called sequentially. */ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging { import OrcColumnarBatchReader._ @@ -55,7 +56,7 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat /** * Record reader from row batch. */ - private var rows: org.apache.orc.RecordReader = _ + private var recordReader: org.apache.orc.RecordReader = _ /** * Required Schema. @@ -95,9 +96,9 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat columnarBatch.close() columnarBatch = null } - if (rows != null) { - rows.close() - rows = null + if (recordReader != null) { + recordReader.close() + recordReader = null } } @@ -115,7 +116,7 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat .filesystem(fileSplit.getPath.getFileSystem(conf))) val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength) - rows = reader.rows(options) + recordReader = reader.rows(options) } /** @@ -136,9 +137,8 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat this.requiredSchema = requiredSchema this.requestedColIds = requestedColIds - val memMode = DEFAULT_MEMORY_MODE val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE - if (memMode == MemoryMode.OFF_HEAP) { + if (DEFAULT_MEMORY_MODE == MemoryMode.OFF_HEAP) { columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema) } else { columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema) @@ -163,7 +163,7 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat return false } - rows.nextBatch(batch) + recordReader.nextBatch(batch) val batchSize = batch.size if (batchSize == 0) { return false @@ -197,17 +197,13 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat case ShortType => val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort toColumn.appendShorts(batchSize, data) - case IntegerType => + case IntegerType | DateType => val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt toColumn.appendInts(batchSize, data) case LongType => val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) toColumn.appendLongs(batchSize, data) - case DateType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt - toColumn.appendInts(batchSize, data) - case TimestampType => val data = fromColumn.asInstanceOf[TimestampColumnVector] toColumn.appendLongs(batchSize, fromTimestampColumnVector(data, 0)) @@ -219,14 +215,7 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0) toColumn.appendDoubles(batchSize, data) - case StringType => - val data = fromColumn.asInstanceOf[BytesColumnVector] - var index = 0 - while (index < batchSize) { - toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0)) - index += 1 - } - case BinaryType => + case StringType | BinaryType => val data = fromColumn.asInstanceOf[BytesColumnVector] var index = 0 while (index < batchSize) { @@ -242,7 +231,7 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat throw new UnsupportedOperationException(s"Unsupported Data Type: $dt") } } - } else if (!field.nullable || fromColumn.noNulls) { + } else if (fromColumn.noNulls) { field.dataType match { case BooleanType => val data = fromColumn.asInstanceOf[LongColumnVector].vector @@ -296,15 +285,7 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat val data = fromColumn.asInstanceOf[DoubleColumnVector].vector toColumn.appendDoubles(batchSize, data, 0) - case StringType => - val data = fromColumn.asInstanceOf[BytesColumnVector] - var index = 0 - while (index < batchSize) { - toColumn.appendByteArray( - data.vector(index), data.start(index), data.length(index)) - index += 1 - } - case BinaryType => + case StringType | BinaryType => val data = fromColumn.asInstanceOf[BytesColumnVector] var index = 0 while (index < batchSize) { @@ -316,8 +297,7 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat val data = fromColumn.asInstanceOf[DecimalColumnVector] var index = 0 while (index < batchSize) { - val d = data.vector(index) - appendDecimalWritable(toColumn, precision, scale, d) + appendDecimalWritable(toColumn, precision, scale, data.vector(index)) index += 1 } @@ -358,11 +338,7 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(index) toColumn.appendDouble(data) - case StringType => - val v = fromColumn.asInstanceOf[BytesColumnVector] - toColumn.appendByteArray(v.vector(index), v.start(index), v.length(index)) - - case BinaryType => + case StringType | BinaryType => val v = fromColumn.asInstanceOf[BytesColumnVector] toColumn.appendByteArray(v.vector(index), v.start(index), v.length(index)) @@ -408,7 +384,7 @@ object OrcColumnarBatchReader { vector.time(index) * 1000L + vector.nanos(index) / 1000L /** - * Append a decimalWritable to a writableColumnVector. + * Append a `HiveDecimalWritable` to a `WritableColumnVector`. */ private def appendDecimalWritable( toColumn: WritableColumnVector, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala index 59cfcb38f58b8..248c6ccafa488 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala @@ -305,7 +305,7 @@ object OrcReadBenchmark { spark.range(values).map(_ => Random.nextLong).toDF() .selectExpr(selectExpr: _*).createOrReplaceTempView("t1") - prepareTable(dir, spark.sql(s"SELECT * FROM t1")) + prepareTable(dir, spark.sql("SELECT * FROM t1")) sqlBenchmark.addCase("Native ORC") { _ => spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect() From 56087ac566c91df256614d5db6550db8169cbeb9 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 28 Dec 2017 00:37:23 -0800 Subject: [PATCH 05/23] Revert the change on DataSourceScanExec. --- .../org/apache/spark/sql/execution/DataSourceScanExec.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 664edf05dbdda..d1ff82c7c06bc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -31,7 +31,6 @@ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning} import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat => ParquetSource} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.sources.{BaseRelation, Filter} @@ -170,8 +169,6 @@ case class FileSourceScanExec( val needsUnsafeRowConversion: Boolean = if (relation.fileFormat.isInstanceOf[ParquetSource]) { SparkSession.getActiveSession.get.sessionState.conf.parquetVectorizedReaderEnabled - } else if (relation.fileFormat.isInstanceOf[OrcFileFormat]) { - SparkSession.getActiveSession.get.sessionState.conf.orcVectorizedReaderEnabled } else { false } From e52fd276e82305ce779976757ed73f60618b592a Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 2 Jan 2018 14:41:10 -0800 Subject: [PATCH 06/23] Rebased to the master and address some comments. --- .../orc/OrcColumnarBatchReader.scala | 195 +++++++++++++----- .../execution/datasources/orc/OrcUtils.scala | 3 +- .../spark/sql/hive/orc/OrcReadBenchmark.scala | 174 +++++++++++----- 3 files changed, 268 insertions(+), 104 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala index bb727a870e456..46a6b3642353b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala @@ -24,18 +24,18 @@ import org.apache.orc.mapred.OrcInputFormat import org.apache.orc.storage.ql.exec.vector._ import org.apache.orc.storage.serde2.io.HiveDecimalWritable -import org.apache.spark.internal.Logging import org.apache.spark.memory.MemoryMode import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.vectorized._ import org.apache.spark.sql.types._ +import org.apache.spark.sql.vectorized._ /** * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch. * After creating, `initialize` and `setRequiredSchema` should be called sequentially. */ -private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging { +private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] { import OrcColumnarBatchReader._ /** @@ -69,7 +69,7 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat private var columnarBatch: ColumnarBatch = _ /** - * Writable columnVectors of ColumnarBatch. + * Writable column vectors of ColumnarBatch. */ private var columnVectors: Seq[WritableColumnVector] = _ @@ -117,6 +117,7 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat val options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart, fileSplit.getLength) recordReader = reader.rows(options) + totalRowCount = reader.getNumberOfRows } /** @@ -130,14 +131,12 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat requiredSchema: StructType, partitionValues: InternalRow): Unit = { batch = orcSchema.createRowBatch(DEFAULT_SIZE) - assert(!batch.selectedInUse) - totalRowCount = reader.getNumberOfRows - logDebug(s"totalRowCount = $totalRowCount") + assert(!batch.selectedInUse, "No filters are allowed") this.requiredSchema = requiredSchema this.requestedColIds = requestedColIds - val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE + val capacity = DEFAULT_SIZE if (DEFAULT_MEMORY_MODE == MemoryMode.OFF_HEAP) { columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema) } else { @@ -152,6 +151,14 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat columnVectors(i + partitionIdx).setIsConstant() } } + + // Initialize the missing columns once. + for (i <- 0 until requiredSchema.length) { + if (requestedColIds(i) < 0) { + columnVectors(i).putNulls(0, columnarBatch.capacity) + columnVectors(i).setIsConstant() + } + } } /** @@ -164,12 +171,13 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat } recordReader.nextBatch(batch) + assert(!batch.selectedInUse, "No filters are allowed") val batchSize = batch.size if (batchSize == 0) { return false } rowsReturned += batchSize - columnarBatch.reset() + columnVectors.foreach(_.reset) columnarBatch.setNumRows(batchSize) var i = 0 @@ -177,9 +185,7 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat val field = requiredSchema(i) val toColumn = columnVectors(i) - if (requestedColIds(i) < 0) { - toColumn.appendNulls(batchSize) - } else { + if (requestedColIds(i) >= 0) { val fromColumn = batch.cols(requestedColIds(i)) if (fromColumn.isRepeating) { @@ -305,51 +311,130 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat throw new UnsupportedOperationException(s"Unsupported Data Type: $dt") } } else { - for (index <- 0 until batchSize) { - if (fromColumn.isNull(index)) { - toColumn.appendNull() - } else { - field.dataType match { - case BooleanType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector(index) == 1 - toColumn.appendBoolean(data) - - case ByteType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector(index).toByte - toColumn.appendByte(data) - case ShortType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector(index).toShort - toColumn.appendShort(data) - case IntegerType | DateType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector(index).toInt - toColumn.appendInt(data) - case LongType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector(index) - toColumn.appendLong(data) - - case TimestampType => - val data = fromColumn.asInstanceOf[TimestampColumnVector] - toColumn.appendLong(fromTimestampColumnVector(data, index)) - - case FloatType => - val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(index).toFloat - toColumn.appendFloat(data) - case DoubleType => - val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(index) - toColumn.appendDouble(data) - - case StringType | BinaryType => - val v = fromColumn.asInstanceOf[BytesColumnVector] - toColumn.appendByteArray(v.vector(index), v.start(index), v.length(index)) - - case DecimalType.Fixed(precision, scale) => - val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(index) - appendDecimalWritable(toColumn, precision, scale, d) - - case dt => - throw new UnsupportedOperationException(s"Unsupported Data Type: $dt") + field.dataType match { + case BooleanType => + val vector = fromColumn.asInstanceOf[LongColumnVector].vector + var index = 0 + while (index < batchSize) { + if (fromColumn.isNull(index)) { + toColumn.appendNull() + } else { + toColumn.appendBoolean(vector(index) == 1) + } + index += 1 } - } + + case ByteType => + val vector = fromColumn.asInstanceOf[LongColumnVector].vector + var index = 0 + while (index < batchSize) { + if (fromColumn.isNull(index)) { + toColumn.appendNull() + } else { + toColumn.appendByte(vector(index).toByte) + } + index += 1 + } + + case ShortType => + val vector = fromColumn.asInstanceOf[LongColumnVector].vector + var index = 0 + while (index < batchSize) { + if (fromColumn.isNull(index)) { + toColumn.appendNull() + } else { + toColumn.appendShort(vector(index).toShort) + } + index += 1 + } + + case IntegerType | DateType => + val vector = fromColumn.asInstanceOf[LongColumnVector].vector + var index = 0 + while (index < batchSize) { + if (fromColumn.isNull(index)) { + toColumn.appendNull() + } else { + toColumn.appendInt(vector(index).toInt) + } + index += 1 + } + + case LongType => + val vector = fromColumn.asInstanceOf[LongColumnVector].vector + var index = 0 + while (index < batchSize) { + if (fromColumn.isNull(index)) { + toColumn.appendNull() + } else { + toColumn.appendLong(vector(index)) + } + index += 1 + } + + case TimestampType => + val vector = fromColumn.asInstanceOf[TimestampColumnVector] + var index = 0 + while (index < batchSize) { + if (fromColumn.isNull(index)) { + toColumn.appendNull() + } else { + toColumn.appendLong(fromTimestampColumnVector(vector, index)) + } + index += 1 + } + + case FloatType => + val vector = fromColumn.asInstanceOf[DoubleColumnVector].vector + var index = 0 + while (index < batchSize) { + if (fromColumn.isNull(index)) { + toColumn.appendNull() + } else { + toColumn.appendFloat(vector(index).toFloat) + } + index += 1 + } + + case DoubleType => + val vector = fromColumn.asInstanceOf[DoubleColumnVector].vector + var index = 0 + while (index < batchSize) { + if (fromColumn.isNull(index)) { + toColumn.appendNull() + } else { + toColumn.appendDouble(vector(index)) + } + index += 1 + } + + case StringType | BinaryType => + val vector = fromColumn.asInstanceOf[BytesColumnVector] + var index = 0 + while (index < batchSize) { + if (fromColumn.isNull(index)) { + toColumn.appendNull() + } else { + toColumn.appendByteArray( + vector.vector(index), vector.start(index), vector.length(index)) + } + index += 1 + } + + case DecimalType.Fixed(precision, scale) => + val vector = fromColumn.asInstanceOf[DecimalColumnVector].vector + var index = 0 + while (index < batchSize) { + if (fromColumn.isNull(index)) { + toColumn.appendNull() + } else { + appendDecimalWritable(toColumn, precision, scale, vector(index)) + } + index += 1 + } + + case dt => + throw new UnsupportedOperationException(s"Unsupported Data Type: $dt") } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala index daa3ac588dc28..3eab55e1f3159 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala @@ -115,7 +115,8 @@ object OrcUtils extends Logging { def getFixedTypeDescription( schema: TypeDescription, dataSchema: StructType): TypeDescription = { - if (schema.getFieldNames.asScala.forall(_.startsWith("_col"))) { + val fieldNames = schema.getFieldNames.asScala + if (fieldNames.length == dataSchema.length && fieldNames.forall(_.startsWith("_col"))) { var schemaString = schema.toString dataSchema.zipWithIndex.foreach { case (field: StructField, index: Int) => schemaString = schemaString.replace(s"_col$index:", s"${field.name}:") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala index 248c6ccafa488..ad38658269733 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.orc import java.io.File -import scala.util.Random +import scala.util.{Random, Try} import org.apache.spark.SparkConf import org.apache.spark.sql.{DataFrame, SparkSession} @@ -57,6 +57,18 @@ object OrcReadBenchmark { try f finally tableNames.foreach(spark.catalog.dropTempView) } + def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { + val (keys, values) = pairs.unzip + val currentValues = keys.map(key => Try(spark.conf.get(key)).toOption) + (keys, values).zipped.foreach(spark.conf.set) + try f finally { + keys.zip(currentValues).foreach { + case (key, Some(value)) => spark.conf.set(key, value) + case (key, None) => spark.conf.unset(key) + } + } + } + private val NATIVE_ORC_FORMAT = "org.apache.spark.sql.execution.datasources.orc.OrcFileFormat" private val HIVE_ORC_FORMAT = "org.apache.spark.sql.hive.orc.OrcFileFormat" @@ -83,10 +95,16 @@ object OrcReadBenchmark { prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1")) - sqlBenchmark.addCase("Native ORC") { _ => + sqlBenchmark.addCase("Native ORC Vectorized") { _ => spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() } + sqlBenchmark.addCase("Native ORC MR") { _ => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { + spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() + } + } + sqlBenchmark.addCase("Hive built-in ORC") { _ => spark.sql("SELECT sum(id) FROM hiveOrcTable").collect() } @@ -97,33 +115,39 @@ object OrcReadBenchmark { SQL Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC 132 / 138 119.4 8.4 1.0X - Hive built-in ORC 1328 / 1333 11.8 84.5 0.1X + Native ORC Vectorized 156 / 163 100.7 9.9 1.0X + Native ORC MR 1222 / 1236 12.9 77.7 0.1X + Hive built-in ORC 1572 / 1625 10.0 100.0 0.1X SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC 178 / 188 88.2 11.3 1.0X - Hive built-in ORC 1541 / 1560 10.2 98.0 0.1X + Native ORC Vectorized 215 / 225 73.1 13.7 1.0X + Native ORC MR 1337 / 1458 11.8 85.0 0.2X + Hive built-in ORC 1696 / 1707 9.3 107.8 0.1X SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC 242 / 256 64.9 15.4 1.0X - Hive built-in ORC 1650 / 1676 9.5 104.9 0.1X + Native ORC Vectorized 285 / 294 55.2 18.1 1.0X + Native ORC MR 1397 / 1419 11.3 88.8 0.2X + Hive built-in ORC 2086 / 2143 7.5 132.6 0.1X SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC 297 / 309 53.0 18.9 1.0X - Hive built-in ORC 1750 / 1766 9.0 111.3 0.2X + Native ORC Vectorized 341 / 350 46.1 21.7 1.0X + Native ORC MR 1461 / 1492 10.8 92.9 0.2X + Hive built-in ORC 2002 / 2095 7.9 127.3 0.2X SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC 352 / 363 44.7 22.4 1.0X - Hive built-in ORC 1749 / 1764 9.0 111.2 0.2X + Native ORC Vectorized 347 / 356 45.3 22.1 1.0X + Native ORC MR 1524 / 1553 10.3 96.9 0.2X + Hive built-in ORC 2393 / 2404 6.6 152.1 0.1X SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC 436 / 456 36.1 27.7 1.0X - Hive built-in ORC 1852 / 1860 8.5 117.8 0.2X + Native ORC Vectorized 403 / 411 39.0 25.6 1.0X + Native ORC MR 1517 / 1529 10.4 96.5 0.3X + Hive built-in ORC 2054 / 2134 7.7 130.6 0.2X */ sqlBenchmark.run() } @@ -142,10 +166,16 @@ object OrcReadBenchmark { dir, spark.sql("SELECT CAST(value AS INT) AS c1, CAST(value as STRING) AS c2 FROM t1")) - benchmark.addCase("Native ORC") { _ => + benchmark.addCase("Native ORC Vectorized") { _ => spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect() } + benchmark.addCase("Native ORC MR") { _ => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { + spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect() + } + } + benchmark.addCase("Hive built-in ORC") { _ => spark.sql("SELECT sum(c1), sum(length(c2)) FROM hiveOrcTable").collect() } @@ -156,8 +186,9 @@ object OrcReadBenchmark { Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC 1374 / 1376 7.6 131.0 1.0X - Hive built-in ORC 3653 / 3664 2.9 348.4 0.4X + Native ORC Vectorized 1382 / 1400 7.6 131.8 1.0X + Native ORC MR 2689 / 2765 3.9 256.4 0.5X + Hive built-in ORC 3889 / 3894 2.7 370.9 0.4X */ benchmark.run() } @@ -174,26 +205,44 @@ object OrcReadBenchmark { prepareTable(dir, spark.sql("SELECT value % 2 AS p, value AS id FROM t1"), Some("p")) - benchmark.addCase("Read data column - Native ORC") { _ => + benchmark.addCase("Read data column - Native ORC Vectorized") { _ => spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() } + benchmark.addCase("Read data column - Native ORC MR") { _ => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { + spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() + } + } + benchmark.addCase("Read data column - Hive built-in ORC") { _ => spark.sql("SELECT sum(id) FROM hiveOrcTable").collect() } - benchmark.addCase("Read partition column - Native ORC") { _ => + benchmark.addCase("Read partition column - Native ORC Vectorized") { _ => spark.sql("SELECT sum(p) FROM nativeOrcTable").collect() } + benchmark.addCase("Read partition column - Native ORC MR") { _ => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { + spark.sql("SELECT sum(p) FROM nativeOrcTable").collect() + } + } + benchmark.addCase("Read partition column - Hive built-in ORC") { _ => spark.sql("SELECT sum(p) FROM hiveOrcTable").collect() } - benchmark.addCase("Read both columns - Native ORC") { _ => + benchmark.addCase("Read both columns - Native ORC Vectorized") { _ => spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect() } + benchmark.addCase("Read both columns - Native ORC MR") { _ => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { + spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect() + } + } + benchmark.addCase("Read both columns - Hive built-in ORC") { _ => spark.sql("SELECT sum(p), sum(id) FROM hiveOrcTable").collect() } @@ -204,12 +253,15 @@ object OrcReadBenchmark { Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Read data column - Native ORC 321 / 327 49.0 20.4 1.0X - Read data column - Hive built-in ORC 2041 / 2176 7.7 129.8 0.2X - Read partition column - Native ORC 53 / 57 298.2 3.4 6.1X - Read partition column - Hive built-in ORC 1176 / 1183 13.4 74.7 0.3X - Read both columns - Native ORC 335 / 340 47.0 21.3 1.0X - Read both columns - Hive built-in ORC 1970 / 1974 8.0 125.2 0.2X + Read data column - Native ORC Vectorized 347 / 350 45.3 22.1 1.0X + Read data column - Native ORC MR 1611 / 1613 9.8 102.4 0.2X + Read data column - Hive built-in ORC 2082 / 2091 7.6 132.4 0.2X + Read partition column - Native ORC Vectorized 55 / 57 286.5 3.5 6.3X + Read partition column - Native ORC MR 1062 / 1063 14.8 67.5 0.3X + Read partition column - Hive built-in ORC 1334 / 1334 11.8 84.8 0.3X + Read both columns - Native ORC Vectorized 380 / 388 41.3 24.2 0.9X + Read both columns - Native ORC MR 1654 / 1672 9.5 105.2 0.2X + Read both columns - Hive built-in ORC 2209 / 2209 7.1 140.5 0.2X */ benchmark.run() } @@ -225,10 +277,16 @@ object OrcReadBenchmark { prepareTable(dir, spark.sql("SELECT CAST((id % 200) + 10000 as STRING) AS c1 FROM t1")) - benchmark.addCase("Native ORC") { _ => + benchmark.addCase("Native ORC Vectorized") { _ => spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect() } + benchmark.addCase("Native ORC MR") { _ => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { + spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect() + } + } + benchmark.addCase("Hive built-in ORC") { _ => spark.sql("SELECT sum(length(c1)) FROM hiveOrcTable").collect() } @@ -239,8 +297,9 @@ object OrcReadBenchmark { String Dictionary: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC 363 / 382 28.9 34.7 1.0X - Hive built-in ORC 2012 / 2080 5.2 191.9 0.2X + Native ORC Vectorized 406 / 414 25.8 38.7 1.0X + Native ORC MR 1372 / 1381 7.6 130.8 0.3X + Hive built-in ORC 2016 / 2036 5.2 192.2 0.2X */ benchmark.run() } @@ -260,12 +319,19 @@ object OrcReadBenchmark { val benchmark = new Benchmark("String with Nulls Scan", values) - benchmark.addCase(s"Native ORC ($fractionOfNulls%)") { iter => + benchmark.addCase(s"Native ORC Vectorized ($fractionOfNulls%)") { _ => spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " + "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() } - benchmark.addCase(s"Hive built-in ORC ($fractionOfNulls%)") { iter => + benchmark.addCase(s"Native ORC MR ($fractionOfNulls%)") { _ => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { + spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " + + "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() + } + } + + benchmark.addCase(s"Hive built-in ORC ($fractionOfNulls%)") { _ => spark.sql("SELECT SUM(LENGTH(c2)) FROM hiveOrcTable " + "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() } @@ -276,18 +342,21 @@ object OrcReadBenchmark { String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC (0.0%) 1120 / 1142 9.4 106.8 1.0X - Hive built-in ORC (0.0%) 4232 / 4284 2.5 403.6 0.3X + Native ORC Vectorized (0.0%) 1122 / 1129 9.3 107.0 1.0X + Native ORC MR (0.0%) 2551 / 2619 4.1 243.3 0.4X + Hive built-in ORC (0.0%) 3938 / 3966 2.7 375.5 0.3X String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC (0.5%) 1474 / 1512 7.1 140.5 1.0X - Hive built-in ORC (0.5%) 3114 / 3140 3.4 297.0 0.5X + Native ORC Vectorized (0.5%) 1303 / 1314 8.0 124.2 1.0X + Native ORC MR (0.5%) 2470 / 2485 4.2 235.5 0.5X + Hive built-in ORC (0.5%) 2933 / 2950 3.6 279.7 0.4X String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC (0.95%) 568 / 589 18.5 54.1 1.0X - Hive built-in ORC (0.95%) 1548 / 1549 6.8 147.6 0.4X + Native ORC Vectorized (0.95%) 475 / 482 22.1 45.3 1.0X + Native ORC MR (0.95%) 1333 / 1348 7.9 127.2 0.4X + Hive built-in ORC (0.95%) 1695 / 1699 6.2 161.7 0.3X */ benchmark.run() } @@ -295,7 +364,7 @@ object OrcReadBenchmark { } def columnsBenchmark(values: Int, width: Int): Unit = { - val sqlBenchmark = new Benchmark(s"SQL Single Column Scan FROM $width-Column Rows", values) + val sqlBenchmark = new Benchmark(s"SQL Single Column Scan from wide table ($width columns)", values) withTempPath { dir => withTempTable("t1", "nativeOrcTable", "hiveOrcTable") { @@ -307,10 +376,16 @@ object OrcReadBenchmark { prepareTable(dir, spark.sql("SELECT * FROM t1")) - sqlBenchmark.addCase("Native ORC") { _ => + sqlBenchmark.addCase("Native ORC Vectorized") { _ => spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect() } + sqlBenchmark.addCase("Native ORC MR") { _ => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { + spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect() + } + } + sqlBenchmark.addCase("Hive built-in ORC") { _ => spark.sql(s"SELECT sum(c$middle) FROM hiveOrcTable").collect() } @@ -319,20 +394,23 @@ object OrcReadBenchmark { Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz - SQL Single Column Scan FROM 100-Column Rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + SQL Single Column Scan from wide table (100 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC 101 / 109 10.4 95.9 1.0X - Hive built-in ORC 372 / 387 2.8 355.1 0.3X + Native ORC Vectorized 96 / 103 11.0 91.1 1.0X + Native ORC MR 1111 / 1114 0.9 1059.4 0.1X + Hive built-in ORC 387 / 393 2.7 369.1 0.2X - SQL Single Column Scan FROM 200-Column Rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + SQL Single Column Scan from wide table (200 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC 187 / 203 5.6 178.7 1.0X - Hive built-in ORC 635 / 648 1.7 605.4 0.3X + Native ORC Vectorized 162 / 170 6.5 154.4 1.0X + Native ORC MR 2246 / 2257 0.5 2141.8 0.1X + Hive built-in ORC 580 / 586 1.8 553.4 0.3X - SQL Single Column Scan FROM 300-Column Rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + SQL Single Column Scan from wide table (300 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC 357 / 369 2.9 340.6 1.0X - Hive built-in ORC 960 / 967 1.1 915.3 0.4X + Native ORC Vectorized 272 / 281 3.9 259.7 1.0X + Native ORC MR 3376 / 3379 0.3 3220.0 0.1X + Hive built-in ORC 835 / 846 1.3 796.2 0.3X */ sqlBenchmark.run() } From e9675e7274a7bf5b6c575fa1c5bb95bb8de09c7a Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 5 Jan 2018 21:31:05 -0800 Subject: [PATCH 07/23] Update benchmarks and address comments. --- .../orc/OrcColumnarBatchReader.scala | 4 +- .../datasources/orc/OrcFileFormat.scala | 5 +- .../execution/datasources/orc/OrcUtils.scala | 19 -- .../spark/sql/hive/orc/OrcReadBenchmark.scala | 168 +++++++++--------- 4 files changed, 88 insertions(+), 108 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala index 46a6b3642353b..a6887ba1e0b5c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala @@ -131,7 +131,7 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat requiredSchema: StructType, partitionValues: InternalRow): Unit = { batch = orcSchema.createRowBatch(DEFAULT_SIZE) - assert(!batch.selectedInUse, "No filters are allowed") + assert(!batch.selectedInUse, "`selectedInUse` should be initialized with `false`.") this.requiredSchema = requiredSchema this.requestedColIds = requestedColIds @@ -171,7 +171,7 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat } recordReader.nextBatch(batch) - assert(!batch.selectedInUse, "No filters are allowed") + assert(!batch.selectedInUse, "`selectdInUse` is not used and all rows are qualified.") val batchSize = batch.size if (batchSize == 0) { return false diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala index 582ca4e048357..cd04118b1bf7d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala @@ -147,8 +147,7 @@ class OrcFileFormat } val resultSchema = StructType(requiredSchema.fields ++ partitionSchema.fields) - val enableVectorizedReader = sparkSession.sessionState.conf.orcVectorizedReaderEnabled && - supportBatch(sparkSession, resultSchema) + val enableVectorizedReader = supportBatch(sparkSession, resultSchema) val broadcastedConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) @@ -183,7 +182,7 @@ class OrcFileFormat val batchReader = new OrcColumnarBatchReader batchReader.initialize(fileSplit, taskAttemptContext) batchReader.setRequiredSchema( - OrcUtils.getFixedTypeDescription(reader.getSchema, dataSchema), + reader.getSchema, requestedColIds, resultSchema, requiredSchema, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala index 3eab55e1f3159..13a23996f4ade 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala @@ -107,23 +107,4 @@ object OrcUtils extends Logging { } } } - - /** - * Return a fixed ORC schema with data schema information, if needed. - * The schema inside old ORC files might consist of invalid column names like '_col0'. - */ - def getFixedTypeDescription( - schema: TypeDescription, - dataSchema: StructType): TypeDescription = { - val fieldNames = schema.getFieldNames.asScala - if (fieldNames.length == dataSchema.length && fieldNames.forall(_.startsWith("_col"))) { - var schemaString = schema.toString - dataSchema.zipWithIndex.foreach { case (field: StructField, index: Int) => - schemaString = schemaString.replace(s"_col$index:", s"${field.name}:") - } - TypeDescription.fromString(schemaString) - } else { - schema - } - } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala index ad38658269733..1b8987015f1bc 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala @@ -95,16 +95,16 @@ object OrcReadBenchmark { prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1")) - sqlBenchmark.addCase("Native ORC Vectorized") { _ => - spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() - } - sqlBenchmark.addCase("Native ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() } } + sqlBenchmark.addCase("Native ORC Vectorized") { _ => + spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() + } + sqlBenchmark.addCase("Hive built-in ORC") { _ => spark.sql("SELECT sum(id) FROM hiveOrcTable").collect() } @@ -115,39 +115,39 @@ object OrcReadBenchmark { SQL Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC Vectorized 156 / 163 100.7 9.9 1.0X - Native ORC MR 1222 / 1236 12.9 77.7 0.1X - Hive built-in ORC 1572 / 1625 10.0 100.0 0.1X + Native ORC MR 1193 / 1228 13.2 75.9 1.0X + Native ORC Vectorized 160 / 176 98.1 10.2 7.4X + Hive built-in ORC 1479 / 1482 10.6 94.1 0.8X SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC Vectorized 215 / 225 73.1 13.7 1.0X - Native ORC MR 1337 / 1458 11.8 85.0 0.2X - Hive built-in ORC 1696 / 1707 9.3 107.8 0.1X + Native ORC MR 1268 / 1278 12.4 80.6 1.0X + Native ORC Vectorized 164 / 175 95.8 10.4 7.7X + Hive built-in ORC 1814 / 1833 8.7 115.3 0.7X SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC Vectorized 285 / 294 55.2 18.1 1.0X - Native ORC MR 1397 / 1419 11.3 88.8 0.2X - Hive built-in ORC 2086 / 2143 7.5 132.6 0.1X + Native ORC MR 1352 / 1467 11.6 86.0 1.0X + Native ORC Vectorized 232 / 238 67.8 14.7 5.8X + Hive built-in ORC 2006 / 2009 7.8 127.5 0.7X SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC Vectorized 341 / 350 46.1 21.7 1.0X - Native ORC MR 1461 / 1492 10.8 92.9 0.2X - Hive built-in ORC 2002 / 2095 7.9 127.3 0.2X + Native ORC MR 1364 / 1367 11.5 86.7 1.0X + Native ORC Vectorized 288 / 293 54.6 18.3 4.7X + Hive built-in ORC 2015 / 2060 7.8 128.1 0.7X SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC Vectorized 347 / 356 45.3 22.1 1.0X - Native ORC MR 1524 / 1553 10.3 96.9 0.2X - Hive built-in ORC 2393 / 2404 6.6 152.1 0.1X + Native ORC MR 1443 / 1467 10.9 91.7 1.0X + Native ORC Vectorized 332 / 334 47.4 21.1 4.4X + Hive built-in ORC 2126 / 2146 7.4 135.2 0.7X SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC Vectorized 403 / 411 39.0 25.6 1.0X - Native ORC MR 1517 / 1529 10.4 96.5 0.3X - Hive built-in ORC 2054 / 2134 7.7 130.6 0.2X + Native ORC MR 1548 / 1563 10.2 98.4 1.0X + Native ORC Vectorized 393 / 397 40.0 25.0 3.9X + Hive built-in ORC 2156 / 2161 7.3 137.1 0.7X */ sqlBenchmark.run() } @@ -166,16 +166,16 @@ object OrcReadBenchmark { dir, spark.sql("SELECT CAST(value AS INT) AS c1, CAST(value as STRING) AS c2 FROM t1")) - benchmark.addCase("Native ORC Vectorized") { _ => - spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect() - } - benchmark.addCase("Native ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect() } } + benchmark.addCase("Native ORC Vectorized") { _ => + spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect() + } + benchmark.addCase("Hive built-in ORC") { _ => spark.sql("SELECT sum(c1), sum(length(c2)) FROM hiveOrcTable").collect() } @@ -186,9 +186,9 @@ object OrcReadBenchmark { Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC Vectorized 1382 / 1400 7.6 131.8 1.0X - Native ORC MR 2689 / 2765 3.9 256.4 0.5X - Hive built-in ORC 3889 / 3894 2.7 370.9 0.4X + Native ORC MR 2730 / 2743 3.8 260.4 1.0X + Native ORC Vectorized 1369 / 1376 7.7 130.5 2.0X + Hive built-in ORC 3802 / 3803 2.8 362.6 0.7X */ benchmark.run() } @@ -205,22 +205,18 @@ object OrcReadBenchmark { prepareTable(dir, spark.sql("SELECT value % 2 AS p, value AS id FROM t1"), Some("p")) - benchmark.addCase("Read data column - Native ORC Vectorized") { _ => - spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() - } - benchmark.addCase("Read data column - Native ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() } } - benchmark.addCase("Read data column - Hive built-in ORC") { _ => - spark.sql("SELECT sum(id) FROM hiveOrcTable").collect() + benchmark.addCase("Read data column - Native ORC Vectorized") { _ => + spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() } - benchmark.addCase("Read partition column - Native ORC Vectorized") { _ => - spark.sql("SELECT sum(p) FROM nativeOrcTable").collect() + benchmark.addCase("Read data column - Hive built-in ORC") { _ => + spark.sql("SELECT sum(id) FROM hiveOrcTable").collect() } benchmark.addCase("Read partition column - Native ORC MR") { _ => @@ -229,12 +225,12 @@ object OrcReadBenchmark { } } - benchmark.addCase("Read partition column - Hive built-in ORC") { _ => - spark.sql("SELECT sum(p) FROM hiveOrcTable").collect() + benchmark.addCase("Read partition column - Native ORC Vectorized") { _ => + spark.sql("SELECT sum(p) FROM nativeOrcTable").collect() } - benchmark.addCase("Read both columns - Native ORC Vectorized") { _ => - spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect() + benchmark.addCase("Read partition column - Hive built-in ORC") { _ => + spark.sql("SELECT sum(p) FROM hiveOrcTable").collect() } benchmark.addCase("Read both columns - Native ORC MR") { _ => @@ -243,6 +239,10 @@ object OrcReadBenchmark { } } + benchmark.addCase("Read both columns - Native ORC Vectorized") { _ => + spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect() + } + benchmark.addCase("Read both columns - Hive built-in ORC") { _ => spark.sql("SELECT sum(p), sum(id) FROM hiveOrcTable").collect() } @@ -253,15 +253,15 @@ object OrcReadBenchmark { Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Read data column - Native ORC Vectorized 347 / 350 45.3 22.1 1.0X - Read data column - Native ORC MR 1611 / 1613 9.8 102.4 0.2X - Read data column - Hive built-in ORC 2082 / 2091 7.6 132.4 0.2X - Read partition column - Native ORC Vectorized 55 / 57 286.5 3.5 6.3X - Read partition column - Native ORC MR 1062 / 1063 14.8 67.5 0.3X - Read partition column - Hive built-in ORC 1334 / 1334 11.8 84.8 0.3X - Read both columns - Native ORC Vectorized 380 / 388 41.3 24.2 0.9X - Read both columns - Native ORC MR 1654 / 1672 9.5 105.2 0.2X - Read both columns - Hive built-in ORC 2209 / 2209 7.1 140.5 0.2X + Read data column - Native ORC MR 1518 / 1543 10.4 96.5 1.0X + Read data column - Native ORC Vectorized 293 / 300 53.7 18.6 5.2X + Read data column - Hive built-in ORC 2104 / 2128 7.5 133.8 0.7X + Read partition column - Native ORC MR 1043 / 1056 15.1 66.3 1.5X + Read partition column - Native ORC Vectorized 53 / 56 294.4 3.4 28.4X + Read partition column - Hive built-in ORC 1328 / 1346 11.8 84.4 1.1X + Read both columns - Native ORC MR 1573 / 1594 10.0 100.0 1.0X + Read both columns - Native ORC Vectorized 327 / 335 48.1 20.8 4.6X + Read both columns - Hive built-in ORC 2149 / 2177 7.3 136.6 0.7X */ benchmark.run() } @@ -277,16 +277,16 @@ object OrcReadBenchmark { prepareTable(dir, spark.sql("SELECT CAST((id % 200) + 10000 as STRING) AS c1 FROM t1")) - benchmark.addCase("Native ORC Vectorized") { _ => - spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect() - } - benchmark.addCase("Native ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect() } } + benchmark.addCase("Native ORC Vectorized") { _ => + spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect() + } + benchmark.addCase("Hive built-in ORC") { _ => spark.sql("SELECT sum(length(c1)) FROM hiveOrcTable").collect() } @@ -297,9 +297,9 @@ object OrcReadBenchmark { String Dictionary: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC Vectorized 406 / 414 25.8 38.7 1.0X - Native ORC MR 1372 / 1381 7.6 130.8 0.3X - Hive built-in ORC 2016 / 2036 5.2 192.2 0.2X + Native ORC MR 1424 / 1425 7.4 135.8 1.0X + Native ORC Vectorized 374 / 387 28.0 35.7 3.8X + Hive built-in ORC 2042 / 2065 5.1 194.7 0.7X */ benchmark.run() } @@ -319,11 +319,6 @@ object OrcReadBenchmark { val benchmark = new Benchmark("String with Nulls Scan", values) - benchmark.addCase(s"Native ORC Vectorized ($fractionOfNulls%)") { _ => - spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " + - "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() - } - benchmark.addCase(s"Native ORC MR ($fractionOfNulls%)") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " + @@ -331,6 +326,11 @@ object OrcReadBenchmark { } } + benchmark.addCase(s"Native ORC Vectorized ($fractionOfNulls%)") { _ => + spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " + + "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() + } + benchmark.addCase(s"Hive built-in ORC ($fractionOfNulls%)") { _ => spark.sql("SELECT SUM(LENGTH(c2)) FROM hiveOrcTable " + "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() @@ -342,21 +342,21 @@ object OrcReadBenchmark { String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC Vectorized (0.0%) 1122 / 1129 9.3 107.0 1.0X - Native ORC MR (0.0%) 2551 / 2619 4.1 243.3 0.4X - Hive built-in ORC (0.0%) 3938 / 3966 2.7 375.5 0.3X + Native ORC MR (0.0%) 2512 / 2544 4.2 239.5 1.0X + Native ORC Vectorized (0.0%) 1054 / 1058 9.9 100.5 2.4X + Hive built-in ORC (0.0%) 3902 / 3920 2.7 372.2 0.6X String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC Vectorized (0.5%) 1303 / 1314 8.0 124.2 1.0X - Native ORC MR (0.5%) 2470 / 2485 4.2 235.5 0.5X - Hive built-in ORC (0.5%) 2933 / 2950 3.6 279.7 0.4X + Native ORC MR (0.5%) 2401 / 2419 4.4 228.9 1.0X + Native ORC Vectorized (0.5%) 1253 / 1256 8.4 119.5 1.9X + Hive built-in ORC (0.5%) 3299 / 3306 3.2 314.6 0.7X String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC Vectorized (0.95%) 475 / 482 22.1 45.3 1.0X - Native ORC MR (0.95%) 1333 / 1348 7.9 127.2 0.4X - Hive built-in ORC (0.95%) 1695 / 1699 6.2 161.7 0.3X + Native ORC MR (0.95%) 1292 / 1303 8.1 123.2 1.0X + Native ORC Vectorized (0.95%) 464 / 480 22.6 44.3 2.8X + Hive built-in ORC (0.95%) 2091 / 2116 5.0 199.4 0.6X */ benchmark.run() } @@ -376,16 +376,16 @@ object OrcReadBenchmark { prepareTable(dir, spark.sql("SELECT * FROM t1")) - sqlBenchmark.addCase("Native ORC Vectorized") { _ => - spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect() - } - sqlBenchmark.addCase("Native ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect() } } + sqlBenchmark.addCase("Native ORC Vectorized") { _ => + spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect() + } + sqlBenchmark.addCase("Hive built-in ORC") { _ => spark.sql(s"SELECT sum(c$middle) FROM hiveOrcTable").collect() } @@ -396,21 +396,21 @@ object OrcReadBenchmark { SQL Single Column Scan from wide table (100 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC Vectorized 96 / 103 11.0 91.1 1.0X - Native ORC MR 1111 / 1114 0.9 1059.4 0.1X - Hive built-in ORC 387 / 393 2.7 369.1 0.2X + Native ORC MR 1082 / 1096 1.0 1032.1 1.0X + Native ORC Vectorized 93 / 99 11.3 88.3 11.7X + Hive built-in ORC 394 / 404 2.7 375.3 2.8X SQL Single Column Scan from wide table (200 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC Vectorized 162 / 170 6.5 154.4 1.0X - Native ORC MR 2246 / 2257 0.5 2141.8 0.1X - Hive built-in ORC 580 / 586 1.8 553.4 0.3X + Native ORC MR 2179 / 2190 0.5 2077.9 1.0X + Native ORC Vectorized 158 / 165 6.6 150.5 13.8X + Hive built-in ORC 617 / 627 1.7 588.4 3.5X SQL Single Column Scan from wide table (300 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC Vectorized 272 / 281 3.9 259.7 1.0X - Native ORC MR 3376 / 3379 0.3 3220.0 0.1X - Hive built-in ORC 835 / 846 1.3 796.2 0.3X + Native ORC MR 3229 / 3230 0.3 3079.0 1.0X + Native ORC Vectorized 270 / 279 3.9 257.6 12.0X + Hive built-in ORC 881 / 897 1.2 840.3 3.7X */ sqlBenchmark.run() } From 1e6ca5c7ddb1fab6acc1d4ff130a05d3ca1b576d Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 5 Jan 2018 22:07:27 -0800 Subject: [PATCH 08/23] Address comments. --- .../orc/OrcColumnarBatchReader.scala | 3 +- .../datasources/orc/OrcFileFormat.scala | 2 +- .../spark/sql/hive/orc/OrcReadBenchmark.scala | 102 +++++++++--------- 3 files changed, 54 insertions(+), 53 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala index a6887ba1e0b5c..a02fb5300c0ca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala @@ -127,12 +127,13 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat def setRequiredSchema( orcSchema: TypeDescription, requestedColIds: Array[Int], - resultSchema: StructType, requiredSchema: StructType, + partitionSchema: StructType, partitionValues: InternalRow): Unit = { batch = orcSchema.createRowBatch(DEFAULT_SIZE) assert(!batch.selectedInUse, "`selectedInUse` should be initialized with `false`.") + val resultSchema = StructType(requiredSchema.fields ++ partitionSchema.fields) this.requiredSchema = requiredSchema this.requestedColIds = requestedColIds diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala index cd04118b1bf7d..c5eed92fd94ff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala @@ -184,8 +184,8 @@ class OrcFileFormat batchReader.setRequiredSchema( reader.getSchema, requestedColIds, - resultSchema, requiredSchema, + partitionSchema, file.partitionValues) val iter = new RecordReaderIterator(batchReader) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala index 1b8987015f1bc..125e79ec40c5a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala @@ -115,39 +115,39 @@ object OrcReadBenchmark { SQL Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1193 / 1228 13.2 75.9 1.0X - Native ORC Vectorized 160 / 176 98.1 10.2 7.4X - Hive built-in ORC 1479 / 1482 10.6 94.1 0.8X + Native ORC MR 1184 / 1202 13.3 75.3 1.0X + Native ORC Vectorized 164 / 175 96.0 10.4 7.2X + Hive built-in ORC 1421 / 1446 11.1 90.4 0.8X SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1268 / 1278 12.4 80.6 1.0X - Native ORC Vectorized 164 / 175 95.8 10.4 7.7X - Hive built-in ORC 1814 / 1833 8.7 115.3 0.7X + Native ORC MR 1202 / 1211 13.1 76.4 1.0X + Native ORC Vectorized 166 / 171 95.0 10.5 7.3X + Hive built-in ORC 1666 / 1676 9.4 105.9 0.7X SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1352 / 1467 11.6 86.0 1.0X - Native ORC Vectorized 232 / 238 67.8 14.7 5.8X - Hive built-in ORC 2006 / 2009 7.8 127.5 0.7X + Native ORC MR 1312 / 1374 12.0 83.4 1.0X + Native ORC Vectorized 233 / 245 67.5 14.8 5.6X + Hive built-in ORC 1913 / 1944 8.2 121.6 0.7X SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1364 / 1367 11.5 86.7 1.0X - Native ORC Vectorized 288 / 293 54.6 18.3 4.7X - Hive built-in ORC 2015 / 2060 7.8 128.1 0.7X + Native ORC MR 1479 / 1483 10.6 94.0 1.0X + Native ORC Vectorized 288 / 296 54.6 18.3 5.1X + Hive built-in ORC 1934 / 1948 8.1 122.9 0.8X SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1443 / 1467 10.9 91.7 1.0X - Native ORC Vectorized 332 / 334 47.4 21.1 4.4X - Hive built-in ORC 2126 / 2146 7.4 135.2 0.7X + Native ORC MR 1378 / 1399 11.4 87.6 1.0X + Native ORC Vectorized 330 / 335 47.6 21.0 4.2X + Hive built-in ORC 1938 / 1938 8.1 123.2 0.7X SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1548 / 1563 10.2 98.4 1.0X - Native ORC Vectorized 393 / 397 40.0 25.0 3.9X - Hive built-in ORC 2156 / 2161 7.3 137.1 0.7X + Native ORC MR 1484 / 1486 10.6 94.4 1.0X + Native ORC Vectorized 395 / 403 39.8 25.1 3.8X + Hive built-in ORC 2073 / 2099 7.6 131.8 0.7X */ sqlBenchmark.run() } @@ -186,9 +186,9 @@ object OrcReadBenchmark { Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2730 / 2743 3.8 260.4 1.0X - Native ORC Vectorized 1369 / 1376 7.7 130.5 2.0X - Hive built-in ORC 3802 / 3803 2.8 362.6 0.7X + Native ORC MR 2616 / 2693 4.0 249.5 1.0X + Native ORC Vectorized 1343 / 1355 7.8 128.0 1.9X + Hive built-in ORC 3684 / 3699 2.8 351.3 0.7X */ benchmark.run() } @@ -253,15 +253,15 @@ object OrcReadBenchmark { Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Read data column - Native ORC MR 1518 / 1543 10.4 96.5 1.0X - Read data column - Native ORC Vectorized 293 / 300 53.7 18.6 5.2X - Read data column - Hive built-in ORC 2104 / 2128 7.5 133.8 0.7X - Read partition column - Native ORC MR 1043 / 1056 15.1 66.3 1.5X - Read partition column - Native ORC Vectorized 53 / 56 294.4 3.4 28.4X - Read partition column - Hive built-in ORC 1328 / 1346 11.8 84.4 1.1X - Read both columns - Native ORC MR 1573 / 1594 10.0 100.0 1.0X - Read both columns - Native ORC Vectorized 327 / 335 48.1 20.8 4.6X - Read both columns - Hive built-in ORC 2149 / 2177 7.3 136.6 0.7X + Read data column - Native ORC MR 1532 / 1541 10.3 97.4 1.0X + Read data column - Native ORC Vectorized 296 / 300 53.2 18.8 5.2X + Read data column - Hive built-in ORC 2089 / 2105 7.5 132.8 0.7X + Read partition column - Native ORC MR 1063 / 1074 14.8 67.6 1.4X + Read partition column - Native ORC Vectorized 53 / 56 299.2 3.3 29.1X + Read partition column - Hive built-in ORC 1289 / 1293 12.2 82.0 1.2X + Read both columns - Native ORC MR 1592 / 1605 9.9 101.2 1.0X + Read both columns - Native ORC Vectorized 333 / 338 47.3 21.2 4.6X + Read both columns - Hive built-in ORC 2200 / 2200 7.1 139.9 0.7X */ benchmark.run() } @@ -297,9 +297,9 @@ object OrcReadBenchmark { String Dictionary: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1424 / 1425 7.4 135.8 1.0X - Native ORC Vectorized 374 / 387 28.0 35.7 3.8X - Hive built-in ORC 2042 / 2065 5.1 194.7 0.7X + Native ORC MR 1307 / 1330 8.0 124.6 1.0X + Native ORC Vectorized 377 / 387 27.8 36.0 3.5X + Hive built-in ORC 2009 / 2031 5.2 191.6 0.7X */ benchmark.run() } @@ -342,21 +342,21 @@ object OrcReadBenchmark { String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR (0.0%) 2512 / 2544 4.2 239.5 1.0X - Native ORC Vectorized (0.0%) 1054 / 1058 9.9 100.5 2.4X - Hive built-in ORC (0.0%) 3902 / 3920 2.7 372.2 0.6X + Native ORC MR (0.0%) 2498 / 2539 4.2 238.3 1.0X + Native ORC Vectorized (0.0%) 1047 / 1066 10.0 99.9 2.4X + Hive built-in ORC (0.0%) 3904 / 3957 2.7 372.3 0.6X String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR (0.5%) 2401 / 2419 4.4 228.9 1.0X - Native ORC Vectorized (0.5%) 1253 / 1256 8.4 119.5 1.9X - Hive built-in ORC (0.5%) 3299 / 3306 3.2 314.6 0.7X + Native ORC MR (0.5%) 2396 / 2435 4.4 228.5 1.0X + Native ORC Vectorized (0.5%) 1237 / 1240 8.5 118.0 1.9X + Hive built-in ORC (0.5%) 2923 / 2941 3.6 278.7 0.8X String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR (0.95%) 1292 / 1303 8.1 123.2 1.0X - Native ORC Vectorized (0.95%) 464 / 480 22.6 44.3 2.8X - Hive built-in ORC (0.95%) 2091 / 2116 5.0 199.4 0.6X + Native ORC MR (0.95%) 1341 / 1353 7.8 127.9 1.0X + Native ORC Vectorized (0.95%) 463 / 473 22.6 44.2 2.9X + Hive built-in ORC (0.95%) 1667 / 1692 6.3 158.9 0.8X */ benchmark.run() } @@ -396,21 +396,21 @@ object OrcReadBenchmark { SQL Single Column Scan from wide table (100 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1082 / 1096 1.0 1032.1 1.0X - Native ORC Vectorized 93 / 99 11.3 88.3 11.7X - Hive built-in ORC 394 / 404 2.7 375.3 2.8X + Native ORC MR 1246 / 1260 0.8 1188.5 1.0X + Native ORC Vectorized 92 / 97 11.4 87.6 13.6X + Hive built-in ORC 373 / 386 2.8 355.9 3.3X SQL Single Column Scan from wide table (200 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2179 / 2190 0.5 2077.9 1.0X - Native ORC Vectorized 158 / 165 6.6 150.5 13.8X - Hive built-in ORC 617 / 627 1.7 588.4 3.5X + Native ORC MR 2513 / 2515 0.4 2396.6 1.0X + Native ORC Vectorized 156 / 164 6.7 149.2 16.1X + Hive built-in ORC 582 / 592 1.8 555.4 4.3X SQL Single Column Scan from wide table (300 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 3229 / 3230 0.3 3079.0 1.0X - Native ORC Vectorized 270 / 279 3.9 257.6 12.0X - Hive built-in ORC 881 / 897 1.2 840.3 3.7X + Native ORC MR 3742 / 3746 0.3 3568.2 1.0X + Native ORC Vectorized 263 / 273 4.0 250.6 14.2X + Hive built-in ORC 828 / 844 1.3 789.3 4.5X */ sqlBenchmark.run() } From 803c1d1347ab1e016540d1ae4a9cb44e767794e5 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 6 Jan 2018 15:38:16 -0800 Subject: [PATCH 09/23] fix --- .../datasources/orc/OrcFileFormat.scala | 5 +- .../spark/sql/hive/orc/OrcReadBenchmark.scala | 102 +++++++++--------- 2 files changed, 54 insertions(+), 53 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala index c5eed92fd94ff..acb8066d0b7d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala @@ -171,12 +171,13 @@ class OrcFileFormat val requestedColIds = requestedColIdsOrEmptyFile.get assert(requestedColIds.length == requiredSchema.length, "[BUG] requested column IDs do not match required schema") - conf.set(OrcConf.INCLUDE_COLUMNS.getAttribute, + val taskConf = new Configuration(conf) + taskConf.set(OrcConf.INCLUDE_COLUMNS.getAttribute, requestedColIds.filter(_ != -1).sorted.mkString(",")) val fileSplit = new FileSplit(filePath, file.start, file.length, Array.empty) val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) - val taskAttemptContext = new TaskAttemptContextImpl(conf, attemptId) + val taskAttemptContext = new TaskAttemptContextImpl(taskConf, attemptId) if (enableVectorizedReader) { val batchReader = new OrcColumnarBatchReader diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala index 125e79ec40c5a..60686acc89fba 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala @@ -115,39 +115,39 @@ object OrcReadBenchmark { SQL Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1184 / 1202 13.3 75.3 1.0X - Native ORC Vectorized 164 / 175 96.0 10.4 7.2X - Hive built-in ORC 1421 / 1446 11.1 90.4 0.8X + Native ORC MR 1221 / 1268 12.9 77.6 1.0X + Native ORC Vectorized 164 / 174 96.1 10.4 7.5X + Hive built-in ORC 1440 / 1449 10.9 91.5 0.8X SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1202 / 1211 13.1 76.4 1.0X - Native ORC Vectorized 166 / 171 95.0 10.5 7.3X - Hive built-in ORC 1666 / 1676 9.4 105.9 0.7X + Native ORC MR 1283 / 1322 12.3 81.6 1.0X + Native ORC Vectorized 166 / 180 94.9 10.5 7.7X + Hive built-in ORC 1619 / 1645 9.7 102.9 0.8X SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1312 / 1374 12.0 83.4 1.0X - Native ORC Vectorized 233 / 245 67.5 14.8 5.6X - Hive built-in ORC 1913 / 1944 8.2 121.6 0.7X + Native ORC MR 1313 / 1425 12.0 83.4 1.0X + Native ORC Vectorized 231 / 239 68.0 14.7 5.7X + Hive built-in ORC 2048 / 2062 7.7 130.2 0.6X SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1479 / 1483 10.6 94.0 1.0X - Native ORC Vectorized 288 / 296 54.6 18.3 5.1X - Hive built-in ORC 1934 / 1948 8.1 122.9 0.8X + Native ORC MR 1364 / 1366 11.5 86.7 1.0X + Native ORC Vectorized 291 / 297 54.1 18.5 4.7X + Hive built-in ORC 2056 / 2058 7.6 130.7 0.7X SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1378 / 1399 11.4 87.6 1.0X - Native ORC Vectorized 330 / 335 47.6 21.0 4.2X - Hive built-in ORC 1938 / 1938 8.1 123.2 0.7X + Native ORC MR 1528 / 1563 10.3 97.2 1.0X + Native ORC Vectorized 343 / 358 45.9 21.8 4.5X + Hive built-in ORC 1999 / 2031 7.9 127.1 0.8X SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1484 / 1486 10.6 94.4 1.0X - Native ORC Vectorized 395 / 403 39.8 25.1 3.8X - Hive built-in ORC 2073 / 2099 7.6 131.8 0.7X + Native ORC MR 1483 / 1514 10.6 94.3 1.0X + Native ORC Vectorized 406 / 420 38.7 25.8 3.6X + Hive built-in ORC 2147 / 2160 7.3 136.5 0.7X */ sqlBenchmark.run() } @@ -186,9 +186,9 @@ object OrcReadBenchmark { Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2616 / 2693 4.0 249.5 1.0X - Native ORC Vectorized 1343 / 1355 7.8 128.0 1.9X - Hive built-in ORC 3684 / 3699 2.8 351.3 0.7X + Native ORC MR 2631 / 2700 4.0 250.9 1.0X + Native ORC Vectorized 1358 / 1363 7.7 129.5 1.9X + Hive built-in ORC 3602 / 3645 2.9 343.5 0.7X */ benchmark.run() } @@ -253,15 +253,15 @@ object OrcReadBenchmark { Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Read data column - Native ORC MR 1532 / 1541 10.3 97.4 1.0X - Read data column - Native ORC Vectorized 296 / 300 53.2 18.8 5.2X - Read data column - Hive built-in ORC 2089 / 2105 7.5 132.8 0.7X - Read partition column - Native ORC MR 1063 / 1074 14.8 67.6 1.4X - Read partition column - Native ORC Vectorized 53 / 56 299.2 3.3 29.1X - Read partition column - Hive built-in ORC 1289 / 1293 12.2 82.0 1.2X - Read both columns - Native ORC MR 1592 / 1605 9.9 101.2 1.0X - Read both columns - Native ORC Vectorized 333 / 338 47.3 21.2 4.6X - Read both columns - Hive built-in ORC 2200 / 2200 7.1 139.9 0.7X + Read data column - Native ORC MR 1486 / 1502 10.6 94.5 1.0X + Read data column - Native ORC Vectorized 293 / 301 53.7 18.6 5.1X + Read data column - Hive built-in ORC 2071 / 2078 7.6 131.6 0.7X + Read partition column - Native ORC MR 1048 / 1054 15.0 66.6 1.4X + Read partition column - Native ORC Vectorized 55 / 57 287.4 3.5 27.2X + Read partition column - Hive built-in ORC 1268 / 1280 12.4 80.6 1.2X + Read both columns - Native ORC MR 1547 / 1562 10.2 98.3 1.0X + Read both columns - Native ORC Vectorized 330 / 334 47.6 21.0 4.5X + Read both columns - Hive built-in ORC 2114 / 2122 7.4 134.4 0.7X */ benchmark.run() } @@ -297,9 +297,9 @@ object OrcReadBenchmark { String Dictionary: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1307 / 1330 8.0 124.6 1.0X - Native ORC Vectorized 377 / 387 27.8 36.0 3.5X - Hive built-in ORC 2009 / 2031 5.2 191.6 0.7X + Native ORC MR 1310 / 1317 8.0 124.9 1.0X + Native ORC Vectorized 370 / 379 28.4 35.3 3.5X + Hive built-in ORC 1938 / 1956 5.4 184.9 0.7X */ benchmark.run() } @@ -342,21 +342,21 @@ object OrcReadBenchmark { String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR (0.0%) 2498 / 2539 4.2 238.3 1.0X - Native ORC Vectorized (0.0%) 1047 / 1066 10.0 99.9 2.4X - Hive built-in ORC (0.0%) 3904 / 3957 2.7 372.3 0.6X + Native ORC MR (0.0%) 2505 / 2527 4.2 238.9 1.0X + Native ORC Vectorized (0.0%) 1052 / 1059 10.0 100.3 2.4X + Hive built-in ORC (0.0%) 3855 / 3918 2.7 367.7 0.6X String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR (0.5%) 2396 / 2435 4.4 228.5 1.0X - Native ORC Vectorized (0.5%) 1237 / 1240 8.5 118.0 1.9X - Hive built-in ORC (0.5%) 2923 / 2941 3.6 278.7 0.8X + Native ORC MR (0.5%) 2398 / 2413 4.4 228.7 1.0X + Native ORC Vectorized (0.5%) 1263 / 1266 8.3 120.4 1.9X + Hive built-in ORC (0.5%) 2919 / 2937 3.6 278.4 0.8X String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR (0.95%) 1341 / 1353 7.8 127.9 1.0X - Native ORC Vectorized (0.95%) 463 / 473 22.6 44.2 2.9X - Hive built-in ORC (0.95%) 1667 / 1692 6.3 158.9 0.8X + Native ORC MR (0.95%) 1296 / 1318 8.1 123.6 1.0X + Native ORC Vectorized (0.95%) 464 / 468 22.6 44.2 2.8X + Hive built-in ORC (0.95%) 1592 / 1607 6.6 151.8 0.8X */ benchmark.run() } @@ -396,21 +396,21 @@ object OrcReadBenchmark { SQL Single Column Scan from wide table (100 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1246 / 1260 0.8 1188.5 1.0X - Native ORC Vectorized 92 / 97 11.4 87.6 13.6X - Hive built-in ORC 373 / 386 2.8 355.9 3.3X + Native ORC MR 1068 / 1072 1.0 1018.7 1.0X + Native ORC Vectorized 94 / 99 11.2 89.5 11.4X + Hive built-in ORC 372 / 384 2.8 355.0 2.9X SQL Single Column Scan from wide table (200 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2513 / 2515 0.4 2396.6 1.0X - Native ORC Vectorized 156 / 164 6.7 149.2 16.1X - Hive built-in ORC 582 / 592 1.8 555.4 4.3X + Native ORC MR 2154 / 2162 0.5 2054.3 1.0X + Native ORC Vectorized 158 / 164 6.6 150.9 13.6X + Hive built-in ORC 601 / 606 1.7 573.0 3.6X SQL Single Column Scan from wide table (300 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 3742 / 3746 0.3 3568.2 1.0X - Native ORC Vectorized 263 / 273 4.0 250.6 14.2X - Hive built-in ORC 828 / 844 1.3 789.3 4.5X + Native ORC MR 3217 / 3226 0.3 3068.3 1.0X + Native ORC Vectorized 270 / 277 3.9 257.7 11.9X + Hive built-in ORC 840 / 853 1.2 801.2 3.8X */ sqlBenchmark.run() } From 7214ec03f8e48d51e0fd1f3314a0af6ac8275412 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 7 Jan 2018 12:49:36 -0800 Subject: [PATCH 10/23] Add JavaOrcColumnarBatchReader and benchmark. --- .../apache/spark/sql/internal/SQLConf.scala | 7 + .../orc/JavaOrcColumnarBatchReader.java | 503 ++++++++++++++++++ .../datasources/orc/OrcFileFormat.scala | 30 +- .../spark/sql/hive/orc/OrcReadBenchmark.scala | 206 ++++--- 4 files changed, 676 insertions(+), 70 deletions(-) create mode 100644 sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 74949db883f7a..9451f85b57a4f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -391,6 +391,11 @@ object SQLConf { .booleanConf .createWithDefault(true) + val ORC_VECTORIZED_JAVA_READER_ENABLED = buildConf("spark.sql.orc.enableVectorizedJavaReader") + .doc("Enables vectorized java-version orc decoding.") + .booleanConf + .createWithDefault(true) + val ORC_FILTER_PUSHDOWN_ENABLED = buildConf("spark.sql.orc.filterPushdown") .doc("When true, enable filter pushdown for ORC files.") .booleanConf @@ -1190,6 +1195,8 @@ class SQLConf extends Serializable with Logging { def orcVectorizedReaderEnabled: Boolean = getConf(ORC_VECTORIZED_READER_ENABLED) + def orcVectorizedJavaReaderEnabled: Boolean = getConf(ORC_VECTORIZED_JAVA_READER_ENABLED) + def parquetCompressionCodec: String = getConf(PARQUET_COMPRESSION) def parquetVectorizedReaderEnabled: Boolean = getConf(PARQUET_VECTORIZED_READER_ENABLED) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java new file mode 100644 index 0000000000000..790685d43413b --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java @@ -0,0 +1,503 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.orc; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.lib.input.FileSplit; +import org.apache.orc.OrcConf; +import org.apache.orc.OrcFile; +import org.apache.orc.Reader; +import org.apache.orc.TypeDescription; +import org.apache.orc.mapred.OrcInputFormat; +import org.apache.orc.storage.common.type.HiveDecimal; +import org.apache.orc.storage.ql.exec.vector.*; +import org.apache.orc.storage.serde2.io.HiveDecimalWritable; + +import org.apache.spark.memory.MemoryMode; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils; +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector; +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector; +import org.apache.spark.sql.execution.vectorized.WritableColumnVector; +import org.apache.spark.sql.types.*; +import org.apache.spark.sql.vectorized.ColumnarBatch; + + +/** + * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch. + * After creating, `initialize` and `setRequiredSchema` should be called sequentially. + */ +public class JavaOrcColumnarBatchReader extends RecordReader { + + /** + * ORC File Reader. + */ + private Reader reader; + + /** + * Vectorized Row Batch. + */ + private VectorizedRowBatch batch; + + /** + * Requested Column IDs. + */ + private int[] requestedColIds; + + /** + * Record reader from row batch. + */ + private org.apache.orc.RecordReader recordReader; + + /** + * Required Schema. + */ + private StructType requiredSchema; + + /** + * ColumnarBatch for vectorized execution by whole-stage codegen. + */ + private ColumnarBatch columnarBatch; + + /** + * Writable column vectors of ColumnarBatch. + */ + private WritableColumnVector[] columnVectors; + + /** + * The number of rows read and considered to be returned. + */ + private long rowsReturned = 0L; + + /** + * Total number of rows. + */ + private long totalRowCount = 0L; + + @Override + public Void getCurrentKey() throws IOException, InterruptedException { + return null; + } + + @Override + public ColumnarBatch getCurrentValue() throws IOException, InterruptedException { + return columnarBatch; + } + + @Override + public float getProgress() throws IOException, InterruptedException { + return (float) rowsReturned / totalRowCount; + } + + @Override + public boolean nextKeyValue() throws IOException, InterruptedException { + return nextBatch(); + } + + @Override + public void close() throws IOException { + if (columnarBatch != null) { + columnarBatch.close(); + columnarBatch = null; + } + if (recordReader != null) { + recordReader.close(); + recordReader = null; + } + } + + /** + * Initialize ORC file reader and batch record reader. + * Please note that `setRequiredSchema` is needed to be called after this. + */ + @Override + public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) + throws IOException, InterruptedException { + FileSplit fileSplit = (FileSplit)inputSplit; + Configuration conf = taskAttemptContext.getConfiguration(); + reader = OrcFile.createReader( + fileSplit.getPath(), + OrcFile.readerOptions(conf) + .maxLength(OrcConf.MAX_FILE_LENGTH.getLong(conf)) + .filesystem(fileSplit.getPath().getFileSystem(conf))); + + Reader.Options options = + OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength()); + recordReader = reader.rows(options); + totalRowCount = reader.getNumberOfRows(); + } + + /** + * Set required schema and partition information. + * With this information, this creates ColumnarBatch with the full schema. + */ + public void setRequiredSchema( + TypeDescription orcSchema, + int[] requestedColIds, + StructType requiredSchema, + StructType partitionSchema, + InternalRow partitionValues) { + batch = orcSchema.createRowBatch(DEFAULT_SIZE); + assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`. + + StructType resultSchema = new StructType(requiredSchema.fields()); + for (StructField f : partitionSchema.fields()) + resultSchema = resultSchema.add(f); + this.requiredSchema = requiredSchema; + this.requestedColIds = requestedColIds; + + int capacity = DEFAULT_SIZE; + if (DEFAULT_MEMORY_MODE == MemoryMode.OFF_HEAP) { + columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema); + } else { + columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema); + } + columnarBatch = new ColumnarBatch(resultSchema, columnVectors, capacity); + + if (partitionValues.numFields() > 0) { + int partitionIdx = requiredSchema.fields().length; + for (int i = 0; i < partitionValues.numFields(); i++) { + ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i); + columnVectors[i + partitionIdx].setIsConstant(); + } + } + + // Initialize the missing columns once. + for (int i = 0; i < requiredSchema.length(); i++) { + if (requestedColIds[i] < 0) { + columnVectors[i].putNulls(0, columnarBatch.capacity()); + columnVectors[i].setIsConstant(); + } + } + } + + /** + * Return true if there exists more data in the next batch. If exists, prepare the next batch + * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns. + */ + private boolean nextBatch() throws IOException { + if (rowsReturned >= totalRowCount) { + return false; + } + + recordReader.nextBatch(batch); + int batchSize = batch.size; + if (batchSize == 0) { + return false; + } + rowsReturned += batchSize; + for (WritableColumnVector vector : columnVectors) { + vector.reset(); + } + columnarBatch.setNumRows(batchSize); + int i = 0; + while (i < requiredSchema.length()) { + StructField field = requiredSchema.fields()[i]; + WritableColumnVector toColumn = columnVectors[i]; + + if (requestedColIds[i] < 0) { + toColumn.appendNulls(batchSize); + } else { + ColumnVector fromColumn = batch.cols[requestedColIds[i]]; + + if (fromColumn.isRepeating) { + if (fromColumn.isNull[0]) { + toColumn.appendNulls(batchSize); + } else { + DataType type = field.dataType(); + if (type instanceof BooleanType) { + toColumn.appendBooleans(batchSize, ((LongColumnVector)fromColumn).vector[0] == 1); + } else if (type instanceof ByteType) { + toColumn.appendBytes(batchSize, (byte)((LongColumnVector)fromColumn).vector[0]); + } else if (type instanceof ShortType) { + toColumn.appendShorts(batchSize, (short)((LongColumnVector)fromColumn).vector[0]); + } else if (type instanceof IntegerType || type instanceof DateType) { + toColumn.appendInts(batchSize, (int)((LongColumnVector)fromColumn).vector[0]); + } else if (type instanceof LongType) { + toColumn.appendLongs(batchSize, ((LongColumnVector)fromColumn).vector[0]); + } else if (type instanceof TimestampType) { + toColumn.appendLongs(batchSize, fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0)); + } else if (type instanceof FloatType) { + toColumn.appendFloats(batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]); + } else if (type instanceof DoubleType) { + toColumn.appendDoubles(batchSize, ((DoubleColumnVector)fromColumn).vector[0]); + } else if (type instanceof StringType || type instanceof BinaryType) { + BytesColumnVector data = (BytesColumnVector)fromColumn; + int index = 0; + while (index < batchSize) { + toColumn.appendByteArray(data.vector[0], data.start[0], data.length[0]); + index += 1; + } + } else if (type instanceof DecimalType) { + DecimalType decimalType = (DecimalType)type; + appendDecimalWritable( + toColumn, + decimalType.precision(), + decimalType.scale(), + ((DecimalColumnVector)fromColumn).vector[0]); + } else { + throw new UnsupportedOperationException("Unsupported Data Type: " + type); + } + } + } else if (fromColumn.noNulls) { + DataType type = field.dataType(); + if (type instanceof BooleanType) { + long[] data = ((LongColumnVector)fromColumn).vector; + int index = 0; + while (index < batchSize) { + toColumn.appendBoolean(data[index] == 1); + index += 1; + } + } else if (type instanceof ByteType) { + long[] data = ((LongColumnVector)fromColumn).vector; + int index = 0; + while (index < batchSize) { + toColumn.appendByte((byte)data[index]); + index += 1; + } + } else if (type instanceof ShortType) { + long[] data = ((LongColumnVector)fromColumn).vector; + int index = 0; + while (index < batchSize) { + toColumn.appendShort((short)data[index]); + index += 1; + } + } else if (type instanceof IntegerType || type instanceof DateType) { + long[] data = ((LongColumnVector)fromColumn).vector; + int index = 0; + while (index < batchSize) { + toColumn.appendInt((int)data[index]); + index += 1; + } + } else if (type instanceof LongType) { + toColumn.appendLongs(batchSize, ((LongColumnVector)fromColumn).vector, 0); + } else if (type instanceof TimestampType) { + TimestampColumnVector data = ((TimestampColumnVector)fromColumn); + int index = 0; + while (index < batchSize) { + toColumn.appendLong(fromTimestampColumnVector(data, index)); + index += 1; + } + } else if (type instanceof FloatType) { + double[] data = ((DoubleColumnVector)fromColumn).vector; + int index = 0; + while (index < batchSize) { + toColumn.appendFloat((float)data[index]); + index += 1; + } + } else if (type instanceof DoubleType) { + toColumn.appendDoubles(batchSize, ((DoubleColumnVector)fromColumn).vector, 0); + } else if (type instanceof StringType || type instanceof BinaryType) { + BytesColumnVector data = ((BytesColumnVector)fromColumn); + int index = 0; + while (index < batchSize) { + toColumn.appendByteArray(data.vector[index], data.start[index], data.length[index]); + index += 1; + } + } else if (type instanceof DecimalType) { + DecimalType decimalType = (DecimalType)type; + DecimalColumnVector data = ((DecimalColumnVector)fromColumn); + int index = 0; + while (index < batchSize) { + appendDecimalWritable( + toColumn, + decimalType.precision(), + decimalType.scale(), + data.vector[index]); + index += 1; + } + } else { + throw new UnsupportedOperationException("Unsupported Data Type: " + type); + } + } else { + DataType type = field.dataType(); + if (type instanceof BooleanType) { + long[] vector = ((LongColumnVector)fromColumn).vector; + int index = 0; + while (index < batchSize) { + if (fromColumn.isNull[index]) { + toColumn.appendNull(); + } else { + toColumn.appendBoolean(vector[index] == 1); + } + index += 1; + } + } else if (type instanceof ByteType) { + long[] vector = ((LongColumnVector)fromColumn).vector; + int index = 0; + while (index < batchSize) { + if (fromColumn.isNull[index]) { + toColumn.appendNull(); + } else { + toColumn.appendByte((byte)vector[index]); + } + index += 1; + } + } else if (type instanceof ShortType) { + long[] vector = ((LongColumnVector)fromColumn).vector; + int index = 0; + while (index < batchSize) { + if (fromColumn.isNull[index]) { + toColumn.appendNull(); + } else { + toColumn.appendShort((short)vector[index]); + } + index += 1; + } + } else if (type instanceof IntegerType || type instanceof DateType) { + long[] vector = ((LongColumnVector)fromColumn).vector; + int index = 0; + while (index < batchSize) { + if (fromColumn.isNull[index]) { + toColumn.appendNull(); + } else { + toColumn.appendInt((int)vector[index]); + } + index += 1; + } + } else if (type instanceof LongType) { + long[] vector = ((LongColumnVector)fromColumn).vector; + int index = 0; + while (index < batchSize) { + if (fromColumn.isNull[index]) { + toColumn.appendNull(); + } else { + toColumn.appendLong(vector[index]); + } + index += 1; + } + } else if (type instanceof TimestampType) { + TimestampColumnVector vector = ((TimestampColumnVector)fromColumn); + int index = 0; + while (index < batchSize) { + if (fromColumn.isNull[index]) { + toColumn.appendNull(); + } else { + toColumn.appendLong(fromTimestampColumnVector(vector, index)); + } + index += 1; + } + } else if (type instanceof FloatType) { + double[] vector = ((DoubleColumnVector)fromColumn).vector; + int index = 0; + while (index < batchSize) { + if (fromColumn.isNull[index]) { + toColumn.appendNull(); + } else { + toColumn.appendFloat((float)vector[index]); + } + index += 1; + } + } else if (type instanceof DoubleType) { + double[] vector = ((DoubleColumnVector)fromColumn).vector; + int index = 0; + while (index < batchSize) { + if (fromColumn.isNull[index]) { + toColumn.appendNull(); + } else { + toColumn.appendDouble(vector[index]); + } + index += 1; + } + } else if (type instanceof StringType || type instanceof BinaryType) { + BytesColumnVector vector = (BytesColumnVector)fromColumn; + int index = 0; + while (index < batchSize) { + if (fromColumn.isNull[index]) { + toColumn.appendNull(); + } else { + toColumn.appendByteArray( + vector.vector[index], vector.start[index], vector.length[index]); + } + index += 1; + } + } else if (type instanceof DecimalType) { + DecimalType decimalType = (DecimalType)type; + HiveDecimalWritable[] vector = ((DecimalColumnVector)fromColumn).vector; + int index = 0; + while (index < batchSize) { + if (fromColumn.isNull[index]) { + toColumn.appendNull(); + } else { + appendDecimalWritable( + toColumn, + decimalType.precision(), + decimalType.scale(), + vector[index]); + } + index += 1; + } + } else { + throw new UnsupportedOperationException("Unsupported Data Type: " + type); + } + } + } + i += 1; + } + return true; + } + + /** + * Default memory mode for ColumnarBatch. + */ + public static final MemoryMode DEFAULT_MEMORY_MODE = MemoryMode.ON_HEAP; + + /** + * The default size of batch. We use this value for both ORC and Spark consistently + * because they have different default values like the following. + * + * - ORC's VectorizedRowBatch.DEFAULT_SIZE = 1024 + * - Spark's ColumnarBatch.DEFAULT_BATCH_SIZE = 4 * 1024 + */ + public static final int DEFAULT_SIZE = 4 * 1024; + + /** + * Returns the number of micros since epoch from an element of TimestampColumnVector. + */ + private static long fromTimestampColumnVector(TimestampColumnVector vector, int index) { + return vector.time[index] * 1000L + vector.nanos[index] / 1000L; + } + + /** + * Append a `HiveDecimalWritable` to a `WritableColumnVector`. + */ + private static void appendDecimalWritable( + WritableColumnVector toColumn, + int precision, + int scale, + HiveDecimalWritable decimalWritable) { + HiveDecimal decimal = decimalWritable.getHiveDecimal(); + Decimal value = + Decimal.apply(decimal.bigDecimalValue(), decimal.precision(), decimal.scale()); + value.changePrecision(precision, scale); + + if (precision <= Decimal.MAX_INT_DIGITS()) { + toColumn.appendInt((int) value.toUnscaledLong()); + } else if (precision <= Decimal.MAX_LONG_DIGITS()) { + toColumn.appendLong(value.toUnscaledLong()); + } else { + byte[] bytes = value.toJavaBigDecimal().unscaledValue().toByteArray(); + toColumn.appendByteArray(bytes, 0, bytes.length); + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala index acb8066d0b7d6..12f80a076bc2b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala @@ -148,6 +148,7 @@ class OrcFileFormat val resultSchema = StructType(requiredSchema.fields ++ partitionSchema.fields) val enableVectorizedReader = supportBatch(sparkSession, resultSchema) + val enableVectorizedJavaReader = sparkSession.sessionState.conf.orcVectorizedJavaReaderEnabled val broadcastedConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) @@ -180,14 +181,27 @@ class OrcFileFormat val taskAttemptContext = new TaskAttemptContextImpl(taskConf, attemptId) if (enableVectorizedReader) { - val batchReader = new OrcColumnarBatchReader - batchReader.initialize(fileSplit, taskAttemptContext) - batchReader.setRequiredSchema( - reader.getSchema, - requestedColIds, - requiredSchema, - partitionSchema, - file.partitionValues) + val batchReader = if (enableVectorizedJavaReader) { + val vectorizedReader = new JavaOrcColumnarBatchReader + vectorizedReader.initialize(fileSplit, taskAttemptContext) + vectorizedReader.setRequiredSchema( + reader.getSchema, + requestedColIds, + requiredSchema, + partitionSchema, + file.partitionValues) + vectorizedReader + } else { + val vectorizedReader = new OrcColumnarBatchReader + vectorizedReader.initialize(fileSplit, taskAttemptContext) + vectorizedReader.setRequiredSchema( + reader.getSchema, + requestedColIds, + requiredSchema, + partitionSchema, + file.partitionValues) + vectorizedReader + } val iter = new RecordReaderIterator(batchReader) Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => iter.close())) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala index 60686acc89fba..2f20f579afe2b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala @@ -102,7 +102,15 @@ object OrcReadBenchmark { } sqlBenchmark.addCase("Native ORC Vectorized") { _ => - spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() + withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") { + spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() + } + } + + sqlBenchmark.addCase("Native ORC Vectorized (Java)") { _ => + withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") { + spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() + } } sqlBenchmark.addCase("Hive built-in ORC") { _ => @@ -115,39 +123,45 @@ object OrcReadBenchmark { SQL Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1221 / 1268 12.9 77.6 1.0X - Native ORC Vectorized 164 / 174 96.1 10.4 7.5X - Hive built-in ORC 1440 / 1449 10.9 91.5 0.8X + Native ORC MR 1226 / 1227 12.8 78.0 1.0X + Native ORC Vectorized 165 / 179 95.3 10.5 7.4X + Native ORC Vectorized (Java) 170 / 179 92.8 10.8 7.2X + Hive built-in ORC 1423 / 1425 11.1 90.5 0.9X SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1283 / 1322 12.3 81.6 1.0X - Native ORC Vectorized 166 / 180 94.9 10.5 7.7X - Hive built-in ORC 1619 / 1645 9.7 102.9 0.8X + Native ORC MR 1193 / 1284 13.2 75.9 1.0X + Native ORC Vectorized 164 / 170 95.9 10.4 7.3X + Native ORC Vectorized (Java) 175 / 183 89.6 11.2 6.8X + Hive built-in ORC 1685 / 1701 9.3 107.1 0.7X SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1313 / 1425 12.0 83.4 1.0X - Native ORC Vectorized 231 / 239 68.0 14.7 5.7X - Hive built-in ORC 2048 / 2062 7.7 130.2 0.6X + Native ORC MR 1362 / 1375 11.6 86.6 1.0X + Native ORC Vectorized 273 / 283 57.7 17.3 5.0X + Native ORC Vectorized (Java) 242 / 249 65.1 15.4 5.6X + Hive built-in ORC 1894 / 1905 8.3 120.4 0.7X SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1364 / 1366 11.5 86.7 1.0X - Native ORC Vectorized 291 / 297 54.1 18.5 4.7X - Hive built-in ORC 2056 / 2058 7.6 130.7 0.7X + Native ORC MR 1349 / 1413 11.7 85.7 1.0X + Native ORC Vectorized 290 / 299 54.2 18.4 4.6X + Native ORC Vectorized (Java) 288 / 297 54.7 18.3 4.7X + Hive built-in ORC 1917 / 1928 8.2 121.9 0.7X SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1528 / 1563 10.3 97.2 1.0X - Native ORC Vectorized 343 / 358 45.9 21.8 4.5X - Hive built-in ORC 1999 / 2031 7.9 127.1 0.8X + Native ORC MR 1393 / 1420 11.3 88.6 1.0X + Native ORC Vectorized 383 / 394 41.0 24.4 3.6X + Native ORC Vectorized (Java) 346 / 352 45.5 22.0 4.0X + Hive built-in ORC 1980 / 2016 7.9 125.9 0.7X SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1483 / 1514 10.6 94.3 1.0X - Native ORC Vectorized 406 / 420 38.7 25.8 3.6X - Hive built-in ORC 2147 / 2160 7.3 136.5 0.7X + Native ORC MR 1472 / 1489 10.7 93.6 1.0X + Native ORC Vectorized 389 / 400 40.5 24.7 3.8X + Native ORC Vectorized (Java) 393 / 400 40.0 25.0 3.7X + Hive built-in ORC 2140 / 2144 7.3 136.1 0.7X */ sqlBenchmark.run() } @@ -173,7 +187,15 @@ object OrcReadBenchmark { } benchmark.addCase("Native ORC Vectorized") { _ => - spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect() + withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") { + spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect() + } + } + + benchmark.addCase("Native ORC Vectorized (Java)") { _ => + withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") { + spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect() + } } benchmark.addCase("Hive built-in ORC") { _ => @@ -186,9 +208,10 @@ object OrcReadBenchmark { Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2631 / 2700 4.0 250.9 1.0X - Native ORC Vectorized 1358 / 1363 7.7 129.5 1.9X - Hive built-in ORC 3602 / 3645 2.9 343.5 0.7X + Native ORC MR 2585 / 2656 4.1 246.5 1.0X + Native ORC Vectorized 1370 / 1380 7.7 130.6 1.9X + Native ORC Vectorized (Java) 1385 / 1394 7.6 132.1 1.9X + Hive built-in ORC 3832 / 3906 2.7 365.4 0.7X */ benchmark.run() } @@ -212,7 +235,15 @@ object OrcReadBenchmark { } benchmark.addCase("Read data column - Native ORC Vectorized") { _ => - spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() + withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") { + spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() + } + } + + benchmark.addCase("Read data column - Native ORC Vectorized (Java)") { _ => + withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") { + spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() + } } benchmark.addCase("Read data column - Hive built-in ORC") { _ => @@ -226,7 +257,15 @@ object OrcReadBenchmark { } benchmark.addCase("Read partition column - Native ORC Vectorized") { _ => - spark.sql("SELECT sum(p) FROM nativeOrcTable").collect() + withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") { + spark.sql("SELECT sum(p) FROM nativeOrcTable").collect() + } + } + + benchmark.addCase("Read partition column - Native ORC Vectorized (Java)") { _ => + withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") { + spark.sql("SELECT sum(p) FROM nativeOrcTable").collect() + } } benchmark.addCase("Read partition column - Hive built-in ORC") { _ => @@ -240,7 +279,15 @@ object OrcReadBenchmark { } benchmark.addCase("Read both columns - Native ORC Vectorized") { _ => - spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect() + withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") { + spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect() + } + } + + benchmark.addCase("Read both columns - Native ORC Vectorized (Java)") { _ => + withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") { + spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect() + } } benchmark.addCase("Read both columns - Hive built-in ORC") { _ => @@ -251,17 +298,20 @@ object OrcReadBenchmark { Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz - Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - Read data column - Native ORC MR 1486 / 1502 10.6 94.5 1.0X - Read data column - Native ORC Vectorized 293 / 301 53.7 18.6 5.1X - Read data column - Hive built-in ORC 2071 / 2078 7.6 131.6 0.7X - Read partition column - Native ORC MR 1048 / 1054 15.0 66.6 1.4X - Read partition column - Native ORC Vectorized 55 / 57 287.4 3.5 27.2X - Read partition column - Hive built-in ORC 1268 / 1280 12.4 80.6 1.2X - Read both columns - Native ORC MR 1547 / 1562 10.2 98.3 1.0X - Read both columns - Native ORC Vectorized 330 / 334 47.6 21.0 4.5X - Read both columns - Hive built-in ORC 2114 / 2122 7.4 134.4 0.7X + Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------------ + Read data column - Native ORC MR 1531 / 1555 10.3 97.4 1.0X + Read data column - Native ORC Vectorized 294 / 299 53.4 18.7 5.2X + Read data column - Native ORC Vectorized (Java) 293 / 298 53.6 18.6 5.2X + Read data column - Hive built-in ORC 2083 / 2104 7.5 132.5 0.7X + Read partition column - Native ORC MR 1047 / 1048 15.0 66.6 1.5X + Read partition column - Native ORC Vectorized 53 / 55 296.8 3.4 28.9X + Read partition column - Native ORC Vectorized (Java) 52 / 53 303.8 3.3 29.6X + Read partition column - Hive built-in ORC 1301 / 1312 12.1 82.7 1.2X + Read both columns - Native ORC MR 1597 / 1606 9.8 101.6 1.0X + Read both columns - Native ORC Vectorized 329 / 336 47.8 20.9 4.7X + Read both columns - Native ORC Vectorized (Java) 333 / 338 47.2 21.2 4.6X + Read both columns - Hive built-in ORC 2100 / 2118 7.5 133.5 0.7X */ benchmark.run() } @@ -284,7 +334,15 @@ object OrcReadBenchmark { } benchmark.addCase("Native ORC Vectorized") { _ => - spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect() + withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") { + spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect() + } + } + + benchmark.addCase("Native ORC Vectorized (Java)") { _ => + withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") { + spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect() + } } benchmark.addCase("Hive built-in ORC") { _ => @@ -297,9 +355,10 @@ object OrcReadBenchmark { String Dictionary: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1310 / 1317 8.0 124.9 1.0X - Native ORC Vectorized 370 / 379 28.4 35.3 3.5X - Hive built-in ORC 1938 / 1956 5.4 184.9 0.7X + Native ORC MR 1313 / 1333 8.0 125.3 1.0X + Native ORC Vectorized 371 / 382 28.3 35.4 3.5X + Native ORC Vectorized (Java) 372 / 382 28.2 35.5 3.5X + Hive built-in ORC 1945 / 1989 5.4 185.5 0.7X */ benchmark.run() } @@ -327,8 +386,17 @@ object OrcReadBenchmark { } benchmark.addCase(s"Native ORC Vectorized ($fractionOfNulls%)") { _ => - spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " + - "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() + withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") { + spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " + + "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() + } + } + + benchmark.addCase(s"Native ORC Vectorized ($fractionOfNulls%) (Java)") { _ => + withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") { + spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " + + "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() + } } benchmark.addCase(s"Hive built-in ORC ($fractionOfNulls%)") { _ => @@ -342,21 +410,24 @@ object OrcReadBenchmark { String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR (0.0%) 2505 / 2527 4.2 238.9 1.0X - Native ORC Vectorized (0.0%) 1052 / 1059 10.0 100.3 2.4X - Hive built-in ORC (0.0%) 3855 / 3918 2.7 367.7 0.6X + Native ORC MR (0.0%) 2578 / 2589 4.1 245.8 1.0X + Native ORC Vectorized (0.0%) 1055 / 1058 9.9 100.6 2.4X + Native ORC Vectorized (0.0%) (Java) 1071 / 1089 9.8 102.2 2.4X + Hive built-in ORC (0.0%) 3954 / 3955 2.7 377.0 0.7X String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR (0.5%) 2398 / 2413 4.4 228.7 1.0X - Native ORC Vectorized (0.5%) 1263 / 1266 8.3 120.4 1.9X - Hive built-in ORC (0.5%) 2919 / 2937 3.6 278.4 0.8X + Native ORC MR (0.5%) 2410 / 2414 4.4 229.8 1.0X + Native ORC Vectorized (0.5%) 1248 / 1250 8.4 119.0 1.9X + Native ORC Vectorized (0.5%) (Java) 1248 / 1259 8.4 119.0 1.9X + Hive built-in ORC (0.5%) 2887 / 2927 3.6 275.3 0.8X String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR (0.95%) 1296 / 1318 8.1 123.6 1.0X - Native ORC Vectorized (0.95%) 464 / 468 22.6 44.2 2.8X - Hive built-in ORC (0.95%) 1592 / 1607 6.6 151.8 0.8X + Native ORC MR (0.95%) 1288 / 1311 8.1 122.9 1.0X + Native ORC Vectorized (0.95%) 455 / 463 23.0 43.4 2.8X + Native ORC Vectorized (0.95%) (Java) 461 / 476 22.7 44.0 2.8X + Hive built-in ORC (0.95%) 1658 / 1679 6.3 158.1 0.8X */ benchmark.run() } @@ -383,7 +454,15 @@ object OrcReadBenchmark { } sqlBenchmark.addCase("Native ORC Vectorized") { _ => - spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect() + withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") { + spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect() + } + } + + sqlBenchmark.addCase("Native ORC Vectorized (Java)") { _ => + withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") { + spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect() + } } sqlBenchmark.addCase("Hive built-in ORC") { _ => @@ -396,21 +475,24 @@ object OrcReadBenchmark { SQL Single Column Scan from wide table (100 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1068 / 1072 1.0 1018.7 1.0X - Native ORC Vectorized 94 / 99 11.2 89.5 11.4X - Hive built-in ORC 372 / 384 2.8 355.0 2.9X + Native ORC MR 1076 / 1076 1.0 1026.1 1.0X + Native ORC Vectorized 90 / 98 11.7 85.6 12.0X + Native ORC Vectorized (Java) 91 / 97 11.6 86.4 11.9X + Hive built-in ORC 406 / 408 2.6 386.9 2.7X SQL Single Column Scan from wide table (200 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2154 / 2162 0.5 2054.3 1.0X - Native ORC Vectorized 158 / 164 6.6 150.9 13.6X - Hive built-in ORC 601 / 606 1.7 573.0 3.6X + Native ORC MR 2152 / 2171 0.5 2051.9 1.0X + Native ORC Vectorized 155 / 164 6.7 148.2 13.9X + Native ORC Vectorized (Java) 155 / 162 6.8 147.5 13.9X + Hive built-in ORC 628 / 633 1.7 598.9 3.4X SQL Single Column Scan from wide table (300 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 3217 / 3226 0.3 3068.3 1.0X - Native ORC Vectorized 270 / 277 3.9 257.7 11.9X - Hive built-in ORC 840 / 853 1.2 801.2 3.8X + Native ORC MR 3219 / 3233 0.3 3070.2 1.0X + Native ORC Vectorized 268 / 279 3.9 255.5 12.0X + Native ORC Vectorized (Java) 267 / 275 3.9 254.7 12.1X + Hive built-in ORC 892 / 898 1.2 850.5 3.6X */ sqlBenchmark.run() } From 3a0702ae0b31f762c9f3da06d267a02ec8d1a23b Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 7 Jan 2018 23:40:17 -0800 Subject: [PATCH 11/23] Use putX instead of appendX and address other comments. --- .../orc/JavaOrcColumnarBatchReader.java | 283 ++++++++---------- .../orc/OrcColumnarBatchReader.scala | 138 +++++---- .../datasources/orc/OrcFileFormat.scala | 11 +- .../spark/sql/hive/orc/OrcReadBenchmark.scala | 138 ++++----- 4 files changed, 290 insertions(+), 280 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java index 790685d43413b..9eeb635111210 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java @@ -45,54 +45,47 @@ /** * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch. - * After creating, `initialize` and `setRequiredSchema` should be called sequentially. + * After creating, `initialize` and `initBatch` should be called sequentially. */ public class JavaOrcColumnarBatchReader extends RecordReader { - /** - * ORC File Reader. - */ + // ORC File Reader private Reader reader; - /** - * Vectorized Row Batch. - */ + // Vectorized ORC Row Batch private VectorizedRowBatch batch; /** - * Requested Column IDs. + * The column IDs of the physical ORC file schema which are required by this reader. + * -1 means this required column doesn't exist in the ORC file. */ private int[] requestedColIds; - /** - * Record reader from row batch. - */ + // Record reader from ORC row batch. private org.apache.orc.RecordReader recordReader; - /** - * Required Schema. - */ private StructType requiredSchema; - /** - * ColumnarBatch for vectorized execution by whole-stage codegen. - */ + // The result columnar batch for vectorized execution by whole-stage codegen. private ColumnarBatch columnarBatch; - /** - * Writable column vectors of ColumnarBatch. - */ + // Writable column vectors of the result columnar batch. private WritableColumnVector[] columnVectors; - /** - * The number of rows read and considered to be returned. - */ + // The number of rows read and considered to be returned. private long rowsReturned = 0L; + private long totalRowCount = 0L; + /** - * Total number of rows. + * The memory mode of the columnarBatch */ - private long totalRowCount = 0L; + private final MemoryMode MEMORY_MODE; + + public JavaOrcColumnarBatchReader(boolean useOffHeap) { + MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP; + } + @Override public Void getCurrentKey() throws IOException, InterruptedException { @@ -128,7 +121,7 @@ public void close() throws IOException { /** * Initialize ORC file reader and batch record reader. - * Please note that `setRequiredSchema` is needed to be called after this. + * Please note that `initBatch` is needed to be called after this. */ @Override public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) @@ -142,32 +135,34 @@ public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptCont .filesystem(fileSplit.getPath().getFileSystem(conf))); Reader.Options options = - OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength()); + OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength()); recordReader = reader.rows(options); totalRowCount = reader.getNumberOfRows(); } /** - * Set required schema and partition information. + * Initialize columnar batch by setting required schema and partition information. * With this information, this creates ColumnarBatch with the full schema. */ - public void setRequiredSchema( - TypeDescription orcSchema, - int[] requestedColIds, - StructType requiredSchema, - StructType partitionSchema, - InternalRow partitionValues) { + public void initBatch( + TypeDescription orcSchema, + int[] requestedColIds, + StructType requiredSchema, + StructType partitionSchema, + InternalRow partitionValues) { batch = orcSchema.createRowBatch(DEFAULT_SIZE); assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`. StructType resultSchema = new StructType(requiredSchema.fields()); - for (StructField f : partitionSchema.fields()) + for (StructField f : partitionSchema.fields()) { resultSchema = resultSchema.add(f); + } this.requiredSchema = requiredSchema; this.requestedColIds = requestedColIds; + assert(requiredSchema.length() == requestedColIds.length); int capacity = DEFAULT_SIZE; - if (DEFAULT_MEMORY_MODE == MemoryMode.OFF_HEAP) { + if (MEMORY_MODE == MemoryMode.OFF_HEAP) { columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema); } else { columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema); @@ -184,7 +179,7 @@ public void setRequiredSchema( // Initialize the missing columns once. for (int i = 0; i < requiredSchema.length(); i++) { - if (requestedColIds[i] < 0) { + if (requestedColIds[i] == -1) { columnVectors[i].putNulls(0, columnarBatch.capacity()); columnVectors[i].setIsConstant(); } @@ -210,48 +205,45 @@ private boolean nextBatch() throws IOException { vector.reset(); } columnarBatch.setNumRows(batchSize); - int i = 0; - while (i < requiredSchema.length()) { + for (int i = 0; i < requiredSchema.length(); i++) { StructField field = requiredSchema.fields()[i]; WritableColumnVector toColumn = columnVectors[i]; - if (requestedColIds[i] < 0) { - toColumn.appendNulls(batchSize); - } else { + if (requestedColIds[i] >= 0) { ColumnVector fromColumn = batch.cols[requestedColIds[i]]; + toColumn.reserve(batchSize); if (fromColumn.isRepeating) { if (fromColumn.isNull[0]) { - toColumn.appendNulls(batchSize); + toColumn.putNulls(0, batchSize); } else { DataType type = field.dataType(); if (type instanceof BooleanType) { - toColumn.appendBooleans(batchSize, ((LongColumnVector)fromColumn).vector[0] == 1); + toColumn.putBooleans(0, batchSize, ((LongColumnVector)fromColumn).vector[0] == 1); } else if (type instanceof ByteType) { - toColumn.appendBytes(batchSize, (byte)((LongColumnVector)fromColumn).vector[0]); + toColumn.putBytes(0, batchSize, (byte)((LongColumnVector)fromColumn).vector[0]); } else if (type instanceof ShortType) { - toColumn.appendShorts(batchSize, (short)((LongColumnVector)fromColumn).vector[0]); + toColumn.putShorts(0, batchSize, (short)((LongColumnVector)fromColumn).vector[0]); } else if (type instanceof IntegerType || type instanceof DateType) { - toColumn.appendInts(batchSize, (int)((LongColumnVector)fromColumn).vector[0]); + toColumn.putInts(0, batchSize, (int)((LongColumnVector)fromColumn).vector[0]); } else if (type instanceof LongType) { - toColumn.appendLongs(batchSize, ((LongColumnVector)fromColumn).vector[0]); + toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector[0]); } else if (type instanceof TimestampType) { - toColumn.appendLongs(batchSize, fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0)); + toColumn.putLongs(0, batchSize, fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0)); } else if (type instanceof FloatType) { - toColumn.appendFloats(batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]); + toColumn.putFloats(0, batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]); } else if (type instanceof DoubleType) { - toColumn.appendDoubles(batchSize, ((DoubleColumnVector)fromColumn).vector[0]); + toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]); } else if (type instanceof StringType || type instanceof BinaryType) { BytesColumnVector data = (BytesColumnVector)fromColumn; - int index = 0; - while (index < batchSize) { - toColumn.appendByteArray(data.vector[0], data.start[0], data.length[0]); - index += 1; + for (int index = 0; index < batchSize; index++) { + toColumn.putByteArray(index, data.vector[0], data.start[0], data.length[0]); } } else if (type instanceof DecimalType) { DecimalType decimalType = (DecimalType)type; - appendDecimalWritable( + putDecimalWritables( toColumn, + batchSize, decimalType.precision(), decimalType.scale(), ((DecimalColumnVector)fromColumn).vector[0]); @@ -263,68 +255,53 @@ private boolean nextBatch() throws IOException { DataType type = field.dataType(); if (type instanceof BooleanType) { long[] data = ((LongColumnVector)fromColumn).vector; - int index = 0; - while (index < batchSize) { - toColumn.appendBoolean(data[index] == 1); - index += 1; + for (int index = 0; index < batchSize; index++) { + toColumn.putBoolean(index, data[index] == 1); } } else if (type instanceof ByteType) { long[] data = ((LongColumnVector)fromColumn).vector; - int index = 0; - while (index < batchSize) { - toColumn.appendByte((byte)data[index]); - index += 1; + for (int index = 0; index < batchSize; index++) { + toColumn.putByte(index, (byte)data[index]); } } else if (type instanceof ShortType) { long[] data = ((LongColumnVector)fromColumn).vector; - int index = 0; - while (index < batchSize) { - toColumn.appendShort((short)data[index]); - index += 1; + for (int index = 0; index < batchSize; index++) { + toColumn.putShort(index, (short)data[index]); } } else if (type instanceof IntegerType || type instanceof DateType) { long[] data = ((LongColumnVector)fromColumn).vector; - int index = 0; - while (index < batchSize) { - toColumn.appendInt((int)data[index]); - index += 1; + for (int index = 0; index < batchSize; index++) { + toColumn.putInt(index, (int)data[index]); } } else if (type instanceof LongType) { - toColumn.appendLongs(batchSize, ((LongColumnVector)fromColumn).vector, 0); + toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector, 0); } else if (type instanceof TimestampType) { TimestampColumnVector data = ((TimestampColumnVector)fromColumn); - int index = 0; - while (index < batchSize) { - toColumn.appendLong(fromTimestampColumnVector(data, index)); - index += 1; + for (int index = 0; index < batchSize; index++) { + toColumn.putLong(index, fromTimestampColumnVector(data, index)); } } else if (type instanceof FloatType) { double[] data = ((DoubleColumnVector)fromColumn).vector; - int index = 0; - while (index < batchSize) { - toColumn.appendFloat((float)data[index]); - index += 1; + for (int index = 0; index < batchSize; index++) { + toColumn.putFloat(index, (float)data[index]); } } else if (type instanceof DoubleType) { - toColumn.appendDoubles(batchSize, ((DoubleColumnVector)fromColumn).vector, 0); + toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0); } else if (type instanceof StringType || type instanceof BinaryType) { BytesColumnVector data = ((BytesColumnVector)fromColumn); - int index = 0; - while (index < batchSize) { - toColumn.appendByteArray(data.vector[index], data.start[index], data.length[index]); - index += 1; + for (int index = 0; index < batchSize; index++) { + toColumn.putByteArray(index, data.vector[index], data.start[index], data.length[index]); } } else if (type instanceof DecimalType) { DecimalType decimalType = (DecimalType)type; DecimalColumnVector data = ((DecimalColumnVector)fromColumn); - int index = 0; - while (index < batchSize) { - appendDecimalWritable( + for (int index = 0; index < batchSize; index++) { + putDecimalWritable( toColumn, + index, decimalType.precision(), decimalType.scale(), data.vector[index]); - index += 1; } } else { throw new UnsupportedOperationException("Unsupported Data Type: " + type); @@ -333,135 +310,110 @@ private boolean nextBatch() throws IOException { DataType type = field.dataType(); if (type instanceof BooleanType) { long[] vector = ((LongColumnVector)fromColumn).vector; - int index = 0; - while (index < batchSize) { + for (int index = 0; index < batchSize; index++) { if (fromColumn.isNull[index]) { - toColumn.appendNull(); + toColumn.putNull(index); } else { - toColumn.appendBoolean(vector[index] == 1); + toColumn.putBoolean(index, vector[index] == 1); } - index += 1; } } else if (type instanceof ByteType) { long[] vector = ((LongColumnVector)fromColumn).vector; - int index = 0; - while (index < batchSize) { + for (int index = 0; index < batchSize; index++) { if (fromColumn.isNull[index]) { - toColumn.appendNull(); + toColumn.putNull(index); } else { - toColumn.appendByte((byte)vector[index]); + toColumn.putByte(index, (byte)vector[index]); } - index += 1; } } else if (type instanceof ShortType) { long[] vector = ((LongColumnVector)fromColumn).vector; - int index = 0; - while (index < batchSize) { + for (int index = 0; index < batchSize; index++) { if (fromColumn.isNull[index]) { - toColumn.appendNull(); + toColumn.putNull(index); } else { - toColumn.appendShort((short)vector[index]); + toColumn.putShort(index, (short)vector[index]); } - index += 1; } } else if (type instanceof IntegerType || type instanceof DateType) { long[] vector = ((LongColumnVector)fromColumn).vector; - int index = 0; - while (index < batchSize) { + for (int index = 0; index < batchSize; index++) { if (fromColumn.isNull[index]) { - toColumn.appendNull(); + toColumn.putNull(index); } else { - toColumn.appendInt((int)vector[index]); + toColumn.putInt(index, (int)vector[index]); } - index += 1; } } else if (type instanceof LongType) { long[] vector = ((LongColumnVector)fromColumn).vector; - int index = 0; - while (index < batchSize) { + for (int index = 0; index < batchSize; index++) { if (fromColumn.isNull[index]) { - toColumn.appendNull(); + toColumn.putNull(index); } else { - toColumn.appendLong(vector[index]); + toColumn.putLong(index, vector[index]); } - index += 1; } } else if (type instanceof TimestampType) { TimestampColumnVector vector = ((TimestampColumnVector)fromColumn); - int index = 0; - while (index < batchSize) { + for (int index = 0; index < batchSize; index++) { if (fromColumn.isNull[index]) { - toColumn.appendNull(); + toColumn.putNull(index); } else { - toColumn.appendLong(fromTimestampColumnVector(vector, index)); + toColumn.putLong(index, fromTimestampColumnVector(vector, index)); } - index += 1; } } else if (type instanceof FloatType) { double[] vector = ((DoubleColumnVector)fromColumn).vector; - int index = 0; - while (index < batchSize) { + for (int index = 0; index < batchSize; index++) { if (fromColumn.isNull[index]) { - toColumn.appendNull(); + toColumn.putNull(index); } else { - toColumn.appendFloat((float)vector[index]); + toColumn.putFloat(index, (float)vector[index]); } - index += 1; } } else if (type instanceof DoubleType) { double[] vector = ((DoubleColumnVector)fromColumn).vector; - int index = 0; - while (index < batchSize) { + for (int index = 0; index < batchSize; index++) { if (fromColumn.isNull[index]) { - toColumn.appendNull(); + toColumn.putNull(index); } else { - toColumn.appendDouble(vector[index]); + toColumn.putDouble(index, vector[index]); } - index += 1; } } else if (type instanceof StringType || type instanceof BinaryType) { BytesColumnVector vector = (BytesColumnVector)fromColumn; - int index = 0; - while (index < batchSize) { + for (int index = 0; index < batchSize; index++) { if (fromColumn.isNull[index]) { - toColumn.appendNull(); + toColumn.putNull(index); } else { - toColumn.appendByteArray( - vector.vector[index], vector.start[index], vector.length[index]); + toColumn.putByteArray( + index, vector.vector[index], vector.start[index], vector.length[index]); } - index += 1; } } else if (type instanceof DecimalType) { DecimalType decimalType = (DecimalType)type; HiveDecimalWritable[] vector = ((DecimalColumnVector)fromColumn).vector; - int index = 0; - while (index < batchSize) { + for (int index = 0; index < batchSize; index++) { if (fromColumn.isNull[index]) { - toColumn.appendNull(); + toColumn.putNull(index); } else { - appendDecimalWritable( + putDecimalWritable( toColumn, + index, decimalType.precision(), decimalType.scale(), vector[index]); } - index += 1; } } else { throw new UnsupportedOperationException("Unsupported Data Type: " + type); } } } - i += 1; } return true; } - /** - * Default memory mode for ColumnarBatch. - */ - public static final MemoryMode DEFAULT_MEMORY_MODE = MemoryMode.ON_HEAP; - /** * The default size of batch. We use this value for both ORC and Spark consistently * because they have different default values like the following. @@ -479,10 +431,11 @@ private static long fromTimestampColumnVector(TimestampColumnVector vector, int } /** - * Append a `HiveDecimalWritable` to a `WritableColumnVector`. + * Put a `HiveDecimalWritable` to a `WritableColumnVector`. */ - private static void appendDecimalWritable( + private static void putDecimalWritable( WritableColumnVector toColumn, + int index, int precision, int scale, HiveDecimalWritable decimalWritable) { @@ -492,12 +445,38 @@ private static void appendDecimalWritable( value.changePrecision(precision, scale); if (precision <= Decimal.MAX_INT_DIGITS()) { - toColumn.appendInt((int) value.toUnscaledLong()); + toColumn.putInt(index, (int) value.toUnscaledLong()); } else if (precision <= Decimal.MAX_LONG_DIGITS()) { - toColumn.appendLong(value.toUnscaledLong()); + toColumn.putLong(index, value.toUnscaledLong()); } else { byte[] bytes = value.toJavaBigDecimal().unscaledValue().toByteArray(); - toColumn.appendByteArray(bytes, 0, bytes.length); + toColumn.putByteArray(index, bytes, 0, bytes.length); + } + } + + /** + * Put `HiveDecimalWritable`s to a `WritableColumnVector`. + */ + private static void putDecimalWritables( + WritableColumnVector toColumn, + int size, + int precision, + int scale, + HiveDecimalWritable decimalWritable) { + HiveDecimal decimal = decimalWritable.getHiveDecimal(); + Decimal value = + Decimal.apply(decimal.bigDecimalValue(), decimal.precision(), decimal.scale()); + value.changePrecision(precision, scale); + + if (precision <= Decimal.MAX_INT_DIGITS()) { + toColumn.putInts(0, size, (int) value.toUnscaledLong()); + } else if (precision <= Decimal.MAX_LONG_DIGITS()) { + toColumn.putLongs(0, size, value.toUnscaledLong()); + } else { + byte[] bytes = value.toJavaBigDecimal().unscaledValue().toByteArray(); + for (int index = 0; index < size; index++) { + toColumn.putByteArray(index, bytes, 0, bytes.length); + } } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala index a02fb5300c0ca..e45f611a5a75f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.vectorized._ /** * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch. - * After creating, `initialize` and `setRequiredSchema` should be called sequentially. + * After creating, `initialize` and `initBatch` should be called sequentially. */ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] { import OrcColumnarBatchReader._ @@ -104,7 +104,7 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat /** * Initialize ORC file reader and batch record reader. - * Please note that `setRequiredSchema` is needed to be called after this. + * Please note that `initBatch` is needed to be called after this. */ override def initialize(inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): Unit = { val fileSplit = inputSplit.asInstanceOf[FileSplit] @@ -124,7 +124,8 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat * Set required schema and partition information. * With this information, this creates ColumnarBatch with the full schema. */ - def setRequiredSchema( + def initBatch( + useOffHeap: Boolean, orcSchema: TypeDescription, requestedColIds: Array[Int], requiredSchema: StructType, @@ -138,7 +139,8 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat this.requestedColIds = requestedColIds val capacity = DEFAULT_SIZE - if (DEFAULT_MEMORY_MODE == MemoryMode.OFF_HEAP) { + val memoryMode = if (useOffHeap) MemoryMode.OFF_HEAP else MemoryMode.ON_HEAP + if (memoryMode == MemoryMode.OFF_HEAP) { columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema) } else { columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema) @@ -185,54 +187,55 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat while (i < requiredSchema.length) { val field = requiredSchema(i) val toColumn = columnVectors(i) + toColumn.reserve(batchSize) if (requestedColIds(i) >= 0) { val fromColumn = batch.cols(requestedColIds(i)) if (fromColumn.isRepeating) { if (fromColumn.isNull(0)) { - toColumn.appendNulls(batchSize) + toColumn.putNulls(0, batchSize) } else { field.dataType match { case BooleanType => val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1 - toColumn.appendBooleans(batchSize, data) + toColumn.putBooleans(0, batchSize, data) case ByteType => val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte - toColumn.appendBytes(batchSize, data) + toColumn.putBytes(0, batchSize, data) case ShortType => val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort - toColumn.appendShorts(batchSize, data) + toColumn.putShorts(0, batchSize, data) case IntegerType | DateType => val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt - toColumn.appendInts(batchSize, data) + toColumn.putInts(0, batchSize, data) case LongType => val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) - toColumn.appendLongs(batchSize, data) + toColumn.putLongs(0, batchSize, data) case TimestampType => val data = fromColumn.asInstanceOf[TimestampColumnVector] - toColumn.appendLongs(batchSize, fromTimestampColumnVector(data, 0)) + toColumn.putLongs(0, batchSize, fromTimestampColumnVector(data, 0)) case FloatType => val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0).toFloat - toColumn.appendFloats(batchSize, data) + toColumn.putFloats(0, batchSize, data) case DoubleType => val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0) - toColumn.appendDoubles(batchSize, data) + toColumn.putDoubles(0, batchSize, data) case StringType | BinaryType => val data = fromColumn.asInstanceOf[BytesColumnVector] var index = 0 while (index < batchSize) { - toColumn.appendByteArray(data.vector(0), data.start(0), data.length(0)) + toColumn.putByteArray(index, data.vector(0), data.start(0), data.length(0)) index += 1 } case DecimalType.Fixed(precision, scale) => val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(0) - appendDecimalWritable(toColumn, precision, scale, d) + putDecimalWritables(toColumn, batchSize, precision, scale, d) case dt => throw new UnsupportedOperationException(s"Unsupported Data Type: $dt") @@ -244,7 +247,7 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat val data = fromColumn.asInstanceOf[LongColumnVector].vector var index = 0 while (index < batchSize) { - toColumn.appendBoolean(data(index) == 1) + toColumn.putBoolean(index, data(index) == 1) index += 1 } @@ -252,32 +255,32 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat val data = fromColumn.asInstanceOf[LongColumnVector].vector var index = 0 while (index < batchSize) { - toColumn.appendByte(data(index).toByte) + toColumn.putByte(index, data(index).toByte) index += 1 } case ShortType => val data = fromColumn.asInstanceOf[LongColumnVector].vector var index = 0 while (index < batchSize) { - toColumn.appendShort(data(index).toShort) + toColumn.putShort(index, data(index).toShort) index += 1 } case IntegerType | DateType => val data = fromColumn.asInstanceOf[LongColumnVector].vector var index = 0 while (index < batchSize) { - toColumn.appendInt(data(index).toInt) + toColumn.putInt(index, data(index).toInt) index += 1 } case LongType => val data = fromColumn.asInstanceOf[LongColumnVector].vector - toColumn.appendLongs(batchSize, data, 0) + toColumn.putLongs(0, batchSize, data, 0) case TimestampType => val data = fromColumn.asInstanceOf[TimestampColumnVector] var index = 0 while (index < batchSize) { - toColumn.appendLong(fromTimestampColumnVector(data, index)) + toColumn.putLong(index, fromTimestampColumnVector(data, index)) index += 1 } @@ -285,18 +288,18 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat val data = fromColumn.asInstanceOf[DoubleColumnVector].vector var index = 0 while (index < batchSize) { - toColumn.appendFloat(data(index).toFloat) + toColumn.putFloat(index, data(index).toFloat) index += 1 } case DoubleType => val data = fromColumn.asInstanceOf[DoubleColumnVector].vector - toColumn.appendDoubles(batchSize, data, 0) + toColumn.putDoubles(0, batchSize, data, 0) case StringType | BinaryType => val data = fromColumn.asInstanceOf[BytesColumnVector] var index = 0 while (index < batchSize) { - toColumn.appendByteArray(data.vector(index), data.start(index), data.length(index)) + toColumn.putByteArray(0, data.vector(index), data.start(index), data.length(index)) index += 1 } @@ -304,7 +307,7 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat val data = fromColumn.asInstanceOf[DecimalColumnVector] var index = 0 while (index < batchSize) { - appendDecimalWritable(toColumn, precision, scale, data.vector(index)) + putDecimalWritable(toColumn, index, precision, scale, data.vector(index)) index += 1 } @@ -318,9 +321,9 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat var index = 0 while (index < batchSize) { if (fromColumn.isNull(index)) { - toColumn.appendNull() + toColumn.putNull(index) } else { - toColumn.appendBoolean(vector(index) == 1) + toColumn.putBoolean(index, vector(index) == 1) } index += 1 } @@ -330,9 +333,9 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat var index = 0 while (index < batchSize) { if (fromColumn.isNull(index)) { - toColumn.appendNull() + toColumn.putNull(index) } else { - toColumn.appendByte(vector(index).toByte) + toColumn.putByte(index, vector(index).toByte) } index += 1 } @@ -342,9 +345,9 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat var index = 0 while (index < batchSize) { if (fromColumn.isNull(index)) { - toColumn.appendNull() + toColumn.putNull(index) } else { - toColumn.appendShort(vector(index).toShort) + toColumn.putShort(index, vector(index).toShort) } index += 1 } @@ -354,9 +357,9 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat var index = 0 while (index < batchSize) { if (fromColumn.isNull(index)) { - toColumn.appendNull() + toColumn.putNull(index) } else { - toColumn.appendInt(vector(index).toInt) + toColumn.putInt(index, vector(index).toInt) } index += 1 } @@ -366,9 +369,9 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat var index = 0 while (index < batchSize) { if (fromColumn.isNull(index)) { - toColumn.appendNull() + toColumn.putNull(index) } else { - toColumn.appendLong(vector(index)) + toColumn.putLong(index, vector(index)) } index += 1 } @@ -378,9 +381,9 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat var index = 0 while (index < batchSize) { if (fromColumn.isNull(index)) { - toColumn.appendNull() + toColumn.putNull(index) } else { - toColumn.appendLong(fromTimestampColumnVector(vector, index)) + toColumn.putLong(index, fromTimestampColumnVector(vector, index)) } index += 1 } @@ -390,9 +393,9 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat var index = 0 while (index < batchSize) { if (fromColumn.isNull(index)) { - toColumn.appendNull() + toColumn.putNull(index) } else { - toColumn.appendFloat(vector(index).toFloat) + toColumn.putFloat(index, vector(index).toFloat) } index += 1 } @@ -402,9 +405,9 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat var index = 0 while (index < batchSize) { if (fromColumn.isNull(index)) { - toColumn.appendNull() + toColumn.putNull(index) } else { - toColumn.appendDouble(vector(index)) + toColumn.putDouble(index, vector(index)) } index += 1 } @@ -414,10 +417,10 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat var index = 0 while (index < batchSize) { if (fromColumn.isNull(index)) { - toColumn.appendNull() + toColumn.putNull(index) } else { - toColumn.appendByteArray( - vector.vector(index), vector.start(index), vector.length(index)) + toColumn.putByteArray( + index, vector.vector(index), vector.start(index), vector.length(index)) } index += 1 } @@ -427,9 +430,9 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat var index = 0 while (index < batchSize) { if (fromColumn.isNull(index)) { - toColumn.appendNull() + toColumn.putNull(index) } else { - appendDecimalWritable(toColumn, precision, scale, vector(index)) + putDecimalWritable(toColumn, index, precision, scale, vector(index)) } index += 1 } @@ -449,11 +452,6 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat * Constants for OrcColumnarBatchReader. */ object OrcColumnarBatchReader { - /** - * Default memory mode for ColumnarBatch. - */ - val DEFAULT_MEMORY_MODE = MemoryMode.ON_HEAP - /** * The default size of batch. We use this value for both ORC and Spark consistently * because they have different default values like the following. @@ -470,10 +468,11 @@ object OrcColumnarBatchReader { vector.time(index) * 1000L + vector.nanos(index) / 1000L /** - * Append a `HiveDecimalWritable` to a `WritableColumnVector`. + * Put a `HiveDecimalWritable` to a `WritableColumnVector`. */ - private def appendDecimalWritable( + private def putDecimalWritable( toColumn: WritableColumnVector, + index: Int, precision: Int, scale: Int, decimalWritable: HiveDecimalWritable): Unit = { @@ -482,12 +481,39 @@ object OrcColumnarBatchReader { value.changePrecision(precision, scale) if (precision <= Decimal.MAX_INT_DIGITS) { - toColumn.appendInt(value.toUnscaledLong.toInt) + toColumn.putInt(index, value.toUnscaledLong.toInt) } else if (precision <= Decimal.MAX_LONG_DIGITS) { - toColumn.appendLong(value.toUnscaledLong) + toColumn.putLong(index, value.toUnscaledLong) } else { val bytes = value.toJavaBigDecimal.unscaledValue.toByteArray - toColumn.appendByteArray(bytes, 0, bytes.length) + toColumn.putByteArray(index, bytes, 0, bytes.length) + } + } + + /** + * Put `HiveDecimalWritable`s to a `WritableColumnVector`. + */ + private def putDecimalWritables( + toColumn: WritableColumnVector, + size: Int, + precision: Int, + scale: Int, + decimalWritable: HiveDecimalWritable): Unit = { + val decimal = decimalWritable.getHiveDecimal() + val value = Decimal(decimal.bigDecimalValue, decimal.precision(), decimal.scale()) + value.changePrecision(precision, scale) + + if (precision <= Decimal.MAX_INT_DIGITS) { + toColumn.putInts(0, size, value.toUnscaledLong.toInt) + } else if (precision <= Decimal.MAX_LONG_DIGITS) { + toColumn.putLongs(0, size, value.toUnscaledLong) + } else { + val bytes = value.toJavaBigDecimal.unscaledValue.toByteArray + var index = 0 + while (index < size) { + toColumn.putByteArray(index, bytes, 0, bytes.length) + index += 1 + } } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala index 12f80a076bc2b..ae22e8eaaf80e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala @@ -147,6 +147,8 @@ class OrcFileFormat } val resultSchema = StructType(requiredSchema.fields ++ partitionSchema.fields) + val sqlConf = sparkSession.sessionState.conf + val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled val enableVectorizedReader = supportBatch(sparkSession, resultSchema) val enableVectorizedJavaReader = sparkSession.sessionState.conf.orcVectorizedJavaReaderEnabled @@ -180,11 +182,13 @@ class OrcFileFormat val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) val taskAttemptContext = new TaskAttemptContextImpl(taskConf, attemptId) + val taskContext = Option(TaskContext.get()) if (enableVectorizedReader) { val batchReader = if (enableVectorizedJavaReader) { - val vectorizedReader = new JavaOrcColumnarBatchReader + val vectorizedReader = new JavaOrcColumnarBatchReader( + enableOffHeapColumnVector && taskContext.isDefined) vectorizedReader.initialize(fileSplit, taskAttemptContext) - vectorizedReader.setRequiredSchema( + vectorizedReader.initBatch( reader.getSchema, requestedColIds, requiredSchema, @@ -194,7 +198,8 @@ class OrcFileFormat } else { val vectorizedReader = new OrcColumnarBatchReader vectorizedReader.initialize(fileSplit, taskAttemptContext) - vectorizedReader.setRequiredSchema( + vectorizedReader.initBatch( + enableOffHeapColumnVector && taskContext.isDefined, reader.getSchema, requestedColIds, requiredSchema, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala index 2f20f579afe2b..4b1b66cf5e630 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala @@ -123,45 +123,45 @@ object OrcReadBenchmark { SQL Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1226 / 1227 12.8 78.0 1.0X - Native ORC Vectorized 165 / 179 95.3 10.5 7.4X - Native ORC Vectorized (Java) 170 / 179 92.8 10.8 7.2X - Hive built-in ORC 1423 / 1425 11.1 90.5 0.9X + Native ORC MR 1188 / 1230 13.2 75.6 1.0X + Native ORC Vectorized 163 / 174 96.7 10.3 7.3X + Native ORC Vectorized (Java) 156 / 168 100.8 9.9 7.6X + Hive built-in ORC 1413 / 1416 11.1 89.8 0.8X SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1193 / 1284 13.2 75.9 1.0X - Native ORC Vectorized 164 / 170 95.9 10.4 7.3X - Native ORC Vectorized (Java) 175 / 183 89.6 11.2 6.8X - Hive built-in ORC 1685 / 1701 9.3 107.1 0.7X + Native ORC MR 1270 / 1324 12.4 80.7 1.0X + Native ORC Vectorized 160 / 166 98.2 10.2 7.9X + Native ORC Vectorized (Java) 160 / 169 98.2 10.2 7.9X + Hive built-in ORC 1662 / 1681 9.5 105.6 0.8X SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1362 / 1375 11.6 86.6 1.0X - Native ORC Vectorized 273 / 283 57.7 17.3 5.0X - Native ORC Vectorized (Java) 242 / 249 65.1 15.4 5.6X - Hive built-in ORC 1894 / 1905 8.3 120.4 0.7X + Native ORC MR 1353 / 1365 11.6 86.0 1.0X + Native ORC Vectorized 260 / 274 60.4 16.5 5.2X + Native ORC Vectorized (Java) 225 / 235 69.8 14.3 6.0X + Hive built-in ORC 1908 / 1933 8.2 121.3 0.7X SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1349 / 1413 11.7 85.7 1.0X - Native ORC Vectorized 290 / 299 54.2 18.4 4.6X - Native ORC Vectorized (Java) 288 / 297 54.7 18.3 4.7X - Hive built-in ORC 1917 / 1928 8.2 121.9 0.7X + Native ORC MR 1335 / 1357 11.8 84.9 1.0X + Native ORC Vectorized 288 / 302 54.7 18.3 4.6X + Native ORC Vectorized (Java) 292 / 296 53.9 18.5 4.6X + Hive built-in ORC 1908 / 1973 8.2 121.3 0.7X SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1393 / 1420 11.3 88.6 1.0X - Native ORC Vectorized 383 / 394 41.0 24.4 3.6X - Native ORC Vectorized (Java) 346 / 352 45.5 22.0 4.0X - Hive built-in ORC 1980 / 2016 7.9 125.9 0.7X + Native ORC MR 1405 / 1469 11.2 89.3 1.0X + Native ORC Vectorized 361 / 363 43.6 22.9 3.9X + Native ORC Vectorized (Java) 324 / 332 48.6 20.6 4.3X + Hive built-in ORC 2044 / 2073 7.7 130.0 0.7X SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1472 / 1489 10.7 93.6 1.0X - Native ORC Vectorized 389 / 400 40.5 24.7 3.8X - Native ORC Vectorized (Java) 393 / 400 40.0 25.0 3.7X - Hive built-in ORC 2140 / 2144 7.3 136.1 0.7X + Native ORC MR 1468 / 1482 10.7 93.3 1.0X + Native ORC Vectorized 395 / 403 39.8 25.1 3.7X + Native ORC Vectorized (Java) 397 / 406 39.6 25.2 3.7X + Hive built-in ORC 2078 / 2097 7.6 132.1 0.7X */ sqlBenchmark.run() } @@ -208,10 +208,10 @@ object OrcReadBenchmark { Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2585 / 2656 4.1 246.5 1.0X - Native ORC Vectorized 1370 / 1380 7.7 130.6 1.9X - Native ORC Vectorized (Java) 1385 / 1394 7.6 132.1 1.9X - Hive built-in ORC 3832 / 3906 2.7 365.4 0.7X + Native ORC MR 2586 / 2670 4.1 246.6 1.0X + Native ORC Vectorized 1219 / 1228 8.6 116.3 2.1X + Native ORC Vectorized (Java) 1348 / 1358 7.8 128.6 1.9X + Hive built-in ORC 3677 / 3728 2.9 350.6 0.7X */ benchmark.run() } @@ -298,20 +298,20 @@ object OrcReadBenchmark { Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz - Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------ - Read data column - Native ORC MR 1531 / 1555 10.3 97.4 1.0X - Read data column - Native ORC Vectorized 294 / 299 53.4 18.7 5.2X - Read data column - Native ORC Vectorized (Java) 293 / 298 53.6 18.6 5.2X - Read data column - Hive built-in ORC 2083 / 2104 7.5 132.5 0.7X - Read partition column - Native ORC MR 1047 / 1048 15.0 66.6 1.5X - Read partition column - Native ORC Vectorized 53 / 55 296.8 3.4 28.9X - Read partition column - Native ORC Vectorized (Java) 52 / 53 303.8 3.3 29.6X - Read partition column - Hive built-in ORC 1301 / 1312 12.1 82.7 1.2X - Read both columns - Native ORC MR 1597 / 1606 9.8 101.6 1.0X - Read both columns - Native ORC Vectorized 329 / 336 47.8 20.9 4.7X - Read both columns - Native ORC Vectorized (Java) 333 / 338 47.2 21.2 4.6X - Read both columns - Hive built-in ORC 2100 / 2118 7.5 133.5 0.7X + Read data column - Native ORC MR 1562 / 1578 10.1 99.3 1.0X + Read data column - Native ORC Vectorized 297 / 303 52.9 18.9 5.3X + Read data column - Native ORC Vectorized (Java) 300 / 305 52.5 19.0 5.2X + Read data column - Hive built-in ORC 2119 / 2131 7.4 134.7 0.7X + Read partition column - Native ORC MR 1084 / 1092 14.5 68.9 1.4X + Read partition column - Native ORC Vectorized 53 / 55 297.1 3.4 29.5X + Read partition column - Native ORC Vectorized (Java) 51 / 54 306.8 3.3 30.5X + Read partition column - Hive built-in ORC 1317 / 1318 11.9 83.7 1.2X + Read both columns - Native ORC MR 1580 / 1583 10.0 100.4 1.0X + Read both columns - Native ORC Vectorized 329 / 336 47.8 20.9 4.7X + Read both columns - Native ORC Vectorized (Java) 336 / 340 46.8 21.4 4.6X + Read both columns - Hive built-in ORC 2162 / 2207 7.3 137.5 0.7X */ benchmark.run() } @@ -355,10 +355,10 @@ object OrcReadBenchmark { String Dictionary: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1313 / 1333 8.0 125.3 1.0X - Native ORC Vectorized 371 / 382 28.3 35.4 3.5X - Native ORC Vectorized (Java) 372 / 382 28.2 35.5 3.5X - Hive built-in ORC 1945 / 1989 5.4 185.5 0.7X + Native ORC MR 1340 / 1343 7.8 127.8 1.0X + Native ORC Vectorized 311 / 339 33.7 29.7 4.3X + Native ORC Vectorized (Java) 365 / 374 28.7 34.8 3.7X + Hive built-in ORC 2020 / 2037 5.2 192.7 0.7X */ benchmark.run() } @@ -410,24 +410,24 @@ object OrcReadBenchmark { String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR (0.0%) 2578 / 2589 4.1 245.8 1.0X - Native ORC Vectorized (0.0%) 1055 / 1058 9.9 100.6 2.4X - Native ORC Vectorized (0.0%) (Java) 1071 / 1089 9.8 102.2 2.4X - Hive built-in ORC (0.0%) 3954 / 3955 2.7 377.0 0.7X + Native ORC MR (0.0%) 2515 / 2521 4.2 239.9 1.0X + Native ORC Vectorized (0.0%) 972 / 989 10.8 92.7 2.6X + Native ORC Vectorized (0.0%) (Java) 1040 / 1052 10.1 99.2 2.4X + Hive built-in ORC (0.0%) 4021 / 4035 2.6 383.5 0.6X String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR (0.5%) 2410 / 2414 4.4 229.8 1.0X - Native ORC Vectorized (0.5%) 1248 / 1250 8.4 119.0 1.9X - Native ORC Vectorized (0.5%) (Java) 1248 / 1259 8.4 119.0 1.9X - Hive built-in ORC (0.5%) 2887 / 2927 3.6 275.3 0.8X + Native ORC MR (0.5%) 2421 / 2433 4.3 230.9 1.0X + Native ORC Vectorized (0.5%) 1238 / 1249 8.5 118.1 2.0X + Native ORC Vectorized (0.5%) (Java) 1241 / 1260 8.4 118.4 2.0X + Hive built-in ORC (0.5%) 3004 / 3008 3.5 286.5 0.8X String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR (0.95%) 1288 / 1311 8.1 122.9 1.0X - Native ORC Vectorized (0.95%) 455 / 463 23.0 43.4 2.8X - Native ORC Vectorized (0.95%) (Java) 461 / 476 22.7 44.0 2.8X - Hive built-in ORC (0.95%) 1658 / 1679 6.3 158.1 0.8X + Native ORC MR (0.95%) 1294 / 1318 8.1 123.4 1.0X + Native ORC Vectorized (0.95%) 448 / 454 23.4 42.7 2.9X + Native ORC Vectorized (0.95%) (Java) 441 / 456 23.8 42.1 2.9X + Hive built-in ORC (0.95%) 1660 / 1685 6.3 158.4 0.8X */ benchmark.run() } @@ -475,24 +475,24 @@ object OrcReadBenchmark { SQL Single Column Scan from wide table (100 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1076 / 1076 1.0 1026.1 1.0X - Native ORC Vectorized 90 / 98 11.7 85.6 12.0X - Native ORC Vectorized (Java) 91 / 97 11.6 86.4 11.9X - Hive built-in ORC 406 / 408 2.6 386.9 2.7X + Native ORC MR 1136 / 1137 0.9 1083.6 1.0X + Native ORC Vectorized 90 / 98 11.7 85.7 12.6X + Native ORC Vectorized (Java) 91 / 98 11.5 86.8 12.5X + Hive built-in ORC 379 / 389 2.8 361.2 3.0X SQL Single Column Scan from wide table (200 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2152 / 2171 0.5 2051.9 1.0X - Native ORC Vectorized 155 / 164 6.7 148.2 13.9X - Native ORC Vectorized (Java) 155 / 162 6.8 147.5 13.9X - Hive built-in ORC 628 / 633 1.7 598.9 3.4X + Native ORC MR 2269 / 2295 0.5 2164.0 1.0X + Native ORC Vectorized 158 / 164 6.7 150.2 14.4X + Native ORC Vectorized (Java) 158 / 166 6.6 150.6 14.4X + Hive built-in ORC 602 / 606 1.7 574.2 3.8X SQL Single Column Scan from wide table (300 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 3219 / 3233 0.3 3070.2 1.0X - Native ORC Vectorized 268 / 279 3.9 255.5 12.0X - Native ORC Vectorized (Java) 267 / 275 3.9 254.7 12.1X - Hive built-in ORC 892 / 898 1.2 850.5 3.6X + Native ORC MR 3414 / 3433 0.3 3255.5 1.0X + Native ORC Vectorized 266 / 279 3.9 253.5 12.8X + Native ORC Vectorized (Java) 262 / 274 4.0 249.7 13.0X + Hive built-in ORC 848 / 850 1.2 808.6 4.0X */ sqlBenchmark.run() } From 3df7d1ee9d0cb9ea25a9d1e0e2db539121ad50de Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 8 Jan 2018 01:30:40 -0800 Subject: [PATCH 12/23] Split nextBatch. --- .../orc/JavaOrcColumnarBatchReader.java | 436 ++++++++++-------- .../orc/OrcColumnarBatchReader.scala | 18 +- .../datasources/orc/OrcFileFormat.scala | 4 +- .../spark/sql/hive/orc/OrcReadBenchmark.scala | 136 +++--- 4 files changed, 312 insertions(+), 282 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java index 9eeb635111210..39ea61b1133cb 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java @@ -64,7 +64,7 @@ public class JavaOrcColumnarBatchReader extends RecordReader 0) { - int partitionIdx = requiredSchema.fields().length; + int partitionIdx = requiredFields.length; for (int i = 0; i < partitionValues.numFields(); i++) { ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i); columnVectors[i + partitionIdx].setIsConstant(); @@ -178,7 +179,7 @@ public void initBatch( } // Initialize the missing columns once. - for (int i = 0; i < requiredSchema.length(); i++) { + for (int i = 0; i < requiredFields.length; i++) { if (requestedColIds[i] == -1) { columnVectors[i].putNulls(0, columnarBatch.capacity()); columnVectors[i].setIsConstant(); @@ -205,8 +206,8 @@ private boolean nextBatch() throws IOException { vector.reset(); } columnarBatch.setNumRows(batchSize); - for (int i = 0; i < requiredSchema.length(); i++) { - StructField field = requiredSchema.fields()[i]; + for (int i = 0; i < requiredFields.length; i++) { + StructField field = requiredFields[i]; WritableColumnVector toColumn = columnVectors[i]; if (requestedColIds[i] >= 0) { @@ -214,206 +215,232 @@ private boolean nextBatch() throws IOException { toColumn.reserve(batchSize); if (fromColumn.isRepeating) { - if (fromColumn.isNull[0]) { - toColumn.putNulls(0, batchSize); - } else { - DataType type = field.dataType(); - if (type instanceof BooleanType) { - toColumn.putBooleans(0, batchSize, ((LongColumnVector)fromColumn).vector[0] == 1); - } else if (type instanceof ByteType) { - toColumn.putBytes(0, batchSize, (byte)((LongColumnVector)fromColumn).vector[0]); - } else if (type instanceof ShortType) { - toColumn.putShorts(0, batchSize, (short)((LongColumnVector)fromColumn).vector[0]); - } else if (type instanceof IntegerType || type instanceof DateType) { - toColumn.putInts(0, batchSize, (int)((LongColumnVector)fromColumn).vector[0]); - } else if (type instanceof LongType) { - toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector[0]); - } else if (type instanceof TimestampType) { - toColumn.putLongs(0, batchSize, fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0)); - } else if (type instanceof FloatType) { - toColumn.putFloats(0, batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]); - } else if (type instanceof DoubleType) { - toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]); - } else if (type instanceof StringType || type instanceof BinaryType) { - BytesColumnVector data = (BytesColumnVector)fromColumn; - for (int index = 0; index < batchSize; index++) { - toColumn.putByteArray(index, data.vector[0], data.start[0], data.length[0]); - } - } else if (type instanceof DecimalType) { - DecimalType decimalType = (DecimalType)type; - putDecimalWritables( - toColumn, - batchSize, - decimalType.precision(), - decimalType.scale(), - ((DecimalColumnVector)fromColumn).vector[0]); - } else { - throw new UnsupportedOperationException("Unsupported Data Type: " + type); - } - } + putRepeatingValues(batchSize, field, fromColumn, toColumn); } else if (fromColumn.noNulls) { - DataType type = field.dataType(); - if (type instanceof BooleanType) { - long[] data = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - toColumn.putBoolean(index, data[index] == 1); - } - } else if (type instanceof ByteType) { - long[] data = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - toColumn.putByte(index, (byte)data[index]); - } - } else if (type instanceof ShortType) { - long[] data = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - toColumn.putShort(index, (short)data[index]); - } - } else if (type instanceof IntegerType || type instanceof DateType) { - long[] data = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - toColumn.putInt(index, (int)data[index]); - } - } else if (type instanceof LongType) { - toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector, 0); - } else if (type instanceof TimestampType) { - TimestampColumnVector data = ((TimestampColumnVector)fromColumn); - for (int index = 0; index < batchSize; index++) { - toColumn.putLong(index, fromTimestampColumnVector(data, index)); - } - } else if (type instanceof FloatType) { - double[] data = ((DoubleColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - toColumn.putFloat(index, (float)data[index]); - } - } else if (type instanceof DoubleType) { - toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0); - } else if (type instanceof StringType || type instanceof BinaryType) { - BytesColumnVector data = ((BytesColumnVector)fromColumn); - for (int index = 0; index < batchSize; index++) { - toColumn.putByteArray(index, data.vector[index], data.start[index], data.length[index]); - } - } else if (type instanceof DecimalType) { - DecimalType decimalType = (DecimalType)type; - DecimalColumnVector data = ((DecimalColumnVector)fromColumn); - for (int index = 0; index < batchSize; index++) { - putDecimalWritable( - toColumn, - index, - decimalType.precision(), - decimalType.scale(), - data.vector[index]); - } - } else { - throw new UnsupportedOperationException("Unsupported Data Type: " + type); - } + putNonNullValues(batchSize, field, fromColumn, toColumn); } else { - DataType type = field.dataType(); - if (type instanceof BooleanType) { - long[] vector = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putBoolean(index, vector[index] == 1); - } - } - } else if (type instanceof ByteType) { - long[] vector = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putByte(index, (byte)vector[index]); - } - } - } else if (type instanceof ShortType) { - long[] vector = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putShort(index, (short)vector[index]); - } - } - } else if (type instanceof IntegerType || type instanceof DateType) { - long[] vector = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putInt(index, (int)vector[index]); - } - } - } else if (type instanceof LongType) { - long[] vector = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putLong(index, vector[index]); - } - } - } else if (type instanceof TimestampType) { - TimestampColumnVector vector = ((TimestampColumnVector)fromColumn); - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putLong(index, fromTimestampColumnVector(vector, index)); - } - } - } else if (type instanceof FloatType) { - double[] vector = ((DoubleColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putFloat(index, (float)vector[index]); - } - } - } else if (type instanceof DoubleType) { - double[] vector = ((DoubleColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putDouble(index, vector[index]); - } - } - } else if (type instanceof StringType || type instanceof BinaryType) { - BytesColumnVector vector = (BytesColumnVector)fromColumn; - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putByteArray( - index, vector.vector[index], vector.start[index], vector.length[index]); - } - } - } else if (type instanceof DecimalType) { - DecimalType decimalType = (DecimalType)type; - HiveDecimalWritable[] vector = ((DecimalColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - putDecimalWritable( - toColumn, - index, - decimalType.precision(), - decimalType.scale(), - vector[index]); - } - } - } else { - throw new UnsupportedOperationException("Unsupported Data Type: " + type); - } + putValues(batchSize, field, fromColumn, toColumn); } } } return true; } + private void putRepeatingValues( + int batchSize, + StructField field, + ColumnVector fromColumn, + WritableColumnVector toColumn) { + if (fromColumn.isNull[0]) { + toColumn.putNulls(0, batchSize); + } else { + DataType type = field.dataType(); + if (type instanceof BooleanType) { + toColumn.putBooleans(0, batchSize, ((LongColumnVector)fromColumn).vector[0] == 1); + } else if (type instanceof ByteType) { + toColumn.putBytes(0, batchSize, (byte)((LongColumnVector)fromColumn).vector[0]); + } else if (type instanceof ShortType) { + toColumn.putShorts(0, batchSize, (short)((LongColumnVector)fromColumn).vector[0]); + } else if (type instanceof IntegerType || type instanceof DateType) { + toColumn.putInts(0, batchSize, (int)((LongColumnVector)fromColumn).vector[0]); + } else if (type instanceof LongType) { + toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector[0]); + } else if (type instanceof TimestampType) { + toColumn.putLongs(0, batchSize, + fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0)); + } else if (type instanceof FloatType) { + toColumn.putFloats(0, batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]); + } else if (type instanceof DoubleType) { + toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]); + } else if (type instanceof StringType || type instanceof BinaryType) { + BytesColumnVector data = (BytesColumnVector)fromColumn; + toColumn.putByteArray(0, data.vector[0]); + for (int index = 0; index < batchSize; index++) { + toColumn.putArray(index, data.start[0], data.length[0]); + } + } else if (type instanceof DecimalType) { + DecimalType decimalType = (DecimalType)type; + putDecimalWritables( + toColumn, + batchSize, + decimalType.precision(), + decimalType.scale(), + ((DecimalColumnVector)fromColumn).vector[0]); + } else { + throw new UnsupportedOperationException("Unsupported Data Type: " + type); + } + } + } + + private void putNonNullValues( + int batchSize, + StructField field, + ColumnVector fromColumn, + WritableColumnVector toColumn) { + DataType type = field.dataType(); + if (type instanceof BooleanType) { + long[] data = ((LongColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + toColumn.putBoolean(index, data[index] == 1); + } + } else if (type instanceof ByteType) { + long[] data = ((LongColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + toColumn.putByte(index, (byte)data[index]); + } + } else if (type instanceof ShortType) { + long[] data = ((LongColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + toColumn.putShort(index, (short)data[index]); + } + } else if (type instanceof IntegerType || type instanceof DateType) { + long[] data = ((LongColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + toColumn.putInt(index, (int)data[index]); + } + } else if (type instanceof LongType) { + toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector, 0); + } else if (type instanceof TimestampType) { + TimestampColumnVector data = ((TimestampColumnVector)fromColumn); + for (int index = 0; index < batchSize; index++) { + toColumn.putLong(index, fromTimestampColumnVector(data, index)); + } + } else if (type instanceof FloatType) { + double[] data = ((DoubleColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + toColumn.putFloat(index, (float)data[index]); + } + } else if (type instanceof DoubleType) { + toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0); + } else if (type instanceof StringType || type instanceof BinaryType) { + BytesColumnVector data = ((BytesColumnVector)fromColumn); + for (int index = 0; index < batchSize; index++) { + toColumn.putByteArray(index, data.vector[index], data.start[index], data.length[index]); + } + } else if (type instanceof DecimalType) { + DecimalType decimalType = (DecimalType)type; + DecimalColumnVector data = ((DecimalColumnVector)fromColumn); + for (int index = 0; index < batchSize; index++) { + putDecimalWritable( + toColumn, + index, + decimalType.precision(), + decimalType.scale(), + data.vector[index]); + } + } else { + throw new UnsupportedOperationException("Unsupported Data Type: " + type); + } + } + + private void putValues( + int batchSize, + StructField field, + ColumnVector fromColumn, + WritableColumnVector toColumn) { + DataType type = field.dataType(); + if (type instanceof BooleanType) { + long[] vector = ((LongColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + toColumn.putBoolean(index, vector[index] == 1); + } + } + } else if (type instanceof ByteType) { + long[] vector = ((LongColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + toColumn.putByte(index, (byte)vector[index]); + } + } + } else if (type instanceof ShortType) { + long[] vector = ((LongColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + toColumn.putShort(index, (short)vector[index]); + } + } + } else if (type instanceof IntegerType || type instanceof DateType) { + long[] vector = ((LongColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + toColumn.putInt(index, (int)vector[index]); + } + } + } else if (type instanceof LongType) { + long[] vector = ((LongColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + toColumn.putLong(index, vector[index]); + } + } + } else if (type instanceof TimestampType) { + TimestampColumnVector vector = ((TimestampColumnVector)fromColumn); + for (int index = 0; index < batchSize; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + toColumn.putLong(index, fromTimestampColumnVector(vector, index)); + } + } + } else if (type instanceof FloatType) { + double[] vector = ((DoubleColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + toColumn.putFloat(index, (float)vector[index]); + } + } + } else if (type instanceof DoubleType) { + double[] vector = ((DoubleColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + toColumn.putDouble(index, vector[index]); + } + } + } else if (type instanceof StringType || type instanceof BinaryType) { + BytesColumnVector vector = (BytesColumnVector)fromColumn; + for (int index = 0; index < batchSize; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + toColumn.putByteArray( + index, vector.vector[index], vector.start[index], vector.length[index]); + } + } + } else if (type instanceof DecimalType) { + DecimalType decimalType = (DecimalType)type; + HiveDecimalWritable[] vector = ((DecimalColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + putDecimalWritable( + toColumn, + index, + decimalType.precision(), + decimalType.scale(), + vector[index]); + } + } + } else { + throw new UnsupportedOperationException("Unsupported Data Type: " + type); + } + } + /** * The default size of batch. We use this value for both ORC and Spark consistently * because they have different default values like the following. @@ -465,7 +492,7 @@ private static void putDecimalWritables( HiveDecimalWritable decimalWritable) { HiveDecimal decimal = decimalWritable.getHiveDecimal(); Decimal value = - Decimal.apply(decimal.bigDecimalValue(), decimal.precision(), decimal.scale()); + Decimal.apply(decimal.bigDecimalValue(), decimal.precision(), decimal.scale()); value.changePrecision(precision, scale); if (precision <= Decimal.MAX_INT_DIGITS()) { @@ -474,8 +501,9 @@ private static void putDecimalWritables( toColumn.putLongs(0, size, value.toUnscaledLong()); } else { byte[] bytes = value.toJavaBigDecimal().unscaledValue().toByteArray(); + toColumn.putByteArray(0, bytes); for (int index = 0; index < size; index++) { - toColumn.putByteArray(index, bytes, 0, bytes.length); + toColumn.putArray(index, 0, bytes.length); } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala index e45f611a5a75f..6d32788aced2f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala @@ -61,7 +61,7 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat /** * Required Schema. */ - private var requiredSchema: StructType = _ + private var requiredFields: Array[StructField] = _ /** * ColumnarBatch for vectorized execution by whole-stage codegen. @@ -128,15 +128,17 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat useOffHeap: Boolean, orcSchema: TypeDescription, requestedColIds: Array[Int], - requiredSchema: StructType, + requiredFields: Array[StructField], partitionSchema: StructType, partitionValues: InternalRow): Unit = { batch = orcSchema.createRowBatch(DEFAULT_SIZE) assert(!batch.selectedInUse, "`selectedInUse` should be initialized with `false`.") - val resultSchema = StructType(requiredSchema.fields ++ partitionSchema.fields) - this.requiredSchema = requiredSchema + this.requiredFields = requiredFields this.requestedColIds = requestedColIds + assert(requiredFields.length == requestedColIds.length) + + val resultSchema = StructType(requiredFields ++ partitionSchema.fields) val capacity = DEFAULT_SIZE val memoryMode = if (useOffHeap) MemoryMode.OFF_HEAP else MemoryMode.ON_HEAP @@ -148,7 +150,7 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat columnarBatch = new ColumnarBatch(resultSchema, columnVectors.toArray, capacity) if (partitionValues.numFields > 0) { - val partitionIdx = requiredSchema.fields.length + val partitionIdx = requiredFields.length for (i <- 0 until partitionValues.numFields) { ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i) columnVectors(i + partitionIdx).setIsConstant() @@ -156,7 +158,7 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat } // Initialize the missing columns once. - for (i <- 0 until requiredSchema.length) { + for (i <- 0 until requiredFields.length) { if (requestedColIds(i) < 0) { columnVectors(i).putNulls(0, columnarBatch.capacity) columnVectors(i).setIsConstant() @@ -184,8 +186,8 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat columnarBatch.setNumRows(batchSize) var i = 0 - while (i < requiredSchema.length) { - val field = requiredSchema(i) + while (i < requiredFields.length) { + val field = requiredFields(i) val toColumn = columnVectors(i) toColumn.reserve(batchSize) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala index ae22e8eaaf80e..7abcdba9639cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala @@ -191,7 +191,7 @@ class OrcFileFormat vectorizedReader.initBatch( reader.getSchema, requestedColIds, - requiredSchema, + requiredSchema.fields, partitionSchema, file.partitionValues) vectorizedReader @@ -202,7 +202,7 @@ class OrcFileFormat enableOffHeapColumnVector && taskContext.isDefined, reader.getSchema, requestedColIds, - requiredSchema, + requiredSchema.fields, partitionSchema, file.partitionValues) vectorizedReader diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala index 4b1b66cf5e630..dc73c10ed7817 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala @@ -123,45 +123,45 @@ object OrcReadBenchmark { SQL Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1188 / 1230 13.2 75.6 1.0X - Native ORC Vectorized 163 / 174 96.7 10.3 7.3X - Native ORC Vectorized (Java) 156 / 168 100.8 9.9 7.6X - Hive built-in ORC 1413 / 1416 11.1 89.8 0.8X + Native ORC MR 1198 / 1236 13.1 76.2 1.0X + Native ORC Vectorized 163 / 170 96.4 10.4 7.3X + Native ORC Vectorized (Java) 156 / 161 101.1 9.9 7.7X + Hive built-in ORC 1508 / 1511 10.4 95.8 0.8X SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1270 / 1324 12.4 80.7 1.0X - Native ORC Vectorized 160 / 166 98.2 10.2 7.9X - Native ORC Vectorized (Java) 160 / 169 98.2 10.2 7.9X - Hive built-in ORC 1662 / 1681 9.5 105.6 0.8X + Native ORC MR 1361 / 1391 11.6 86.5 1.0X + Native ORC Vectorized 197 / 204 80.0 12.5 6.9X + Native ORC Vectorized (Java) 198 / 203 79.5 12.6 6.9X + Hive built-in ORC 1670 / 1680 9.4 106.2 0.8X SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1353 / 1365 11.6 86.0 1.0X - Native ORC Vectorized 260 / 274 60.4 16.5 5.2X - Native ORC Vectorized (Java) 225 / 235 69.8 14.3 6.0X - Hive built-in ORC 1908 / 1933 8.2 121.3 0.7X + Native ORC MR 1643 / 1829 9.6 104.5 1.0X + Native ORC Vectorized 232 / 245 67.9 14.7 7.1X + Native ORC Vectorized (Java) 225 / 232 69.9 14.3 7.3X + Hive built-in ORC 1978 / 1988 8.0 125.8 0.8X SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1335 / 1357 11.8 84.9 1.0X - Native ORC Vectorized 288 / 302 54.7 18.3 4.6X - Native ORC Vectorized (Java) 292 / 296 53.9 18.5 4.6X - Hive built-in ORC 1908 / 1973 8.2 121.3 0.7X + Native ORC MR 1504 / 1522 10.5 95.7 1.0X + Native ORC Vectorized 288 / 298 54.6 18.3 5.2X + Native ORC Vectorized (Java) 281 / 294 55.9 17.9 5.3X + Hive built-in ORC 1913 / 1925 8.2 121.6 0.8X SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1405 / 1469 11.2 89.3 1.0X - Native ORC Vectorized 361 / 363 43.6 22.9 3.9X - Native ORC Vectorized (Java) 324 / 332 48.6 20.6 4.3X - Hive built-in ORC 2044 / 2073 7.7 130.0 0.7X + Native ORC MR 1452 / 1498 10.8 92.3 1.0X + Native ORC Vectorized 324 / 329 48.5 20.6 4.5X + Native ORC Vectorized (Java) 323 / 329 48.7 20.5 4.5X + Hive built-in ORC 1945 / 2013 8.1 123.7 0.7X SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1468 / 1482 10.7 93.3 1.0X - Native ORC Vectorized 395 / 403 39.8 25.1 3.7X - Native ORC Vectorized (Java) 397 / 406 39.6 25.2 3.7X - Hive built-in ORC 2078 / 2097 7.6 132.1 0.7X + Native ORC MR 1548 / 1551 10.2 98.4 1.0X + Native ORC Vectorized 394 / 399 39.9 25.0 3.9X + Native ORC Vectorized (Java) 396 / 406 39.8 25.1 3.9X + Hive built-in ORC 2079 / 2098 7.6 132.2 0.7X */ sqlBenchmark.run() } @@ -208,10 +208,10 @@ object OrcReadBenchmark { Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2586 / 2670 4.1 246.6 1.0X - Native ORC Vectorized 1219 / 1228 8.6 116.3 2.1X - Native ORC Vectorized (Java) 1348 / 1358 7.8 128.6 1.9X - Hive built-in ORC 3677 / 3728 2.9 350.6 0.7X + Native ORC MR 2859 / 2860 3.7 272.6 1.0X + Native ORC Vectorized 1223 / 1227 8.6 116.6 2.3X + Native ORC Vectorized (Java) 1337 / 1339 7.8 127.5 2.1X + Hive built-in ORC 3838 / 3893 2.7 366.0 0.7X */ benchmark.run() } @@ -300,18 +300,18 @@ object OrcReadBenchmark { Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------ - Read data column - Native ORC MR 1562 / 1578 10.1 99.3 1.0X - Read data column - Native ORC Vectorized 297 / 303 52.9 18.9 5.3X - Read data column - Native ORC Vectorized (Java) 300 / 305 52.5 19.0 5.2X - Read data column - Hive built-in ORC 2119 / 2131 7.4 134.7 0.7X - Read partition column - Native ORC MR 1084 / 1092 14.5 68.9 1.4X - Read partition column - Native ORC Vectorized 53 / 55 297.1 3.4 29.5X - Read partition column - Native ORC Vectorized (Java) 51 / 54 306.8 3.3 30.5X - Read partition column - Hive built-in ORC 1317 / 1318 11.9 83.7 1.2X - Read both columns - Native ORC MR 1580 / 1583 10.0 100.4 1.0X - Read both columns - Native ORC Vectorized 329 / 336 47.8 20.9 4.7X - Read both columns - Native ORC Vectorized (Java) 336 / 340 46.8 21.4 4.6X - Read both columns - Hive built-in ORC 2162 / 2207 7.3 137.5 0.7X + Read data column - Native ORC MR 1519 / 1546 10.4 96.6 1.0X + Read data column - Native ORC Vectorized 293 / 301 53.7 18.6 5.2X + Read data column - Native ORC Vectorized (Java) 295 / 300 53.3 18.8 5.1X + Read data column - Hive built-in ORC 2120 / 2163 7.4 134.8 0.7X + Read partition column - Native ORC MR 1077 / 1084 14.6 68.5 1.4X + Read partition column - Native ORC Vectorized 51 / 55 305.6 3.3 29.5X + Read partition column - Native ORC Vectorized (Java) 52 / 55 301.2 3.3 29.1X + Read partition column - Hive built-in ORC 1299 / 1302 12.1 82.6 1.2X + Read both columns - Native ORC MR 1699 / 1720 9.3 108.0 0.9X + Read both columns - Native ORC Vectorized 327 / 333 48.1 20.8 4.6X + Read both columns - Native ORC Vectorized (Java) 330 / 335 47.6 21.0 4.6X + Read both columns - Hive built-in ORC 2149 / 2152 7.3 136.6 0.7X */ benchmark.run() } @@ -355,10 +355,10 @@ object OrcReadBenchmark { String Dictionary: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1340 / 1343 7.8 127.8 1.0X - Native ORC Vectorized 311 / 339 33.7 29.7 4.3X - Native ORC Vectorized (Java) 365 / 374 28.7 34.8 3.7X - Hive built-in ORC 2020 / 2037 5.2 192.7 0.7X + Native ORC MR 1403 / 1407 7.5 133.8 1.0X + Native ORC Vectorized 349 / 360 30.1 33.2 4.0X + Native ORC Vectorized (Java) 360 / 366 29.1 34.3 3.9X + Hive built-in ORC 1950 / 1974 5.4 185.9 0.7X */ benchmark.run() } @@ -410,24 +410,24 @@ object OrcReadBenchmark { String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR (0.0%) 2515 / 2521 4.2 239.9 1.0X - Native ORC Vectorized (0.0%) 972 / 989 10.8 92.7 2.6X - Native ORC Vectorized (0.0%) (Java) 1040 / 1052 10.1 99.2 2.4X - Hive built-in ORC (0.0%) 4021 / 4035 2.6 383.5 0.6X + Native ORC MR (0.0%) 2548 / 2586 4.1 243.0 1.0X + Native ORC Vectorized (0.0%) 965 / 972 10.9 92.0 2.6X + Native ORC Vectorized (0.0%) (Java) 942 / 953 11.1 89.9 2.7X + Hive built-in ORC (0.0%) 3899 / 3930 2.7 371.8 0.7X String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR (0.5%) 2421 / 2433 4.3 230.9 1.0X - Native ORC Vectorized (0.5%) 1238 / 1249 8.5 118.1 2.0X - Native ORC Vectorized (0.5%) (Java) 1241 / 1260 8.4 118.4 2.0X - Hive built-in ORC (0.5%) 3004 / 3008 3.5 286.5 0.8X + Native ORC MR (0.5%) 2431 / 2435 4.3 231.9 1.0X + Native ORC Vectorized (0.5%) 1220 / 1225 8.6 116.3 2.0X + Native ORC Vectorized (0.5%) (Java) 1257 / 1261 8.3 119.9 1.9X + Hive built-in ORC (0.5%) 2944 / 2956 3.6 280.8 0.8X String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR (0.95%) 1294 / 1318 8.1 123.4 1.0X - Native ORC Vectorized (0.95%) 448 / 454 23.4 42.7 2.9X - Native ORC Vectorized (0.95%) (Java) 441 / 456 23.8 42.1 2.9X - Hive built-in ORC (0.95%) 1660 / 1685 6.3 158.4 0.8X + Native ORC MR (0.95%) 1281 / 1288 8.2 122.2 1.0X + Native ORC Vectorized (0.95%) 430 / 434 24.4 41.0 3.0X + Native ORC Vectorized (0.95%) (Java) 490 / 498 21.4 46.7 2.6X + Hive built-in ORC (0.95%) 1641 / 1661 6.4 156.5 0.8X */ benchmark.run() } @@ -475,24 +475,24 @@ object OrcReadBenchmark { SQL Single Column Scan from wide table (100 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1136 / 1137 0.9 1083.6 1.0X - Native ORC Vectorized 90 / 98 11.7 85.7 12.6X - Native ORC Vectorized (Java) 91 / 98 11.5 86.8 12.5X - Hive built-in ORC 379 / 389 2.8 361.2 3.0X + Native ORC MR 1116 / 1124 0.9 1064.1 1.0X + Native ORC Vectorized 91 / 99 11.5 87.0 12.2X + Native ORC Vectorized (Java) 91 / 98 11.5 87.0 12.2X + Hive built-in ORC 383 / 389 2.7 364.9 2.9X SQL Single Column Scan from wide table (200 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2269 / 2295 0.5 2164.0 1.0X - Native ORC Vectorized 158 / 164 6.7 150.2 14.4X - Native ORC Vectorized (Java) 158 / 166 6.6 150.6 14.4X - Hive built-in ORC 602 / 606 1.7 574.2 3.8X + Native ORC MR 2277 / 2283 0.5 2171.2 1.0X + Native ORC Vectorized 158 / 166 6.6 150.5 14.4X + Native ORC Vectorized (Java) 157 / 165 6.7 149.4 14.5X + Hive built-in ORC 588 / 599 1.8 560.8 3.9X SQL Single Column Scan from wide table (300 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 3414 / 3433 0.3 3255.5 1.0X - Native ORC Vectorized 266 / 279 3.9 253.5 12.8X - Native ORC Vectorized (Java) 262 / 274 4.0 249.7 13.0X - Hive built-in ORC 848 / 850 1.2 808.6 4.0X + Native ORC MR 3370 / 3382 0.3 3214.1 1.0X + Native ORC Vectorized 265 / 279 4.0 252.8 12.7X + Native ORC Vectorized (Java) 266 / 277 3.9 253.7 12.7X + Hive built-in ORC 845 / 856 1.2 806.0 4.0X */ sqlBenchmark.run() } From 15cac9cf6b99415b03fc818fbb14a16b722c9058 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 8 Jan 2018 03:04:48 -0800 Subject: [PATCH 13/23] Use `putArray`. --- .../orc/JavaOrcColumnarBatchReader.java | 8 +- .../orc/OrcColumnarBatchReader.scala | 522 +++++++++--------- .../spark/sql/hive/orc/OrcReadBenchmark.scala | 136 ++--- 3 files changed, 346 insertions(+), 320 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java index 39ea61b1133cb..a68575652f7ca 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java @@ -212,7 +212,6 @@ private boolean nextBatch() throws IOException { if (requestedColIds[i] >= 0) { ColumnVector fromColumn = batch.cols[requestedColIds[i]]; - toColumn.reserve(batchSize); if (fromColumn.isRepeating) { putRepeatingValues(batchSize, field, fromColumn, toColumn); @@ -314,8 +313,9 @@ private void putNonNullValues( toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0); } else if (type instanceof StringType || type instanceof BinaryType) { BytesColumnVector data = ((BytesColumnVector)fromColumn); + toColumn.putByteArray(0, data.vector[0]); for (int index = 0; index < batchSize; index++) { - toColumn.putByteArray(index, data.vector[index], data.start[index], data.length[index]); + toColumn.putArray(index, data.start[index], data.length[index]); } } else if (type instanceof DecimalType) { DecimalType decimalType = (DecimalType)type; @@ -413,12 +413,12 @@ private void putValues( } } else if (type instanceof StringType || type instanceof BinaryType) { BytesColumnVector vector = (BytesColumnVector)fromColumn; + toColumn.putByteArray(0, vector.vector[0]); for (int index = 0; index < batchSize; index++) { if (fromColumn.isNull[index]) { toColumn.putNull(index); } else { - toColumn.putByteArray( - index, vector.vector[index], vector.start[index], vector.length[index]); + toColumn.putArray(index, vector.start[index], vector.length[index]); } } } else if (type instanceof DecimalType) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala index 6d32788aced2f..57a08277dabac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala @@ -26,9 +26,10 @@ import org.apache.orc.storage.serde2.io.HiveDecimalWritable import org.apache.spark.memory.MemoryMode import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.vectorized._ +import org.apache.spark.sql.execution.vectorized.{ColumnVectorUtils, OffHeapColumnVector, + OnHeapColumnVector, WritableColumnVector} import org.apache.spark.sql.types._ -import org.apache.spark.sql.vectorized._ +import org.apache.spark.sql.vectorized.ColumnarBatch /** @@ -189,265 +190,290 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat while (i < requiredFields.length) { val field = requiredFields(i) val toColumn = columnVectors(i) - toColumn.reserve(batchSize) if (requestedColIds(i) >= 0) { val fromColumn = batch.cols(requestedColIds(i)) if (fromColumn.isRepeating) { - if (fromColumn.isNull(0)) { - toColumn.putNulls(0, batchSize) - } else { - field.dataType match { - case BooleanType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1 - toColumn.putBooleans(0, batchSize, data) - - case ByteType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte - toColumn.putBytes(0, batchSize, data) - case ShortType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort - toColumn.putShorts(0, batchSize, data) - case IntegerType | DateType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt - toColumn.putInts(0, batchSize, data) - case LongType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) - toColumn.putLongs(0, batchSize, data) - - case TimestampType => - val data = fromColumn.asInstanceOf[TimestampColumnVector] - toColumn.putLongs(0, batchSize, fromTimestampColumnVector(data, 0)) - - case FloatType => - val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0).toFloat - toColumn.putFloats(0, batchSize, data) - case DoubleType => - val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0) - toColumn.putDoubles(0, batchSize, data) - - case StringType | BinaryType => - val data = fromColumn.asInstanceOf[BytesColumnVector] - var index = 0 - while (index < batchSize) { - toColumn.putByteArray(index, data.vector(0), data.start(0), data.length(0)) - index += 1 - } - - case DecimalType.Fixed(precision, scale) => - val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(0) - putDecimalWritables(toColumn, batchSize, precision, scale, d) - - case dt => - throw new UnsupportedOperationException(s"Unsupported Data Type: $dt") - } - } + putRepeatingValues(batchSize, field, fromColumn, toColumn) } else if (fromColumn.noNulls) { - field.dataType match { - case BooleanType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector - var index = 0 - while (index < batchSize) { - toColumn.putBoolean(index, data(index) == 1) - index += 1 - } - - case ByteType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector - var index = 0 - while (index < batchSize) { - toColumn.putByte(index, data(index).toByte) - index += 1 - } - case ShortType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector - var index = 0 - while (index < batchSize) { - toColumn.putShort(index, data(index).toShort) - index += 1 - } - case IntegerType | DateType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector - var index = 0 - while (index < batchSize) { - toColumn.putInt(index, data(index).toInt) - index += 1 - } - case LongType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector - toColumn.putLongs(0, batchSize, data, 0) - - case TimestampType => - val data = fromColumn.asInstanceOf[TimestampColumnVector] - var index = 0 - while (index < batchSize) { - toColumn.putLong(index, fromTimestampColumnVector(data, index)) - index += 1 - } - - case FloatType => - val data = fromColumn.asInstanceOf[DoubleColumnVector].vector - var index = 0 - while (index < batchSize) { - toColumn.putFloat(index, data(index).toFloat) - index += 1 - } - case DoubleType => - val data = fromColumn.asInstanceOf[DoubleColumnVector].vector - toColumn.putDoubles(0, batchSize, data, 0) - - case StringType | BinaryType => - val data = fromColumn.asInstanceOf[BytesColumnVector] - var index = 0 - while (index < batchSize) { - toColumn.putByteArray(0, data.vector(index), data.start(index), data.length(index)) - index += 1 - } - - case DecimalType.Fixed(precision, scale) => - val data = fromColumn.asInstanceOf[DecimalColumnVector] - var index = 0 - while (index < batchSize) { - putDecimalWritable(toColumn, index, precision, scale, data.vector(index)) - index += 1 - } - - case dt => - throw new UnsupportedOperationException(s"Unsupported Data Type: $dt") - } + putNonNullValues(batchSize, field, fromColumn, toColumn) } else { - field.dataType match { - case BooleanType => - val vector = fromColumn.asInstanceOf[LongColumnVector].vector - var index = 0 - while (index < batchSize) { - if (fromColumn.isNull(index)) { - toColumn.putNull(index) - } else { - toColumn.putBoolean(index, vector(index) == 1) - } - index += 1 - } - - case ByteType => - val vector = fromColumn.asInstanceOf[LongColumnVector].vector - var index = 0 - while (index < batchSize) { - if (fromColumn.isNull(index)) { - toColumn.putNull(index) - } else { - toColumn.putByte(index, vector(index).toByte) - } - index += 1 - } - - case ShortType => - val vector = fromColumn.asInstanceOf[LongColumnVector].vector - var index = 0 - while (index < batchSize) { - if (fromColumn.isNull(index)) { - toColumn.putNull(index) - } else { - toColumn.putShort(index, vector(index).toShort) - } - index += 1 - } - - case IntegerType | DateType => - val vector = fromColumn.asInstanceOf[LongColumnVector].vector - var index = 0 - while (index < batchSize) { - if (fromColumn.isNull(index)) { - toColumn.putNull(index) - } else { - toColumn.putInt(index, vector(index).toInt) - } - index += 1 - } - - case LongType => - val vector = fromColumn.asInstanceOf[LongColumnVector].vector - var index = 0 - while (index < batchSize) { - if (fromColumn.isNull(index)) { - toColumn.putNull(index) - } else { - toColumn.putLong(index, vector(index)) - } - index += 1 - } - - case TimestampType => - val vector = fromColumn.asInstanceOf[TimestampColumnVector] - var index = 0 - while (index < batchSize) { - if (fromColumn.isNull(index)) { - toColumn.putNull(index) - } else { - toColumn.putLong(index, fromTimestampColumnVector(vector, index)) - } - index += 1 - } - - case FloatType => - val vector = fromColumn.asInstanceOf[DoubleColumnVector].vector - var index = 0 - while (index < batchSize) { - if (fromColumn.isNull(index)) { - toColumn.putNull(index) - } else { - toColumn.putFloat(index, vector(index).toFloat) - } - index += 1 - } - - case DoubleType => - val vector = fromColumn.asInstanceOf[DoubleColumnVector].vector - var index = 0 - while (index < batchSize) { - if (fromColumn.isNull(index)) { - toColumn.putNull(index) - } else { - toColumn.putDouble(index, vector(index)) - } - index += 1 - } - - case StringType | BinaryType => - val vector = fromColumn.asInstanceOf[BytesColumnVector] - var index = 0 - while (index < batchSize) { - if (fromColumn.isNull(index)) { - toColumn.putNull(index) - } else { - toColumn.putByteArray( - index, vector.vector(index), vector.start(index), vector.length(index)) - } - index += 1 - } - - case DecimalType.Fixed(precision, scale) => - val vector = fromColumn.asInstanceOf[DecimalColumnVector].vector - var index = 0 - while (index < batchSize) { - if (fromColumn.isNull(index)) { - toColumn.putNull(index) - } else { - putDecimalWritable(toColumn, index, precision, scale, vector(index)) - } - index += 1 - } - - case dt => - throw new UnsupportedOperationException(s"Unsupported Data Type: $dt") - } + putValues(batchSize, field, fromColumn, toColumn) } } i += 1 } true } + + private def putRepeatingValues( + batchSize: Int, + field: StructField, + fromColumn: ColumnVector, + toColumn: WritableColumnVector) : Unit = { + if (fromColumn.isNull(0)) { + toColumn.putNulls(0, batchSize) + } else { + field.dataType match { + case BooleanType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1 + toColumn.putBooleans(0, batchSize, data) + + case ByteType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte + toColumn.putBytes(0, batchSize, data) + case ShortType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort + toColumn.putShorts(0, batchSize, data) + case IntegerType | DateType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt + toColumn.putInts(0, batchSize, data) + case LongType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) + toColumn.putLongs(0, batchSize, data) + + case TimestampType => + val data = fromColumn.asInstanceOf[TimestampColumnVector] + toColumn.putLongs(0, batchSize, fromTimestampColumnVector(data, 0)) + + case FloatType => + val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0).toFloat + toColumn.putFloats(0, batchSize, data) + case DoubleType => + val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0) + toColumn.putDoubles(0, batchSize, data) + + case StringType | BinaryType => + val data = fromColumn.asInstanceOf[BytesColumnVector] + toColumn.putByteArray(0, data.vector(0)) + var index = 0 + while (index < batchSize) { + toColumn.putArray(index, data.start(index), data.length(index)) + index += 1 + } + + case DecimalType.Fixed(precision, scale) => + val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(0) + putDecimalWritables(toColumn, batchSize, precision, scale, d) + + case dt => + throw new UnsupportedOperationException(s"Unsupported Data Type: $dt") + } + } + } + + private def putNonNullValues( + batchSize: Int, + field: StructField, + fromColumn: ColumnVector, + toColumn: WritableColumnVector) : Unit = { + field.dataType match { + case BooleanType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector + var index = 0 + while (index < batchSize) { + toColumn.putBoolean(index, data(index) == 1) + index += 1 + } + + case ByteType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector + var index = 0 + while (index < batchSize) { + toColumn.putByte(index, data(index).toByte) + index += 1 + } + case ShortType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector + var index = 0 + while (index < batchSize) { + toColumn.putShort(index, data(index).toShort) + index += 1 + } + case IntegerType | DateType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector + var index = 0 + while (index < batchSize) { + toColumn.putInt(index, data(index).toInt) + index += 1 + } + case LongType => + val data = fromColumn.asInstanceOf[LongColumnVector].vector + toColumn.putLongs(0, batchSize, data, 0) + + case TimestampType => + val data = fromColumn.asInstanceOf[TimestampColumnVector] + var index = 0 + while (index < batchSize) { + toColumn.putLong(index, fromTimestampColumnVector(data, index)) + index += 1 + } + + case FloatType => + val data = fromColumn.asInstanceOf[DoubleColumnVector].vector + var index = 0 + while (index < batchSize) { + toColumn.putFloat(index, data(index).toFloat) + index += 1 + } + case DoubleType => + val data = fromColumn.asInstanceOf[DoubleColumnVector].vector + toColumn.putDoubles(0, batchSize, data, 0) + + case StringType | BinaryType => + val data = fromColumn.asInstanceOf[BytesColumnVector] + var index = 0 + toColumn.putByteArray(0, data.vector(0)) + while (index < batchSize) { + toColumn.putArray(index, data.start(index), data.length(index)) + index += 1 + } + + case DecimalType.Fixed(precision, scale) => + val data = fromColumn.asInstanceOf[DecimalColumnVector] + var index = 0 + while (index < batchSize) { + putDecimalWritable(toColumn, index, precision, scale, data.vector(index)) + index += 1 + } + + case dt => + throw new UnsupportedOperationException(s"Unsupported Data Type: $dt") + } + } + + private def putValues( + batchSize: Int, + field: StructField, + fromColumn: ColumnVector, + toColumn: WritableColumnVector) : Unit = { + field.dataType match { + case BooleanType => + val vector = fromColumn.asInstanceOf[LongColumnVector].vector + var index = 0 + while (index < batchSize) { + if (fromColumn.isNull(index)) { + toColumn.putNull(index) + } else { + toColumn.putBoolean(index, vector(index) == 1) + } + index += 1 + } + + case ByteType => + val vector = fromColumn.asInstanceOf[LongColumnVector].vector + var index = 0 + while (index < batchSize) { + if (fromColumn.isNull(index)) { + toColumn.putNull(index) + } else { + toColumn.putByte(index, vector(index).toByte) + } + index += 1 + } + + case ShortType => + val vector = fromColumn.asInstanceOf[LongColumnVector].vector + var index = 0 + while (index < batchSize) { + if (fromColumn.isNull(index)) { + toColumn.putNull(index) + } else { + toColumn.putShort(index, vector(index).toShort) + } + index += 1 + } + + case IntegerType | DateType => + val vector = fromColumn.asInstanceOf[LongColumnVector].vector + var index = 0 + while (index < batchSize) { + if (fromColumn.isNull(index)) { + toColumn.putNull(index) + } else { + toColumn.putInt(index, vector(index).toInt) + } + index += 1 + } + + case LongType => + val vector = fromColumn.asInstanceOf[LongColumnVector].vector + var index = 0 + while (index < batchSize) { + if (fromColumn.isNull(index)) { + toColumn.putNull(index) + } else { + toColumn.putLong(index, vector(index)) + } + index += 1 + } + + case TimestampType => + val vector = fromColumn.asInstanceOf[TimestampColumnVector] + var index = 0 + while (index < batchSize) { + if (fromColumn.isNull(index)) { + toColumn.putNull(index) + } else { + toColumn.putLong(index, fromTimestampColumnVector(vector, index)) + } + index += 1 + } + + case FloatType => + val vector = fromColumn.asInstanceOf[DoubleColumnVector].vector + var index = 0 + while (index < batchSize) { + if (fromColumn.isNull(index)) { + toColumn.putNull(index) + } else { + toColumn.putFloat(index, vector(index).toFloat) + } + index += 1 + } + + case DoubleType => + val vector = fromColumn.asInstanceOf[DoubleColumnVector].vector + var index = 0 + while (index < batchSize) { + if (fromColumn.isNull(index)) { + toColumn.putNull(index) + } else { + toColumn.putDouble(index, vector(index)) + } + index += 1 + } + + case StringType | BinaryType => + val vector = fromColumn.asInstanceOf[BytesColumnVector] + toColumn.putByteArray(0, vector.vector(0)) + var index = 0 + while (index < batchSize) { + if (fromColumn.isNull(index)) { + toColumn.putNull(index) + } else { + toColumn.putArray(index, vector.start(index), vector.length(index)) + } + index += 1 + } + + case DecimalType.Fixed(precision, scale) => + val vector = fromColumn.asInstanceOf[DecimalColumnVector].vector + var index = 0 + while (index < batchSize) { + if (fromColumn.isNull(index)) { + toColumn.putNull(index) + } else { + putDecimalWritable(toColumn, index, precision, scale, vector(index)) + } + index += 1 + } + + case dt => + throw new UnsupportedOperationException(s"Unsupported Data Type: $dt") + } + } } /** diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala index dc73c10ed7817..175201eca51cd 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala @@ -123,45 +123,45 @@ object OrcReadBenchmark { SQL Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1198 / 1236 13.1 76.2 1.0X - Native ORC Vectorized 163 / 170 96.4 10.4 7.3X - Native ORC Vectorized (Java) 156 / 161 101.1 9.9 7.7X - Hive built-in ORC 1508 / 1511 10.4 95.8 0.8X + Native ORC MR 1161 / 1168 13.5 73.8 1.0X + Native ORC Vectorized 163 / 171 96.3 10.4 7.1X + Native ORC Vectorized (Java) 155 / 163 101.6 9.8 7.5X + Hive built-in ORC 1427 / 1427 11.0 90.7 0.8X SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1361 / 1391 11.6 86.5 1.0X - Native ORC Vectorized 197 / 204 80.0 12.5 6.9X - Native ORC Vectorized (Java) 198 / 203 79.5 12.6 6.9X - Hive built-in ORC 1670 / 1680 9.4 106.2 0.8X + Native ORC MR 1261 / 1321 12.5 80.2 1.0X + Native ORC Vectorized 160 / 167 98.2 10.2 7.9X + Native ORC Vectorized (Java) 160 / 167 98.4 10.2 7.9X + Hive built-in ORC 1655 / 1687 9.5 105.2 0.8X SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1643 / 1829 9.6 104.5 1.0X - Native ORC Vectorized 232 / 245 67.9 14.7 7.1X - Native ORC Vectorized (Java) 225 / 232 69.9 14.3 7.3X - Hive built-in ORC 1978 / 1988 8.0 125.8 0.8X + Native ORC MR 1369 / 1449 11.5 87.1 1.0X + Native ORC Vectorized 263 / 277 59.8 16.7 5.2X + Native ORC Vectorized (Java) 225 / 237 70.0 14.3 6.1X + Hive built-in ORC 1867 / 1899 8.4 118.7 0.7X SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1504 / 1522 10.5 95.7 1.0X - Native ORC Vectorized 288 / 298 54.6 18.3 5.2X - Native ORC Vectorized (Java) 281 / 294 55.9 17.9 5.3X - Hive built-in ORC 1913 / 1925 8.2 121.6 0.8X + Native ORC MR 1472 / 1474 10.7 93.6 1.0X + Native ORC Vectorized 289 / 300 54.5 18.4 5.1X + Native ORC Vectorized (Java) 286 / 294 54.9 18.2 5.1X + Hive built-in ORC 1917 / 1934 8.2 121.9 0.8X SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1452 / 1498 10.8 92.3 1.0X - Native ORC Vectorized 324 / 329 48.5 20.6 4.5X - Native ORC Vectorized (Java) 323 / 329 48.7 20.5 4.5X - Hive built-in ORC 1945 / 2013 8.1 123.7 0.7X + Native ORC MR 1484 / 1484 10.6 94.3 1.0X + Native ORC Vectorized 365 / 370 43.1 23.2 4.1X + Native ORC Vectorized (Java) 326 / 335 48.2 20.7 4.5X + Hive built-in ORC 1978 / 2049 8.0 125.8 0.8X SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1548 / 1551 10.2 98.4 1.0X - Native ORC Vectorized 394 / 399 39.9 25.0 3.9X - Native ORC Vectorized (Java) 396 / 406 39.8 25.1 3.9X - Hive built-in ORC 2079 / 2098 7.6 132.2 0.7X + Native ORC MR 1550 / 1554 10.1 98.6 1.0X + Native ORC Vectorized 396 / 405 39.7 25.2 3.9X + Native ORC Vectorized (Java) 394 / 402 39.9 25.1 3.9X + Hive built-in ORC 2072 / 2084 7.6 131.8 0.7X */ sqlBenchmark.run() } @@ -208,10 +208,10 @@ object OrcReadBenchmark { Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2859 / 2860 3.7 272.6 1.0X - Native ORC Vectorized 1223 / 1227 8.6 116.6 2.3X - Native ORC Vectorized (Java) 1337 / 1339 7.8 127.5 2.1X - Hive built-in ORC 3838 / 3893 2.7 366.0 0.7X + Native ORC MR 2636 / 2734 4.0 251.4 1.0X + Native ORC Vectorized 1267 / 1267 8.3 120.9 2.1X + Native ORC Vectorized (Java) 1182 / 1183 8.9 112.7 2.2X + Hive built-in ORC 3724 / 3764 2.8 355.2 0.7X */ benchmark.run() } @@ -300,18 +300,18 @@ object OrcReadBenchmark { Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------ - Read data column - Native ORC MR 1519 / 1546 10.4 96.6 1.0X - Read data column - Native ORC Vectorized 293 / 301 53.7 18.6 5.2X - Read data column - Native ORC Vectorized (Java) 295 / 300 53.3 18.8 5.1X - Read data column - Hive built-in ORC 2120 / 2163 7.4 134.8 0.7X - Read partition column - Native ORC MR 1077 / 1084 14.6 68.5 1.4X - Read partition column - Native ORC Vectorized 51 / 55 305.6 3.3 29.5X - Read partition column - Native ORC Vectorized (Java) 52 / 55 301.2 3.3 29.1X - Read partition column - Hive built-in ORC 1299 / 1302 12.1 82.6 1.2X - Read both columns - Native ORC MR 1699 / 1720 9.3 108.0 0.9X - Read both columns - Native ORC Vectorized 327 / 333 48.1 20.8 4.6X - Read both columns - Native ORC Vectorized (Java) 330 / 335 47.6 21.0 4.6X - Read both columns - Hive built-in ORC 2149 / 2152 7.3 136.6 0.7X + Read data column - Native ORC MR 1587 / 1592 9.9 100.9 1.0X + Read data column - Native ORC Vectorized 290 / 309 54.3 18.4 5.5X + Read data column - Native ORC Vectorized (Java) 293 / 297 53.7 18.6 5.4X + Read data column - Hive built-in ORC 2204 / 2214 7.1 140.1 0.7X + Read partition column - Native ORC MR 1078 / 1097 14.6 68.5 1.5X + Read partition column - Native ORC Vectorized 53 / 56 294.0 3.4 29.7X + Read partition column - Native ORC Vectorized (Java) 52 / 55 300.7 3.3 30.4X + Read partition column - Hive built-in ORC 1279 / 1287 12.3 81.3 1.2X + Read both columns - Native ORC MR 1665 / 1674 9.4 105.9 1.0X + Read both columns - Native ORC Vectorized 327 / 333 48.0 20.8 4.8X + Read both columns - Native ORC Vectorized (Java) 327 / 332 48.2 20.8 4.9X + Read both columns - Hive built-in ORC 2157 / 2169 7.3 137.1 0.7X */ benchmark.run() } @@ -355,10 +355,10 @@ object OrcReadBenchmark { String Dictionary: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1403 / 1407 7.5 133.8 1.0X - Native ORC Vectorized 349 / 360 30.1 33.2 4.0X - Native ORC Vectorized (Java) 360 / 366 29.1 34.3 3.9X - Hive built-in ORC 1950 / 1974 5.4 185.9 0.7X + Native ORC MR 1355 / 1355 7.7 129.2 1.0X + Native ORC Vectorized 262 / 270 40.0 25.0 5.2X + Native ORC Vectorized (Java) 223 / 227 46.9 21.3 6.1X + Hive built-in ORC 2017 / 2027 5.2 192.4 0.7X */ benchmark.run() } @@ -410,24 +410,24 @@ object OrcReadBenchmark { String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR (0.0%) 2548 / 2586 4.1 243.0 1.0X - Native ORC Vectorized (0.0%) 965 / 972 10.9 92.0 2.6X - Native ORC Vectorized (0.0%) (Java) 942 / 953 11.1 89.9 2.7X - Hive built-in ORC (0.0%) 3899 / 3930 2.7 371.8 0.7X + Native ORC MR (0.0%) 2575 / 2618 4.1 245.5 1.0X + Native ORC Vectorized (0.0%) 841 / 852 12.5 80.2 3.1X + Native ORC Vectorized (0.0%) (Java) 757 / 760 13.9 72.2 3.4X + Hive built-in ORC (0.0%) 4149 / 4162 2.5 395.7 0.6X String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR (0.5%) 2431 / 2435 4.3 231.9 1.0X - Native ORC Vectorized (0.5%) 1220 / 1225 8.6 116.3 2.0X - Native ORC Vectorized (0.5%) (Java) 1257 / 1261 8.3 119.9 1.9X - Hive built-in ORC (0.5%) 2944 / 2956 3.6 280.8 0.8X + Native ORC MR (0.5%) 2446 / 2460 4.3 233.3 1.0X + Native ORC Vectorized (0.5%) 1081 / 1084 9.7 103.1 2.3X + Native ORC Vectorized (0.5%) (Java) 1066 / 1069 9.8 101.6 2.3X + Hive built-in ORC (0.5%) 2928 / 2938 3.6 279.2 0.8X String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR (0.95%) 1281 / 1288 8.2 122.2 1.0X - Native ORC Vectorized (0.95%) 430 / 434 24.4 41.0 3.0X - Native ORC Vectorized (0.95%) (Java) 490 / 498 21.4 46.7 2.6X - Hive built-in ORC (0.95%) 1641 / 1661 6.4 156.5 0.8X + Native ORC MR (0.95%) 1287 / 1331 8.2 122.7 1.0X + Native ORC Vectorized (0.95%) 404 / 407 26.0 38.5 3.2X + Native ORC Vectorized (0.95%) (Java) 405 / 409 25.9 38.6 3.2X + Hive built-in ORC (0.95%) 1612 / 1644 6.5 153.7 0.8X */ benchmark.run() } @@ -475,24 +475,24 @@ object OrcReadBenchmark { SQL Single Column Scan from wide table (100 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1116 / 1124 0.9 1064.1 1.0X - Native ORC Vectorized 91 / 99 11.5 87.0 12.2X - Native ORC Vectorized (Java) 91 / 98 11.5 87.0 12.2X - Hive built-in ORC 383 / 389 2.7 364.9 2.9X + Native ORC MR 1116 / 1117 0.9 1064.7 1.0X + Native ORC Vectorized 92 / 98 11.3 88.2 12.1X + Native ORC Vectorized (Java) 90 / 96 11.6 86.0 12.4X + Hive built-in ORC 376 / 386 2.8 358.6 3.0X SQL Single Column Scan from wide table (200 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2277 / 2283 0.5 2171.2 1.0X - Native ORC Vectorized 158 / 166 6.6 150.5 14.4X - Native ORC Vectorized (Java) 157 / 165 6.7 149.4 14.5X - Hive built-in ORC 588 / 599 1.8 560.8 3.9X + Native ORC MR 2268 / 2283 0.5 2162.6 1.0X + Native ORC Vectorized 158 / 165 6.6 150.5 14.4X + Native ORC Vectorized (Java) 155 / 167 6.8 147.8 14.6X + Hive built-in ORC 597 / 601 1.8 569.0 3.8X SQL Single Column Scan from wide table (300 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 3370 / 3382 0.3 3214.1 1.0X - Native ORC Vectorized 265 / 279 4.0 252.8 12.7X - Native ORC Vectorized (Java) 266 / 277 3.9 253.7 12.7X - Hive built-in ORC 845 / 856 1.2 806.0 4.0X + Native ORC MR 3377 / 3387 0.3 3220.7 1.0X + Native ORC Vectorized 274 / 282 3.8 261.7 12.3X + Native ORC Vectorized (Java) 269 / 277 3.9 256.7 12.5X + Hive built-in ORC 845 / 858 1.2 805.8 4.0X */ sqlBenchmark.run() } From e3d6342233a5eacdbcb90c0d53ab094a27ef8130 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 8 Jan 2018 03:49:13 -0800 Subject: [PATCH 14/23] Revert putArray bacause BytesColumnVector doesn't use a single base byte array. --- .../datasources/orc/JavaOrcColumnarBatchReader.java | 7 +++---- .../execution/datasources/orc/OrcColumnarBatchReader.scala | 7 +++---- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java index a68575652f7ca..8427cc8676395 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java @@ -313,9 +313,8 @@ private void putNonNullValues( toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0); } else if (type instanceof StringType || type instanceof BinaryType) { BytesColumnVector data = ((BytesColumnVector)fromColumn); - toColumn.putByteArray(0, data.vector[0]); for (int index = 0; index < batchSize; index++) { - toColumn.putArray(index, data.start[index], data.length[index]); + toColumn.putByteArray(index, data.vector[index], data.start[index], data.length[index]); } } else if (type instanceof DecimalType) { DecimalType decimalType = (DecimalType)type; @@ -413,12 +412,12 @@ private void putValues( } } else if (type instanceof StringType || type instanceof BinaryType) { BytesColumnVector vector = (BytesColumnVector)fromColumn; - toColumn.putByteArray(0, vector.vector[0]); for (int index = 0; index < batchSize; index++) { if (fromColumn.isNull[index]) { toColumn.putNull(index); } else { - toColumn.putArray(index, vector.start[index], vector.length[index]); + toColumn.putByteArray( + index, vector.vector[index], vector.start[index], vector.length[index]); } } } else if (type instanceof DecimalType) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala index 57a08277dabac..c7d9009723833 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala @@ -324,9 +324,8 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat case StringType | BinaryType => val data = fromColumn.asInstanceOf[BytesColumnVector] var index = 0 - toColumn.putByteArray(0, data.vector(0)) while (index < batchSize) { - toColumn.putArray(index, data.start(index), data.length(index)) + toColumn.putByteArray(index, data.vector(index), data.start(index), data.length(index)) index += 1 } @@ -447,13 +446,13 @@ private[orc] class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBat case StringType | BinaryType => val vector = fromColumn.asInstanceOf[BytesColumnVector] - toColumn.putByteArray(0, vector.vector(0)) var index = 0 while (index < batchSize) { if (fromColumn.isNull(index)) { toColumn.putNull(index) } else { - toColumn.putArray(index, vector.start(index), vector.length(index)) + toColumn.putByteArray( + index, vector.vector(index), vector.start(index), vector.length(index)) } index += 1 } From 10e5d7a4bbac748019508fe3104e48c392696d9f Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 8 Jan 2018 04:27:02 -0800 Subject: [PATCH 15/23] Remove scala version --- .../apache/spark/sql/internal/SQLConf.scala | 7 - ...eader.java => OrcColumnarBatchReader.java} | 4 +- .../orc/OrcColumnarBatchReader.scala | 547 ------------------ .../datasources/orc/OrcFileFormat.scala | 33 +- .../spark/sql/hive/orc/OrcReadBenchmark.scala | 204 ++----- 5 files changed, 72 insertions(+), 723 deletions(-) rename sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/{JavaOrcColumnarBatchReader.java => OrcColumnarBatchReader.java} (99%) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 9451f85b57a4f..74949db883f7a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -391,11 +391,6 @@ object SQLConf { .booleanConf .createWithDefault(true) - val ORC_VECTORIZED_JAVA_READER_ENABLED = buildConf("spark.sql.orc.enableVectorizedJavaReader") - .doc("Enables vectorized java-version orc decoding.") - .booleanConf - .createWithDefault(true) - val ORC_FILTER_PUSHDOWN_ENABLED = buildConf("spark.sql.orc.filterPushdown") .doc("When true, enable filter pushdown for ORC files.") .booleanConf @@ -1195,8 +1190,6 @@ class SQLConf extends Serializable with Logging { def orcVectorizedReaderEnabled: Boolean = getConf(ORC_VECTORIZED_READER_ENABLED) - def orcVectorizedJavaReaderEnabled: Boolean = getConf(ORC_VECTORIZED_JAVA_READER_ENABLED) - def parquetCompressionCodec: String = getConf(PARQUET_COMPRESSION) def parquetVectorizedReaderEnabled: Boolean = getConf(PARQUET_VECTORIZED_READER_ENABLED) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java similarity index 99% rename from sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java rename to sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java index 8427cc8676395..cc01dfb4f3ef1 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/JavaOrcColumnarBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java @@ -47,7 +47,7 @@ * To support vectorization in WholeStageCodeGen, this reader returns ColumnarBatch. * After creating, `initialize` and `initBatch` should be called sequentially. */ -public class JavaOrcColumnarBatchReader extends RecordReader { +public class OrcColumnarBatchReader extends RecordReader { // ORC File Reader private Reader reader; @@ -82,7 +82,7 @@ public class JavaOrcColumnarBatchReader extends RecordReader 0) { - val partitionIdx = requiredFields.length - for (i <- 0 until partitionValues.numFields) { - ColumnVectorUtils.populate(columnVectors(i + partitionIdx), partitionValues, i) - columnVectors(i + partitionIdx).setIsConstant() - } - } - - // Initialize the missing columns once. - for (i <- 0 until requiredFields.length) { - if (requestedColIds(i) < 0) { - columnVectors(i).putNulls(0, columnarBatch.capacity) - columnVectors(i).setIsConstant() - } - } - } - - /** - * Return true if there exists more data in the next batch. If exists, prepare the next batch - * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns. - */ - private def nextBatch(): Boolean = { - if (rowsReturned >= totalRowCount) { - return false - } - - recordReader.nextBatch(batch) - assert(!batch.selectedInUse, "`selectdInUse` is not used and all rows are qualified.") - val batchSize = batch.size - if (batchSize == 0) { - return false - } - rowsReturned += batchSize - columnVectors.foreach(_.reset) - columnarBatch.setNumRows(batchSize) - - var i = 0 - while (i < requiredFields.length) { - val field = requiredFields(i) - val toColumn = columnVectors(i) - - if (requestedColIds(i) >= 0) { - val fromColumn = batch.cols(requestedColIds(i)) - - if (fromColumn.isRepeating) { - putRepeatingValues(batchSize, field, fromColumn, toColumn) - } else if (fromColumn.noNulls) { - putNonNullValues(batchSize, field, fromColumn, toColumn) - } else { - putValues(batchSize, field, fromColumn, toColumn) - } - } - i += 1 - } - true - } - - private def putRepeatingValues( - batchSize: Int, - field: StructField, - fromColumn: ColumnVector, - toColumn: WritableColumnVector) : Unit = { - if (fromColumn.isNull(0)) { - toColumn.putNulls(0, batchSize) - } else { - field.dataType match { - case BooleanType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) == 1 - toColumn.putBooleans(0, batchSize, data) - - case ByteType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toByte - toColumn.putBytes(0, batchSize, data) - case ShortType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toShort - toColumn.putShorts(0, batchSize, data) - case IntegerType | DateType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector(0).toInt - toColumn.putInts(0, batchSize, data) - case LongType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector(0) - toColumn.putLongs(0, batchSize, data) - - case TimestampType => - val data = fromColumn.asInstanceOf[TimestampColumnVector] - toColumn.putLongs(0, batchSize, fromTimestampColumnVector(data, 0)) - - case FloatType => - val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0).toFloat - toColumn.putFloats(0, batchSize, data) - case DoubleType => - val data = fromColumn.asInstanceOf[DoubleColumnVector].vector(0) - toColumn.putDoubles(0, batchSize, data) - - case StringType | BinaryType => - val data = fromColumn.asInstanceOf[BytesColumnVector] - toColumn.putByteArray(0, data.vector(0)) - var index = 0 - while (index < batchSize) { - toColumn.putArray(index, data.start(index), data.length(index)) - index += 1 - } - - case DecimalType.Fixed(precision, scale) => - val d = fromColumn.asInstanceOf[DecimalColumnVector].vector(0) - putDecimalWritables(toColumn, batchSize, precision, scale, d) - - case dt => - throw new UnsupportedOperationException(s"Unsupported Data Type: $dt") - } - } - } - - private def putNonNullValues( - batchSize: Int, - field: StructField, - fromColumn: ColumnVector, - toColumn: WritableColumnVector) : Unit = { - field.dataType match { - case BooleanType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector - var index = 0 - while (index < batchSize) { - toColumn.putBoolean(index, data(index) == 1) - index += 1 - } - - case ByteType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector - var index = 0 - while (index < batchSize) { - toColumn.putByte(index, data(index).toByte) - index += 1 - } - case ShortType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector - var index = 0 - while (index < batchSize) { - toColumn.putShort(index, data(index).toShort) - index += 1 - } - case IntegerType | DateType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector - var index = 0 - while (index < batchSize) { - toColumn.putInt(index, data(index).toInt) - index += 1 - } - case LongType => - val data = fromColumn.asInstanceOf[LongColumnVector].vector - toColumn.putLongs(0, batchSize, data, 0) - - case TimestampType => - val data = fromColumn.asInstanceOf[TimestampColumnVector] - var index = 0 - while (index < batchSize) { - toColumn.putLong(index, fromTimestampColumnVector(data, index)) - index += 1 - } - - case FloatType => - val data = fromColumn.asInstanceOf[DoubleColumnVector].vector - var index = 0 - while (index < batchSize) { - toColumn.putFloat(index, data(index).toFloat) - index += 1 - } - case DoubleType => - val data = fromColumn.asInstanceOf[DoubleColumnVector].vector - toColumn.putDoubles(0, batchSize, data, 0) - - case StringType | BinaryType => - val data = fromColumn.asInstanceOf[BytesColumnVector] - var index = 0 - while (index < batchSize) { - toColumn.putByteArray(index, data.vector(index), data.start(index), data.length(index)) - index += 1 - } - - case DecimalType.Fixed(precision, scale) => - val data = fromColumn.asInstanceOf[DecimalColumnVector] - var index = 0 - while (index < batchSize) { - putDecimalWritable(toColumn, index, precision, scale, data.vector(index)) - index += 1 - } - - case dt => - throw new UnsupportedOperationException(s"Unsupported Data Type: $dt") - } - } - - private def putValues( - batchSize: Int, - field: StructField, - fromColumn: ColumnVector, - toColumn: WritableColumnVector) : Unit = { - field.dataType match { - case BooleanType => - val vector = fromColumn.asInstanceOf[LongColumnVector].vector - var index = 0 - while (index < batchSize) { - if (fromColumn.isNull(index)) { - toColumn.putNull(index) - } else { - toColumn.putBoolean(index, vector(index) == 1) - } - index += 1 - } - - case ByteType => - val vector = fromColumn.asInstanceOf[LongColumnVector].vector - var index = 0 - while (index < batchSize) { - if (fromColumn.isNull(index)) { - toColumn.putNull(index) - } else { - toColumn.putByte(index, vector(index).toByte) - } - index += 1 - } - - case ShortType => - val vector = fromColumn.asInstanceOf[LongColumnVector].vector - var index = 0 - while (index < batchSize) { - if (fromColumn.isNull(index)) { - toColumn.putNull(index) - } else { - toColumn.putShort(index, vector(index).toShort) - } - index += 1 - } - - case IntegerType | DateType => - val vector = fromColumn.asInstanceOf[LongColumnVector].vector - var index = 0 - while (index < batchSize) { - if (fromColumn.isNull(index)) { - toColumn.putNull(index) - } else { - toColumn.putInt(index, vector(index).toInt) - } - index += 1 - } - - case LongType => - val vector = fromColumn.asInstanceOf[LongColumnVector].vector - var index = 0 - while (index < batchSize) { - if (fromColumn.isNull(index)) { - toColumn.putNull(index) - } else { - toColumn.putLong(index, vector(index)) - } - index += 1 - } - - case TimestampType => - val vector = fromColumn.asInstanceOf[TimestampColumnVector] - var index = 0 - while (index < batchSize) { - if (fromColumn.isNull(index)) { - toColumn.putNull(index) - } else { - toColumn.putLong(index, fromTimestampColumnVector(vector, index)) - } - index += 1 - } - - case FloatType => - val vector = fromColumn.asInstanceOf[DoubleColumnVector].vector - var index = 0 - while (index < batchSize) { - if (fromColumn.isNull(index)) { - toColumn.putNull(index) - } else { - toColumn.putFloat(index, vector(index).toFloat) - } - index += 1 - } - - case DoubleType => - val vector = fromColumn.asInstanceOf[DoubleColumnVector].vector - var index = 0 - while (index < batchSize) { - if (fromColumn.isNull(index)) { - toColumn.putNull(index) - } else { - toColumn.putDouble(index, vector(index)) - } - index += 1 - } - - case StringType | BinaryType => - val vector = fromColumn.asInstanceOf[BytesColumnVector] - var index = 0 - while (index < batchSize) { - if (fromColumn.isNull(index)) { - toColumn.putNull(index) - } else { - toColumn.putByteArray( - index, vector.vector(index), vector.start(index), vector.length(index)) - } - index += 1 - } - - case DecimalType.Fixed(precision, scale) => - val vector = fromColumn.asInstanceOf[DecimalColumnVector].vector - var index = 0 - while (index < batchSize) { - if (fromColumn.isNull(index)) { - toColumn.putNull(index) - } else { - putDecimalWritable(toColumn, index, precision, scale, vector(index)) - } - index += 1 - } - - case dt => - throw new UnsupportedOperationException(s"Unsupported Data Type: $dt") - } - } -} - -/** - * Constants for OrcColumnarBatchReader. - */ -object OrcColumnarBatchReader { - /** - * The default size of batch. We use this value for both ORC and Spark consistently - * because they have different default values like the following. - * - * - ORC's VectorizedRowBatch.DEFAULT_SIZE = 1024 - * - Spark's ColumnarBatch.DEFAULT_BATCH_SIZE = 4 * 1024 - */ - val DEFAULT_SIZE: Int = 4 * 1024 - - /** - * Returns the number of micros since epoch from an element of TimestampColumnVector. - */ - private def fromTimestampColumnVector(vector: TimestampColumnVector, index: Int): Long = - vector.time(index) * 1000L + vector.nanos(index) / 1000L - - /** - * Put a `HiveDecimalWritable` to a `WritableColumnVector`. - */ - private def putDecimalWritable( - toColumn: WritableColumnVector, - index: Int, - precision: Int, - scale: Int, - decimalWritable: HiveDecimalWritable): Unit = { - val decimal = decimalWritable.getHiveDecimal() - val value = Decimal(decimal.bigDecimalValue, decimal.precision(), decimal.scale()) - value.changePrecision(precision, scale) - - if (precision <= Decimal.MAX_INT_DIGITS) { - toColumn.putInt(index, value.toUnscaledLong.toInt) - } else if (precision <= Decimal.MAX_LONG_DIGITS) { - toColumn.putLong(index, value.toUnscaledLong) - } else { - val bytes = value.toJavaBigDecimal.unscaledValue.toByteArray - toColumn.putByteArray(index, bytes, 0, bytes.length) - } - } - - /** - * Put `HiveDecimalWritable`s to a `WritableColumnVector`. - */ - private def putDecimalWritables( - toColumn: WritableColumnVector, - size: Int, - precision: Int, - scale: Int, - decimalWritable: HiveDecimalWritable): Unit = { - val decimal = decimalWritable.getHiveDecimal() - val value = Decimal(decimal.bigDecimalValue, decimal.precision(), decimal.scale()) - value.changePrecision(precision, scale) - - if (precision <= Decimal.MAX_INT_DIGITS) { - toColumn.putInts(0, size, value.toUnscaledLong.toInt) - } else if (precision <= Decimal.MAX_LONG_DIGITS) { - toColumn.putLongs(0, size, value.toUnscaledLong) - } else { - val bytes = value.toJavaBigDecimal.unscaledValue.toByteArray - var index = 0 - while (index < size) { - toColumn.putByteArray(index, bytes, 0, bytes.length) - index += 1 - } - } - } -} - diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala index 7abcdba9639cd..b8bacfa1838ae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala @@ -150,7 +150,6 @@ class OrcFileFormat val sqlConf = sparkSession.sessionState.conf val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled val enableVectorizedReader = supportBatch(sparkSession, resultSchema) - val enableVectorizedJavaReader = sparkSession.sessionState.conf.orcVectorizedJavaReaderEnabled val broadcastedConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) @@ -184,29 +183,15 @@ class OrcFileFormat val taskContext = Option(TaskContext.get()) if (enableVectorizedReader) { - val batchReader = if (enableVectorizedJavaReader) { - val vectorizedReader = new JavaOrcColumnarBatchReader( - enableOffHeapColumnVector && taskContext.isDefined) - vectorizedReader.initialize(fileSplit, taskAttemptContext) - vectorizedReader.initBatch( - reader.getSchema, - requestedColIds, - requiredSchema.fields, - partitionSchema, - file.partitionValues) - vectorizedReader - } else { - val vectorizedReader = new OrcColumnarBatchReader - vectorizedReader.initialize(fileSplit, taskAttemptContext) - vectorizedReader.initBatch( - enableOffHeapColumnVector && taskContext.isDefined, - reader.getSchema, - requestedColIds, - requiredSchema.fields, - partitionSchema, - file.partitionValues) - vectorizedReader - } + val batchReader = + new OrcColumnarBatchReader(enableOffHeapColumnVector && taskContext.isDefined) + batchReader.initialize(fileSplit, taskAttemptContext) + batchReader.initBatch( + reader.getSchema, + requestedColIds, + requiredSchema.fields, + partitionSchema, + file.partitionValues) val iter = new RecordReaderIterator(batchReader) Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => iter.close())) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala index 175201eca51cd..3c53a815b502b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala @@ -102,15 +102,7 @@ object OrcReadBenchmark { } sqlBenchmark.addCase("Native ORC Vectorized") { _ => - withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") { - spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() - } - } - - sqlBenchmark.addCase("Native ORC Vectorized (Java)") { _ => - withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") { - spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() - } + spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() } sqlBenchmark.addCase("Hive built-in ORC") { _ => @@ -123,45 +115,39 @@ object OrcReadBenchmark { SQL Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1161 / 1168 13.5 73.8 1.0X - Native ORC Vectorized 163 / 171 96.3 10.4 7.1X - Native ORC Vectorized (Java) 155 / 163 101.6 9.8 7.5X - Hive built-in ORC 1427 / 1427 11.0 90.7 0.8X + Native ORC MR 1263 / 1296 12.5 80.3 1.0X + Native ORC Vectorized 159 / 166 98.6 10.1 7.9X + Hive built-in ORC 1513 / 1525 10.4 96.2 0.8X SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1261 / 1321 12.5 80.2 1.0X - Native ORC Vectorized 160 / 167 98.2 10.2 7.9X - Native ORC Vectorized (Java) 160 / 167 98.4 10.2 7.9X - Hive built-in ORC 1655 / 1687 9.5 105.2 0.8X + Native ORC MR 1196 / 1232 13.1 76.1 1.0X + Native ORC Vectorized 163 / 168 96.7 10.3 7.4X + Hive built-in ORC 1625 / 1640 9.7 103.3 0.7X SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1369 / 1449 11.5 87.1 1.0X - Native ORC Vectorized 263 / 277 59.8 16.7 5.2X - Native ORC Vectorized (Java) 225 / 237 70.0 14.3 6.1X - Hive built-in ORC 1867 / 1899 8.4 118.7 0.7X + Native ORC MR 1292 / 1378 12.2 82.2 1.0X + Native ORC Vectorized 228 / 236 68.9 14.5 5.7X + Hive built-in ORC 1829 / 1835 8.6 116.3 0.7X SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1472 / 1474 10.7 93.6 1.0X - Native ORC Vectorized 289 / 300 54.5 18.4 5.1X - Native ORC Vectorized (Java) 286 / 294 54.9 18.2 5.1X - Hive built-in ORC 1917 / 1934 8.2 121.9 0.8X + Native ORC MR 1372 / 1398 11.5 87.2 1.0X + Native ORC Vectorized 286 / 300 55.1 18.2 4.8X + Hive built-in ORC 1911 / 1913 8.2 121.5 0.7X SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1484 / 1484 10.6 94.3 1.0X - Native ORC Vectorized 365 / 370 43.1 23.2 4.1X - Native ORC Vectorized (Java) 326 / 335 48.2 20.7 4.5X - Hive built-in ORC 1978 / 2049 8.0 125.8 0.8X + Native ORC MR 1387 / 1415 11.3 88.2 1.0X + Native ORC Vectorized 326 / 329 48.2 20.7 4.3X + Hive built-in ORC 2004 / 2013 7.8 127.4 0.7X SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1550 / 1554 10.1 98.6 1.0X - Native ORC Vectorized 396 / 405 39.7 25.2 3.9X - Native ORC Vectorized (Java) 394 / 402 39.9 25.1 3.9X - Hive built-in ORC 2072 / 2084 7.6 131.8 0.7X + Native ORC MR 1478 / 1524 10.6 94.0 1.0X + Native ORC Vectorized 412 / 416 38.2 26.2 3.6X + Hive built-in ORC 2070 / 2106 7.6 131.6 0.7X */ sqlBenchmark.run() } @@ -187,15 +173,7 @@ object OrcReadBenchmark { } benchmark.addCase("Native ORC Vectorized") { _ => - withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") { - spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect() - } - } - - benchmark.addCase("Native ORC Vectorized (Java)") { _ => - withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") { - spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect() - } + spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect() } benchmark.addCase("Hive built-in ORC") { _ => @@ -208,10 +186,9 @@ object OrcReadBenchmark { Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2636 / 2734 4.0 251.4 1.0X - Native ORC Vectorized 1267 / 1267 8.3 120.9 2.1X - Native ORC Vectorized (Java) 1182 / 1183 8.9 112.7 2.2X - Hive built-in ORC 3724 / 3764 2.8 355.2 0.7X + Native ORC MR 2597 / 2671 4.0 247.7 1.0X + Native ORC Vectorized 1307 / 1315 8.0 124.7 2.0X + Hive built-in ORC 3867 / 3878 2.7 368.8 0.7X */ benchmark.run() } @@ -235,15 +212,7 @@ object OrcReadBenchmark { } benchmark.addCase("Read data column - Native ORC Vectorized") { _ => - withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") { - spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() - } - } - - benchmark.addCase("Read data column - Native ORC Vectorized (Java)") { _ => - withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") { - spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() - } + spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() } benchmark.addCase("Read data column - Hive built-in ORC") { _ => @@ -257,15 +226,7 @@ object OrcReadBenchmark { } benchmark.addCase("Read partition column - Native ORC Vectorized") { _ => - withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") { - spark.sql("SELECT sum(p) FROM nativeOrcTable").collect() - } - } - - benchmark.addCase("Read partition column - Native ORC Vectorized (Java)") { _ => - withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") { - spark.sql("SELECT sum(p) FROM nativeOrcTable").collect() - } + spark.sql("SELECT sum(p) FROM nativeOrcTable").collect() } benchmark.addCase("Read partition column - Hive built-in ORC") { _ => @@ -279,15 +240,7 @@ object OrcReadBenchmark { } benchmark.addCase("Read both columns - Native ORC Vectorized") { _ => - withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") { - spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect() - } - } - - benchmark.addCase("Read both columns - Native ORC Vectorized (Java)") { _ => - withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") { - spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect() - } + spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect() } benchmark.addCase("Read both columns - Hive built-in ORC") { _ => @@ -298,20 +251,17 @@ object OrcReadBenchmark { Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz - Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------ - Read data column - Native ORC MR 1587 / 1592 9.9 100.9 1.0X - Read data column - Native ORC Vectorized 290 / 309 54.3 18.4 5.5X - Read data column - Native ORC Vectorized (Java) 293 / 297 53.7 18.6 5.4X - Read data column - Hive built-in ORC 2204 / 2214 7.1 140.1 0.7X - Read partition column - Native ORC MR 1078 / 1097 14.6 68.5 1.5X - Read partition column - Native ORC Vectorized 53 / 56 294.0 3.4 29.7X - Read partition column - Native ORC Vectorized (Java) 52 / 55 300.7 3.3 30.4X - Read partition column - Hive built-in ORC 1279 / 1287 12.3 81.3 1.2X - Read both columns - Native ORC MR 1665 / 1674 9.4 105.9 1.0X - Read both columns - Native ORC Vectorized 327 / 333 48.0 20.8 4.8X - Read both columns - Native ORC Vectorized (Java) 327 / 332 48.2 20.8 4.9X - Read both columns - Hive built-in ORC 2157 / 2169 7.3 137.1 0.7X + Read data column - Native ORC MR 1560 / 1562 10.1 99.2 1.0X + Read data column - Native ORC Vectorized 294 / 301 53.4 18.7 5.3X + Read data column - Hive built-in ORC 2101 / 2128 7.5 133.6 0.7X + Read partition column - Native ORC MR 1080 / 1087 14.6 68.7 1.4X + Read partition column - Native ORC Vectorized 54 / 58 289.5 3.5 28.7X + Read partition column - Hive built-in ORC 1315 / 1316 12.0 83.6 1.2X + Read both columns - Native ORC MR 1581 / 1591 9.9 100.5 1.0X + Read both columns - Native ORC Vectorized 329 / 339 47.8 20.9 4.7X + Read both columns - Hive built-in ORC 2124 / 2158 7.4 135.0 0.7X */ benchmark.run() } @@ -334,15 +284,7 @@ object OrcReadBenchmark { } benchmark.addCase("Native ORC Vectorized") { _ => - withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") { - spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect() - } - } - - benchmark.addCase("Native ORC Vectorized (Java)") { _ => - withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") { - spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect() - } + spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect() } benchmark.addCase("Hive built-in ORC") { _ => @@ -355,10 +297,9 @@ object OrcReadBenchmark { String Dictionary: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1355 / 1355 7.7 129.2 1.0X - Native ORC Vectorized 262 / 270 40.0 25.0 5.2X - Native ORC Vectorized (Java) 223 / 227 46.9 21.3 6.1X - Hive built-in ORC 2017 / 2027 5.2 192.4 0.7X + Native ORC MR 1307 / 1309 8.0 124.6 1.0X + Native ORC Vectorized 327 / 336 32.1 31.2 4.0X + Hive built-in ORC 2009 / 2072 5.2 191.6 0.7X */ benchmark.run() } @@ -386,17 +327,8 @@ object OrcReadBenchmark { } benchmark.addCase(s"Native ORC Vectorized ($fractionOfNulls%)") { _ => - withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") { - spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " + - "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() - } - } - - benchmark.addCase(s"Native ORC Vectorized ($fractionOfNulls%) (Java)") { _ => - withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") { - spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " + - "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() - } + spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " + + "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() } benchmark.addCase(s"Hive built-in ORC ($fractionOfNulls%)") { _ => @@ -410,24 +342,21 @@ object OrcReadBenchmark { String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR (0.0%) 2575 / 2618 4.1 245.5 1.0X - Native ORC Vectorized (0.0%) 841 / 852 12.5 80.2 3.1X - Native ORC Vectorized (0.0%) (Java) 757 / 760 13.9 72.2 3.4X - Hive built-in ORC (0.0%) 4149 / 4162 2.5 395.7 0.6X + Native ORC MR (0.0%) 2534 / 2535 4.1 241.7 1.0X + Native ORC Vectorized (0.0%) 961 / 963 10.9 91.6 2.6X + Hive built-in ORC (0.0%) 3980 / 4005 2.6 379.5 0.6X String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR (0.5%) 2446 / 2460 4.3 233.3 1.0X - Native ORC Vectorized (0.5%) 1081 / 1084 9.7 103.1 2.3X - Native ORC Vectorized (0.5%) (Java) 1066 / 1069 9.8 101.6 2.3X - Hive built-in ORC (0.5%) 2928 / 2938 3.6 279.2 0.8X + Native ORC MR (0.5%) 2417 / 2422 4.3 230.5 1.0X + Native ORC Vectorized (0.5%) 1314 / 1315 8.0 125.3 1.8X + Hive built-in ORC (0.5%) 2940 / 2953 3.6 280.4 0.8X String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR (0.95%) 1287 / 1331 8.2 122.7 1.0X - Native ORC Vectorized (0.95%) 404 / 407 26.0 38.5 3.2X - Native ORC Vectorized (0.95%) (Java) 405 / 409 25.9 38.6 3.2X - Hive built-in ORC (0.95%) 1612 / 1644 6.5 153.7 0.8X + Native ORC MR (0.95%) 1306 / 1318 8.0 124.5 1.0X + Native ORC Vectorized (0.95%) 521 / 539 20.1 49.7 2.5X + Hive built-in ORC (0.95%) 1636 / 1641 6.4 156.0 0.8X */ benchmark.run() } @@ -454,15 +383,7 @@ object OrcReadBenchmark { } sqlBenchmark.addCase("Native ORC Vectorized") { _ => - withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "false") { - spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect() - } - } - - sqlBenchmark.addCase("Native ORC Vectorized (Java)") { _ => - withSQLConf(SQLConf.ORC_VECTORIZED_JAVA_READER_ENABLED.key -> "true") { - spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect() - } + spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect() } sqlBenchmark.addCase("Hive built-in ORC") { _ => @@ -475,24 +396,21 @@ object OrcReadBenchmark { SQL Single Column Scan from wide table (100 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1116 / 1117 0.9 1064.7 1.0X - Native ORC Vectorized 92 / 98 11.3 88.2 12.1X - Native ORC Vectorized (Java) 90 / 96 11.6 86.0 12.4X - Hive built-in ORC 376 / 386 2.8 358.6 3.0X + Native ORC MR 1119 / 1126 0.9 1067.4 1.0X + Native ORC Vectorized 93 / 99 11.3 88.3 12.1X + Hive built-in ORC 386 / 389 2.7 368.3 2.9X SQL Single Column Scan from wide table (200 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2268 / 2283 0.5 2162.6 1.0X - Native ORC Vectorized 158 / 165 6.6 150.5 14.4X - Native ORC Vectorized (Java) 155 / 167 6.8 147.8 14.6X - Hive built-in ORC 597 / 601 1.8 569.0 3.8X + Native ORC MR 2250 / 2271 0.5 2145.6 1.0X + Native ORC Vectorized 156 / 164 6.7 149.1 14.4X + Hive built-in ORC 589 / 601 1.8 561.7 3.8X SQL Single Column Scan from wide table (300 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 3377 / 3387 0.3 3220.7 1.0X - Native ORC Vectorized 274 / 282 3.8 261.7 12.3X - Native ORC Vectorized (Java) 269 / 277 3.9 256.7 12.5X - Hive built-in ORC 845 / 858 1.2 805.8 4.0X + Native ORC MR 3340 / 3360 0.3 3185.5 1.0X + Native ORC Vectorized 273 / 283 3.8 260.8 12.2X + Hive built-in ORC 851 / 861 1.2 811.4 3.9X */ sqlBenchmark.run() } From ba03d20ac6c826b5f16307884e34c1f4022eb814 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 8 Jan 2018 12:38:40 -0800 Subject: [PATCH 16/23] Remove `putByteArray`. --- .../orc/OrcColumnarBatchReader.java | 37 +++-- .../spark/sql/hive/orc/OrcReadBenchmark.scala | 140 +++++++++--------- 2 files changed, 98 insertions(+), 79 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java index cc01dfb4f3ef1..af4f8f85d9c04 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.datasources.orc; import java.io.IOException; +import java.util.stream.IntStream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.InputSplit; @@ -253,9 +254,12 @@ private void putRepeatingValues( toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]); } else if (type instanceof StringType || type instanceof BinaryType) { BytesColumnVector data = (BytesColumnVector)fromColumn; - toColumn.putByteArray(0, data.vector[0]); + WritableColumnVector arrayData = toColumn.getChildColumn(0); + int size = data.vector[0].length; + arrayData.reserve(size); + arrayData.putBytes(0, size, data.vector[0], 0); for (int index = 0; index < batchSize; index++) { - toColumn.putArray(index, data.start[0], data.length[0]); + toColumn.putArray(index, 0, size); } } else if (type instanceof DecimalType) { DecimalType decimalType = (DecimalType)type; @@ -313,12 +317,20 @@ private void putNonNullValues( toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0); } else if (type instanceof StringType || type instanceof BinaryType) { BytesColumnVector data = ((BytesColumnVector)fromColumn); - for (int index = 0; index < batchSize; index++) { - toColumn.putByteArray(index, data.vector[index], data.start[index], data.length[index]); + WritableColumnVector arrayData = toColumn.getChildColumn(0); + int totalNumBytes = IntStream.of(data.length).sum(); + arrayData.reserve(totalNumBytes); + for (int index = 0, pos = 0; index < batchSize; pos += data.length[index], index++) { + arrayData.putBytes(pos, data.length[index], data.vector[index], data.start[index]); + toColumn.putArray(index, pos, data.length[index]); } } else if (type instanceof DecimalType) { DecimalType decimalType = (DecimalType)type; DecimalColumnVector data = ((DecimalColumnVector)fromColumn); + if (decimalType.precision() > Decimal.MAX_INT_DIGITS()) { + WritableColumnVector arrayData = toColumn.getChildColumn(0); + arrayData.reserve(batchSize * 16); + } for (int index = 0; index < batchSize; index++) { putDecimalWritable( toColumn, @@ -412,12 +424,15 @@ private void putValues( } } else if (type instanceof StringType || type instanceof BinaryType) { BytesColumnVector vector = (BytesColumnVector)fromColumn; - for (int index = 0; index < batchSize; index++) { + WritableColumnVector arrayData = toColumn.getChildColumn(0); + int totalNumBytes = IntStream.of(vector.length).sum(); + arrayData.reserve(totalNumBytes); + for (int index = 0, pos = 0; index < batchSize; pos += vector.length[index], index++) { if (fromColumn.isNull[index]) { toColumn.putNull(index); } else { - toColumn.putByteArray( - index, vector.vector[index], vector.start[index], vector.length[index]); + arrayData.putBytes(pos, vector.length[index], vector.vector[index], vector.start[index]); + toColumn.putArray(index, pos, vector.length[index]); } } } else if (type instanceof DecimalType) { @@ -476,7 +491,9 @@ private static void putDecimalWritable( toColumn.putLong(index, value.toUnscaledLong()); } else { byte[] bytes = value.toJavaBigDecimal().unscaledValue().toByteArray(); - toColumn.putByteArray(index, bytes, 0, bytes.length); + WritableColumnVector arrayData = toColumn.getChildColumn(0); + arrayData.putBytes(index * 4, bytes.length, bytes, 0); + toColumn.putArray(index, index * 4, bytes.length); } } @@ -500,7 +517,9 @@ private static void putDecimalWritables( toColumn.putLongs(0, size, value.toUnscaledLong()); } else { byte[] bytes = value.toJavaBigDecimal().unscaledValue().toByteArray(); - toColumn.putByteArray(0, bytes); + WritableColumnVector arrayData = toColumn.getChildColumn(0); + arrayData.reserve(bytes.length); + arrayData.putBytes(0, bytes.length, bytes, 0); for (int index = 0; index < size; index++) { toColumn.putArray(index, 0, bytes.length); } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala index 3c53a815b502b..e06da64c25e0d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala @@ -69,8 +69,8 @@ object OrcReadBenchmark { } } - private val NATIVE_ORC_FORMAT = "org.apache.spark.sql.execution.datasources.orc.OrcFileFormat" - private val HIVE_ORC_FORMAT = "org.apache.spark.sql.hive.orc.OrcFileFormat" + private val NATIVE_ORC_FORMAT = classOf[org.apache.spark.sql.execution.datasources.orc.OrcFileFormat].getCanonicalName + private val HIVE_ORC_FORMAT = classOf[org.apache.spark.sql.hive.orc.OrcFileFormat].getCanonicalName private def prepareTable(dir: File, df: DataFrame, partition: Option[String] = None): Unit = { val dirORC = dir.getCanonicalPath @@ -115,39 +115,39 @@ object OrcReadBenchmark { SQL Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1263 / 1296 12.5 80.3 1.0X - Native ORC Vectorized 159 / 166 98.6 10.1 7.9X - Hive built-in ORC 1513 / 1525 10.4 96.2 0.8X + Native ORC MR 1180 / 1230 13.3 75.0 1.0X + Native ORC Vectorized 159 / 169 98.8 10.1 7.4X + Hive built-in ORC 1395 / 1396 11.3 88.7 0.8X SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1196 / 1232 13.1 76.1 1.0X - Native ORC Vectorized 163 / 168 96.7 10.3 7.4X - Hive built-in ORC 1625 / 1640 9.7 103.3 0.7X + Native ORC MR 1216 / 1267 12.9 77.3 1.0X + Native ORC Vectorized 163 / 172 96.4 10.4 7.5X + Hive built-in ORC 1649 / 1672 9.5 104.8 0.7X SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1292 / 1378 12.2 82.2 1.0X - Native ORC Vectorized 228 / 236 68.9 14.5 5.7X - Hive built-in ORC 1829 / 1835 8.6 116.3 0.7X + Native ORC MR 1331 / 1332 11.8 84.6 1.0X + Native ORC Vectorized 233 / 245 67.6 14.8 5.7X + Hive built-in ORC 1832 / 1839 8.6 116.5 0.7X SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1372 / 1398 11.5 87.2 1.0X - Native ORC Vectorized 286 / 300 55.1 18.2 4.8X - Hive built-in ORC 1911 / 1913 8.2 121.5 0.7X + Native ORC MR 1382 / 1389 11.4 87.8 1.0X + Native ORC Vectorized 291 / 299 54.1 18.5 4.8X + Hive built-in ORC 1926 / 1936 8.2 122.5 0.7X SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1387 / 1415 11.3 88.2 1.0X - Native ORC Vectorized 326 / 329 48.2 20.7 4.3X - Hive built-in ORC 2004 / 2013 7.8 127.4 0.7X + Native ORC MR 1399 / 1478 11.2 88.9 1.0X + Native ORC Vectorized 324 / 329 48.6 20.6 4.3X + Hive built-in ORC 1938 / 1945 8.1 123.2 0.7X SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1478 / 1524 10.6 94.0 1.0X - Native ORC Vectorized 412 / 416 38.2 26.2 3.6X - Hive built-in ORC 2070 / 2106 7.6 131.6 0.7X + Native ORC MR 1441 / 1470 10.9 91.6 1.0X + Native ORC Vectorized 406 / 408 38.8 25.8 3.6X + Hive built-in ORC 2031 / 2039 7.7 129.1 0.7X */ sqlBenchmark.run() } @@ -186,9 +186,9 @@ object OrcReadBenchmark { Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2597 / 2671 4.0 247.7 1.0X - Native ORC Vectorized 1307 / 1315 8.0 124.7 2.0X - Hive built-in ORC 3867 / 3878 2.7 368.8 0.7X + Native ORC MR 2716 / 2738 3.9 259.0 1.0X + Native ORC Vectorized 1325 / 1325 7.9 126.4 2.0X + Hive built-in ORC 3607 / 3645 2.9 344.0 0.8X */ benchmark.run() } @@ -251,25 +251,25 @@ object OrcReadBenchmark { Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz - Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------------ - Read data column - Native ORC MR 1560 / 1562 10.1 99.2 1.0X - Read data column - Native ORC Vectorized 294 / 301 53.4 18.7 5.3X - Read data column - Hive built-in ORC 2101 / 2128 7.5 133.6 0.7X - Read partition column - Native ORC MR 1080 / 1087 14.6 68.7 1.4X - Read partition column - Native ORC Vectorized 54 / 58 289.5 3.5 28.7X - Read partition column - Hive built-in ORC 1315 / 1316 12.0 83.6 1.2X - Read both columns - Native ORC MR 1581 / 1591 9.9 100.5 1.0X - Read both columns - Native ORC Vectorized 329 / 339 47.8 20.9 4.7X - Read both columns - Hive built-in ORC 2124 / 2158 7.4 135.0 0.7X + Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Read data column - Native ORC MR 1544 / 1547 10.2 98.1 1.0X + Read data column - Native ORC Vectorized 298 / 306 52.7 19.0 5.2X + Read data column - Hive built-in ORC 2089 / 2097 7.5 132.8 0.7X + Read partition column - Native ORC MR 1050 / 1051 15.0 66.8 1.5X + Read partition column - Native ORC Vectorized 54 / 57 290.0 3.4 28.5X + Read partition column - Hive built-in ORC 1271 / 1280 12.4 80.8 1.2X + Read both columns - Native ORC MR 1572 / 1605 10.0 100.0 1.0X + Read both columns - Native ORC Vectorized 332 / 338 47.4 21.1 4.6X + Read both columns - Hive built-in ORC 2108 / 2123 7.5 134.0 0.7X */ benchmark.run() } } } - def stringDictionaryScanBenchmark(values: Int): Unit = { - val benchmark = new Benchmark("String Dictionary", values) + def repeatedStringScanBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Repeated String", values) withTempPath { dir => withTempTable("t1", "nativeOrcTable", "hiveOrcTable") { @@ -295,11 +295,11 @@ object OrcReadBenchmark { Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz - String Dictionary: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + Repeated String: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1307 / 1309 8.0 124.6 1.0X - Native ORC Vectorized 327 / 336 32.1 31.2 4.0X - Hive built-in ORC 2009 / 2072 5.2 191.6 0.7X + Native ORC MR 1297 / 1327 8.1 123.7 1.0X + Native ORC Vectorized 317 / 327 33.1 30.2 4.1X + Hive built-in ORC 1970 / 1973 5.3 187.9 0.7X */ benchmark.run() } @@ -317,21 +317,21 @@ object OrcReadBenchmark { s"SELECT IF(RAND(1) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c1, " + s"IF(RAND(2) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c2 FROM t1")) - val benchmark = new Benchmark("String with Nulls Scan", values) + val benchmark = new Benchmark(s"String with Nulls Scan ($fractionOfNulls%)", values) - benchmark.addCase(s"Native ORC MR ($fractionOfNulls%)") { _ => + benchmark.addCase("Native ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " + "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() } } - benchmark.addCase(s"Native ORC Vectorized ($fractionOfNulls%)") { _ => + benchmark.addCase("Native ORC Vectorized") { _ => spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " + "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() } - benchmark.addCase(s"Hive built-in ORC ($fractionOfNulls%)") { _ => + benchmark.addCase("Hive built-in ORC") { _ => spark.sql("SELECT SUM(LENGTH(c2)) FROM hiveOrcTable " + "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() } @@ -340,23 +340,23 @@ object OrcReadBenchmark { Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz - String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + String with Nulls Scan (0.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR (0.0%) 2534 / 2535 4.1 241.7 1.0X - Native ORC Vectorized (0.0%) 961 / 963 10.9 91.6 2.6X - Hive built-in ORC (0.0%) 3980 / 4005 2.6 379.5 0.6X + Native ORC MR 2531 / 2542 4.1 241.4 1.0X + Native ORC Vectorized 947 / 952 11.1 90.3 2.7X + Hive built-in ORC 4012 / 4034 2.6 382.6 0.6X - String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + String with Nulls Scan (0.5%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR (0.5%) 2417 / 2422 4.3 230.5 1.0X - Native ORC Vectorized (0.5%) 1314 / 1315 8.0 125.3 1.8X - Hive built-in ORC (0.5%) 2940 / 2953 3.6 280.4 0.8X + Native ORC MR 2388 / 2407 4.4 227.8 1.0X + Native ORC Vectorized 1235 / 1236 8.5 117.8 1.9X + Hive built-in ORC 2951 / 2958 3.6 281.4 0.8X - String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + String with Nulls Scan (0.95%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR (0.95%) 1306 / 1318 8.0 124.5 1.0X - Native ORC Vectorized (0.95%) 521 / 539 20.1 49.7 2.5X - Hive built-in ORC (0.95%) 1636 / 1641 6.4 156.0 0.8X + Native ORC MR 1325 / 1346 7.9 126.4 1.0X + Native ORC Vectorized 460 / 468 22.8 43.9 2.9X + Hive built-in ORC 1600 / 1607 6.6 152.6 0.8X */ benchmark.run() } @@ -364,7 +364,7 @@ object OrcReadBenchmark { } def columnsBenchmark(values: Int, width: Int): Unit = { - val sqlBenchmark = new Benchmark(s"SQL Single Column Scan from wide table ($width columns)", values) + val sqlBenchmark = new Benchmark(s"SQL Single Column Scan from $width columns", values) withTempPath { dir => withTempTable("t1", "nativeOrcTable", "hiveOrcTable") { @@ -394,23 +394,23 @@ object OrcReadBenchmark { Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz - SQL Single Column Scan from wide table (100 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + SQL Single Column Scan from 100 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1119 / 1126 0.9 1067.4 1.0X - Native ORC Vectorized 93 / 99 11.3 88.3 12.1X - Hive built-in ORC 386 / 389 2.7 368.3 2.9X + Native ORC MR 1107 / 1118 0.9 1056.1 1.0X + Native ORC Vectorized 94 / 100 11.1 89.8 11.8X + Hive built-in ORC 382 / 390 2.7 364.0 2.9X - SQL Single Column Scan from wide table (200 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + SQL Single Column Scan from 200 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2250 / 2271 0.5 2145.6 1.0X - Native ORC Vectorized 156 / 164 6.7 149.1 14.4X - Hive built-in ORC 589 / 601 1.8 561.7 3.8X + Native ORC MR 2278 / 2287 0.5 2172.0 1.0X + Native ORC Vectorized 158 / 165 6.6 150.6 14.4X + Hive built-in ORC 585 / 590 1.8 557.7 3.9X - SQL Single Column Scan from wide table (300 columns): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + SQL Single Column Scan from 300 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 3340 / 3360 0.3 3185.5 1.0X - Native ORC Vectorized 273 / 283 3.8 260.8 12.2X - Hive built-in ORC 851 / 861 1.2 811.4 3.9X + Native ORC MR 3386 / 3394 0.3 3229.1 1.0X + Native ORC Vectorized 271 / 281 3.9 258.2 12.5X + Hive built-in ORC 843 / 852 1.2 803.6 4.0X */ sqlBenchmark.run() } @@ -423,7 +423,7 @@ object OrcReadBenchmark { } intStringScanBenchmark(1024 * 1024 * 10) partitionTableScanBenchmark(1024 * 1024 * 15) - stringDictionaryScanBenchmark(1024 * 1024 * 10) + repeatedStringScanBenchmark(1024 * 1024 * 10) for (fractionOfNulls <- List(0.0, 0.50, 0.95)) { stringWithNullsScanBenchmark(1024 * 1024 * 10, fractionOfNulls) } From b623ca4621d22d294974cb3a7f88260052b1f38c Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 8 Jan 2018 17:14:40 -0800 Subject: [PATCH 17/23] fix typo. --- .../sql/execution/datasources/orc/OrcColumnarBatchReader.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java index af4f8f85d9c04..792eb0beb9f0c 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java @@ -492,8 +492,8 @@ private static void putDecimalWritable( } else { byte[] bytes = value.toJavaBigDecimal().unscaledValue().toByteArray(); WritableColumnVector arrayData = toColumn.getChildColumn(0); - arrayData.putBytes(index * 4, bytes.length, bytes, 0); - toColumn.putArray(index, index * 4, bytes.length); + arrayData.putBytes(index * 16, bytes.length, bytes, 0); + toColumn.putArray(index, index * 16, bytes.length); } } From 8fc2162c3be968324c40a8717e4ddcc5cf173ec9 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 8 Jan 2018 19:07:33 -0800 Subject: [PATCH 18/23] Split functions. --- .../orc/OrcColumnarBatchReader.java | 403 +++++++++++------- .../spark/sql/hive/orc/OrcReadBenchmark.scala | 114 ++--- 2 files changed, 295 insertions(+), 222 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java index 792eb0beb9f0c..caffdbd7cacf5 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java @@ -49,6 +49,21 @@ * After creating, `initialize` and `initBatch` should be called sequentially. */ public class OrcColumnarBatchReader extends RecordReader { + /** + * The default size of batch. We use this value for both ORC and Spark consistently + * because they have different default values like the following. + * + * - ORC's VectorizedRowBatch.DEFAULT_SIZE = 1024 + * - Spark's ColumnarBatch.DEFAULT_BATCH_SIZE = 4 * 1024 + */ + public static final int DEFAULT_SIZE = 4 * 1024; + + /** + * Returns the number of micros since epoch from an element of TimestampColumnVector. + */ + private static long fromTimestampColumnVector(TimestampColumnVector vector, int index) { + return vector.time[index] * 1000L + vector.nanos[index] / 1000L; + } // ORC File Reader private Reader reader; @@ -253,14 +268,7 @@ private void putRepeatingValues( } else if (type instanceof DoubleType) { toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]); } else if (type instanceof StringType || type instanceof BinaryType) { - BytesColumnVector data = (BytesColumnVector)fromColumn; - WritableColumnVector arrayData = toColumn.getChildColumn(0); - int size = data.vector[0].length; - arrayData.reserve(size); - arrayData.putBytes(0, size, data.vector[0], 0); - for (int index = 0; index < batchSize; index++) { - toColumn.putArray(index, 0, size); - } + putByteArrays(batchSize, toColumn, ((BytesColumnVector)fromColumn).vector[0]); } else if (type instanceof DecimalType) { DecimalType decimalType = (DecimalType)type; putDecimalWritables( @@ -282,68 +290,31 @@ private void putNonNullValues( WritableColumnVector toColumn) { DataType type = field.dataType(); if (type instanceof BooleanType) { - long[] data = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - toColumn.putBoolean(index, data[index] == 1); - } + putNonNullBooleans(batchSize, (LongColumnVector) fromColumn, toColumn); } else if (type instanceof ByteType) { - long[] data = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - toColumn.putByte(index, (byte)data[index]); - } + putNonNullBytes(batchSize, (LongColumnVector) fromColumn, toColumn); } else if (type instanceof ShortType) { - long[] data = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - toColumn.putShort(index, (short)data[index]); - } + putNonNullShorts(batchSize, (LongColumnVector) fromColumn, toColumn); } else if (type instanceof IntegerType || type instanceof DateType) { - long[] data = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - toColumn.putInt(index, (int)data[index]); - } + putNonNullInts(batchSize, (LongColumnVector) fromColumn, toColumn); } else if (type instanceof LongType) { toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector, 0); } else if (type instanceof TimestampType) { - TimestampColumnVector data = ((TimestampColumnVector)fromColumn); - for (int index = 0; index < batchSize; index++) { - toColumn.putLong(index, fromTimestampColumnVector(data, index)); - } + putNonNullTimestamps(batchSize, (TimestampColumnVector) fromColumn, toColumn); } else if (type instanceof FloatType) { - double[] data = ((DoubleColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - toColumn.putFloat(index, (float)data[index]); - } + putNonNullFloats(batchSize, (DoubleColumnVector) fromColumn, toColumn); } else if (type instanceof DoubleType) { toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0); } else if (type instanceof StringType || type instanceof BinaryType) { - BytesColumnVector data = ((BytesColumnVector)fromColumn); - WritableColumnVector arrayData = toColumn.getChildColumn(0); - int totalNumBytes = IntStream.of(data.length).sum(); - arrayData.reserve(totalNumBytes); - for (int index = 0, pos = 0; index < batchSize; pos += data.length[index], index++) { - arrayData.putBytes(pos, data.length[index], data.vector[index], data.start[index]); - toColumn.putArray(index, pos, data.length[index]); - } + putNonNullByteArray(batchSize, (BytesColumnVector) fromColumn, toColumn); } else if (type instanceof DecimalType) { - DecimalType decimalType = (DecimalType)type; - DecimalColumnVector data = ((DecimalColumnVector)fromColumn); - if (decimalType.precision() > Decimal.MAX_INT_DIGITS()) { - WritableColumnVector arrayData = toColumn.getChildColumn(0); - arrayData.reserve(batchSize * 16); - } - for (int index = 0; index < batchSize; index++) { - putDecimalWritable( - toColumn, - index, - decimalType.precision(), - decimalType.scale(), - data.vector[index]); - } + putNonNullDecimals(batchSize, (DecimalColumnVector) fromColumn, toColumn, (DecimalType) type); } else { throw new UnsupportedOperationException("Unsupported Data Type: " + type); } } + private void putValues( int batchSize, StructField field, @@ -351,129 +322,34 @@ private void putValues( WritableColumnVector toColumn) { DataType type = field.dataType(); if (type instanceof BooleanType) { - long[] vector = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putBoolean(index, vector[index] == 1); - } - } + putBooleans(batchSize, (LongColumnVector)fromColumn, toColumn); } else if (type instanceof ByteType) { - long[] vector = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putByte(index, (byte)vector[index]); - } - } + putBytes(batchSize, (LongColumnVector)fromColumn, toColumn); } else if (type instanceof ShortType) { - long[] vector = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putShort(index, (short)vector[index]); - } - } + putShorts(batchSize, (LongColumnVector)fromColumn, toColumn); } else if (type instanceof IntegerType || type instanceof DateType) { - long[] vector = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putInt(index, (int)vector[index]); - } - } + putInts(batchSize, (LongColumnVector)fromColumn, toColumn); } else if (type instanceof LongType) { - long[] vector = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putLong(index, vector[index]); - } - } + putLongs(batchSize, (LongColumnVector)fromColumn, toColumn); } else if (type instanceof TimestampType) { - TimestampColumnVector vector = ((TimestampColumnVector)fromColumn); - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putLong(index, fromTimestampColumnVector(vector, index)); - } - } + putTimestamps(batchSize, (TimestampColumnVector)fromColumn, toColumn); } else if (type instanceof FloatType) { - double[] vector = ((DoubleColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putFloat(index, (float)vector[index]); - } - } + putFloats(batchSize, (DoubleColumnVector)fromColumn, toColumn); } else if (type instanceof DoubleType) { - double[] vector = ((DoubleColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putDouble(index, vector[index]); - } - } + putDoubles(batchSize, (DoubleColumnVector)fromColumn, toColumn); } else if (type instanceof StringType || type instanceof BinaryType) { - BytesColumnVector vector = (BytesColumnVector)fromColumn; - WritableColumnVector arrayData = toColumn.getChildColumn(0); - int totalNumBytes = IntStream.of(vector.length).sum(); - arrayData.reserve(totalNumBytes); - for (int index = 0, pos = 0; index < batchSize; pos += vector.length[index], index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - arrayData.putBytes(pos, vector.length[index], vector.vector[index], vector.start[index]); - toColumn.putArray(index, pos, vector.length[index]); - } - } + putByteArrays(batchSize, (BytesColumnVector)fromColumn, toColumn); } else if (type instanceof DecimalType) { - DecimalType decimalType = (DecimalType)type; - HiveDecimalWritable[] vector = ((DecimalColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - putDecimalWritable( - toColumn, - index, - decimalType.precision(), - decimalType.scale(), - vector[index]); - } - } + putDecimals(batchSize, (DecimalColumnVector)fromColumn, toColumn, (DecimalType) type); } else { throw new UnsupportedOperationException("Unsupported Data Type: " + type); } } - /** - * The default size of batch. We use this value for both ORC and Spark consistently - * because they have different default values like the following. - * - * - ORC's VectorizedRowBatch.DEFAULT_SIZE = 1024 - * - Spark's ColumnarBatch.DEFAULT_BATCH_SIZE = 4 * 1024 - */ - public static final int DEFAULT_SIZE = 4 * 1024; - - /** - * Returns the number of micros since epoch from an element of TimestampColumnVector. - */ - private static long fromTimestampColumnVector(TimestampColumnVector vector, int index) { - return vector.time[index] * 1000L + vector.nanos[index] / 1000L; - } + // -------------------------------------------------------------------------- + // Put a value + // -------------------------------------------------------------------------- - /** - * Put a `HiveDecimalWritable` to a `WritableColumnVector`. - */ private static void putDecimalWritable( WritableColumnVector toColumn, int index, @@ -497,9 +373,20 @@ private static void putDecimalWritable( } } - /** - * Put `HiveDecimalWritable`s to a `WritableColumnVector`. - */ + // -------------------------------------------------------------------------- + // Put repeating values + // -------------------------------------------------------------------------- + + private void putByteArrays(int count, WritableColumnVector toColumn, byte[] bytes) { + WritableColumnVector arrayData = toColumn.getChildColumn(0); + int size = bytes.length; + arrayData.reserve(size); + arrayData.putBytes(0, size, bytes, 0); + for (int index = 0; index < count; index++) { + toColumn.putArray(index, 0, size); + } + } + private static void putDecimalWritables( WritableColumnVector toColumn, int size, @@ -525,4 +412,190 @@ private static void putDecimalWritables( } } } + + // -------------------------------------------------------------------------- + // Put non-null values + // -------------------------------------------------------------------------- + + private void putNonNullBooleans(int count, LongColumnVector fromColumn, WritableColumnVector toColumn) { + long[] data = fromColumn.vector; + for (int index = 0; index < count; index++) { + toColumn.putBoolean(index, data[index] == 1); + } + } + + private void putNonNullByteArray(int count, BytesColumnVector fromColumn, WritableColumnVector toColumn) { + BytesColumnVector data = fromColumn; + WritableColumnVector arrayData = toColumn.getChildColumn(0); + int totalNumBytes = IntStream.of(data.length).sum(); + arrayData.reserve(totalNumBytes); + for (int index = 0, pos = 0; index < count; pos += data.length[index], index++) { + arrayData.putBytes(pos, data.length[index], data.vector[index], data.start[index]); + toColumn.putArray(index, pos, data.length[index]); + } + } + + private void putNonNullBytes(int count, LongColumnVector fromColumn, WritableColumnVector toColumn) { + long[] data = fromColumn.vector; + for (int index = 0; index < count; index++) { + toColumn.putByte(index, (byte)data[index]); + } + } + + private void putNonNullShorts(int count, LongColumnVector fromColumn, WritableColumnVector toColumn) { + long[] data = fromColumn.vector; + for (int index = 0; index < count; index++) { + toColumn.putShort(index, (short)data[index]); + } + } + + private void putNonNullInts(int count, LongColumnVector fromColumn, WritableColumnVector toColumn) { + long[] data = fromColumn.vector; + for (int index = 0; index < count; index++) { + toColumn.putInt(index, (int)data[index]); + } + } + + private void putNonNullTimestamps(int count, TimestampColumnVector fromColumn, WritableColumnVector toColumn) { + TimestampColumnVector data = fromColumn; + for (int index = 0; index < count; index++) { + toColumn.putLong(index, fromTimestampColumnVector(data, index)); + } + } + + private void putNonNullFloats(int count, DoubleColumnVector fromColumn, WritableColumnVector toColumn) { + double[] data = fromColumn.vector; + for (int index = 0; index < count; index++) { + toColumn.putFloat(index, (float)data[index]); + } + } + + private void putNonNullDecimals(int count, DecimalColumnVector fromColumn, WritableColumnVector toColumn, DecimalType type) { + DecimalColumnVector data = fromColumn; + if (type.precision() > Decimal.MAX_INT_DIGITS()) { + WritableColumnVector arrayData = toColumn.getChildColumn(0); + arrayData.reserve(count * 16); + } + for (int index = 0; index < count; index++) { + putDecimalWritable(toColumn, index, type.precision(), type.scale(), data.vector[index]); + } + } + + // -------------------------------------------------------------------------- + // Put values + // -------------------------------------------------------------------------- + + private void putBooleans(int count, LongColumnVector fromColumn, WritableColumnVector toColumn) { + long[] vector = fromColumn.vector; + for (int index = 0; index < count; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + toColumn.putBoolean(index, vector[index] == 1); + } + } + } + + private void putBytes(int count, LongColumnVector fromColumn, WritableColumnVector toColumn) { + long[] vector = fromColumn.vector; + for (int index = 0; index < count; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + toColumn.putByte(index, (byte)vector[index]); + } + } + } + + private void putShorts(int count, LongColumnVector fromColumn, WritableColumnVector toColumn) { + long[] vector = fromColumn.vector; + for (int index = 0; index < count; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + toColumn.putShort(index, (short)vector[index]); + } + } + } + + private void putInts(int count, LongColumnVector fromColumn, WritableColumnVector toColumn) { + long[] vector = fromColumn.vector; + for (int index = 0; index < count; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + toColumn.putInt(index, (int)vector[index]); + } + } + } + + private void putLongs(int count, LongColumnVector fromColumn, WritableColumnVector toColumn) { + long[] vector = fromColumn.vector; + for (int index = 0; index < count; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + toColumn.putLong(index, vector[index]); + } + } + } + + private void putTimestamps(int count, TimestampColumnVector fromColumn, WritableColumnVector toColumn) { + TimestampColumnVector vector = fromColumn; + for (int index = 0; index < count; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + toColumn.putLong(index, fromTimestampColumnVector(vector, index)); + } + } + } + + private void putFloats(int count, DoubleColumnVector fromColumn, WritableColumnVector toColumn) { + double[] vector = fromColumn.vector; + for (int index = 0; index < count; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + toColumn.putFloat(index, (float)vector[index]); + } + } + } + + private void putDoubles(int count, DoubleColumnVector fromColumn, WritableColumnVector toColumn) { + double[] vector = fromColumn.vector; + for (int index = 0; index < count; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + toColumn.putDouble(index, vector[index]); + } + } + } + + private void putByteArrays(int count, BytesColumnVector fromColumn, WritableColumnVector toColumn) { + BytesColumnVector vector = fromColumn; + WritableColumnVector arrayData = toColumn.getChildColumn(0); + int totalNumBytes = IntStream.of(vector.length).sum(); + arrayData.reserve(totalNumBytes); + for (int index = 0, pos = 0; index < count; pos += vector.length[index], index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + arrayData.putBytes(pos, vector.length[index], vector.vector[index], vector.start[index]); + toColumn.putArray(index, pos, vector.length[index]); + } + } + } + + private void putDecimals(int count, DecimalColumnVector fromColumn, WritableColumnVector toColumn, DecimalType type) { + HiveDecimalWritable[] vector = fromColumn.vector; + for (int index = 0; index < count; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + putDecimalWritable(toColumn, index, type.precision(), type.scale(), vector[index]); + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala index e06da64c25e0d..0ce89f833b274 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala @@ -110,44 +110,44 @@ object OrcReadBenchmark { } /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 + Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz SQL Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1180 / 1230 13.3 75.0 1.0X - Native ORC Vectorized 159 / 169 98.8 10.1 7.4X - Hive built-in ORC 1395 / 1396 11.3 88.7 0.8X + Native ORC MR 1202 / 1265 13.1 76.4 1.0X + Native ORC Vectorized 162 / 172 97.1 10.3 7.4X + Hive built-in ORC 1410 / 1428 11.2 89.6 0.9X SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1216 / 1267 12.9 77.3 1.0X - Native ORC Vectorized 163 / 172 96.4 10.4 7.5X - Hive built-in ORC 1649 / 1672 9.5 104.8 0.7X + Native ORC MR 1266 / 1286 12.4 80.5 1.0X + Native ORC Vectorized 165 / 174 95.2 10.5 7.7X + Hive built-in ORC 1701 / 1704 9.2 108.2 0.7X SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1331 / 1332 11.8 84.6 1.0X - Native ORC Vectorized 233 / 245 67.6 14.8 5.7X - Hive built-in ORC 1832 / 1839 8.6 116.5 0.7X + Native ORC MR 1307 / 1307 12.0 83.1 1.0X + Native ORC Vectorized 232 / 248 67.9 14.7 5.6X + Hive built-in ORC 1793 / 1793 8.8 114.0 0.7X SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1382 / 1389 11.4 87.8 1.0X - Native ORC Vectorized 291 / 299 54.1 18.5 4.8X - Hive built-in ORC 1926 / 1936 8.2 122.5 0.7X + Native ORC MR 1360 / 1372 11.6 86.5 1.0X + Native ORC Vectorized 293 / 303 53.8 18.6 4.7X + Hive built-in ORC 1913 / 1933 8.2 121.6 0.7X SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1399 / 1478 11.2 88.9 1.0X - Native ORC Vectorized 324 / 329 48.6 20.6 4.3X - Hive built-in ORC 1938 / 1945 8.1 123.2 0.7X + Native ORC MR 1389 / 1488 11.3 88.3 1.0X + Native ORC Vectorized 340 / 346 46.2 21.6 4.1X + Hive built-in ORC 1976 / 1997 8.0 125.6 0.7X SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1441 / 1470 10.9 91.6 1.0X - Native ORC Vectorized 406 / 408 38.8 25.8 3.6X - Hive built-in ORC 2031 / 2039 7.7 129.1 0.7X + Native ORC MR 1461 / 1465 10.8 92.9 1.0X + Native ORC Vectorized 395 / 406 39.8 25.1 3.7X + Hive built-in ORC 2127 / 2146 7.4 135.2 0.7X */ sqlBenchmark.run() } @@ -181,14 +181,14 @@ object OrcReadBenchmark { } /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 + Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2716 / 2738 3.9 259.0 1.0X - Native ORC Vectorized 1325 / 1325 7.9 126.4 2.0X - Hive built-in ORC 3607 / 3645 2.9 344.0 0.8X + Native ORC MR 2843 / 2935 3.7 271.1 1.0X + Native ORC Vectorized 1349 / 1359 7.8 128.6 2.1X + Hive built-in ORC 3862 / 3881 2.7 368.3 0.7X */ benchmark.run() } @@ -248,20 +248,20 @@ object OrcReadBenchmark { } /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 + Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Read data column - Native ORC MR 1544 / 1547 10.2 98.1 1.0X - Read data column - Native ORC Vectorized 298 / 306 52.7 19.0 5.2X - Read data column - Hive built-in ORC 2089 / 2097 7.5 132.8 0.7X - Read partition column - Native ORC MR 1050 / 1051 15.0 66.8 1.5X - Read partition column - Native ORC Vectorized 54 / 57 290.0 3.4 28.5X - Read partition column - Hive built-in ORC 1271 / 1280 12.4 80.8 1.2X - Read both columns - Native ORC MR 1572 / 1605 10.0 100.0 1.0X - Read both columns - Native ORC Vectorized 332 / 338 47.4 21.1 4.6X - Read both columns - Hive built-in ORC 2108 / 2123 7.5 134.0 0.7X + Read data column - Native ORC MR 1545 / 1568 10.2 98.2 1.0X + Read data column - Native ORC Vectorized 300 / 304 52.4 19.1 5.1X + Read data column - Hive built-in ORC 2097 / 2117 7.5 133.3 0.7X + Read partition column - Native ORC MR 1023 / 1026 15.4 65.1 1.5X + Read partition column - Native ORC Vectorized 54 / 56 292.3 3.4 28.7X + Read partition column - Hive built-in ORC 1285 / 1289 12.2 81.7 1.2X + Read both columns - Native ORC MR 1564 / 1565 10.1 99.4 1.0X + Read both columns - Native ORC Vectorized 336 / 340 46.8 21.4 4.6X + Read both columns - Hive built-in ORC 2100 / 2123 7.5 133.5 0.7X */ benchmark.run() } @@ -292,14 +292,14 @@ object OrcReadBenchmark { } /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 + Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz Repeated String: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1297 / 1327 8.1 123.7 1.0X - Native ORC Vectorized 317 / 327 33.1 30.2 4.1X - Hive built-in ORC 1970 / 1973 5.3 187.9 0.7X + Native ORC MR 1338 / 1340 7.8 127.6 1.0X + Native ORC Vectorized 342 / 350 30.6 32.6 3.9X + Hive built-in ORC 2036 / 2117 5.2 194.2 0.7X */ benchmark.run() } @@ -337,26 +337,26 @@ object OrcReadBenchmark { } /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 + Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz String with Nulls Scan (0.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2531 / 2542 4.1 241.4 1.0X - Native ORC Vectorized 947 / 952 11.1 90.3 2.7X - Hive built-in ORC 4012 / 4034 2.6 382.6 0.6X + Native ORC MR 2510 / 2527 4.2 239.3 1.0X + Native ORC Vectorized 1012 / 1012 10.4 96.5 2.5X + Hive built-in ORC 3967 / 3984 2.6 378.3 0.6X String with Nulls Scan (0.5%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2388 / 2407 4.4 227.8 1.0X - Native ORC Vectorized 1235 / 1236 8.5 117.8 1.9X - Hive built-in ORC 2951 / 2958 3.6 281.4 0.8X + Native ORC MR 2374 / 2388 4.4 226.4 1.0X + Native ORC Vectorized 1269 / 1275 8.3 121.0 1.9X + Hive built-in ORC 2994 / 2998 3.5 285.5 0.8X String with Nulls Scan (0.95%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1325 / 1346 7.9 126.4 1.0X - Native ORC Vectorized 460 / 468 22.8 43.9 2.9X - Hive built-in ORC 1600 / 1607 6.6 152.6 0.8X + Native ORC MR 1285 / 1312 8.2 122.6 1.0X + Native ORC Vectorized 500 / 508 21.0 47.7 2.6X + Hive built-in ORC 1630 / 1638 6.4 155.5 0.8X */ benchmark.run() } @@ -391,26 +391,26 @@ object OrcReadBenchmark { } /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 + Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz SQL Single Column Scan from 100 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1107 / 1118 0.9 1056.1 1.0X - Native ORC Vectorized 94 / 100 11.1 89.8 11.8X - Hive built-in ORC 382 / 390 2.7 364.0 2.9X + Native ORC MR 1107 / 1111 0.9 1055.4 1.0X + Native ORC Vectorized 93 / 102 11.3 88.8 11.9X + Hive built-in ORC 377 / 389 2.8 359.6 2.9X SQL Single Column Scan from 200 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2278 / 2287 0.5 2172.0 1.0X - Native ORC Vectorized 158 / 165 6.6 150.6 14.4X - Hive built-in ORC 585 / 590 1.8 557.7 3.9X + Native ORC MR 2258 / 2270 0.5 2153.8 1.0X + Native ORC Vectorized 160 / 168 6.6 152.6 14.1X + Hive built-in ORC 591 / 597 1.8 563.7 3.8X SQL Single Column Scan from 300 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 3386 / 3394 0.3 3229.1 1.0X - Native ORC Vectorized 271 / 281 3.9 258.2 12.5X - Hive built-in ORC 843 / 852 1.2 803.6 4.0X + Native ORC MR 3364 / 3391 0.3 3208.3 1.0X + Native ORC Vectorized 273 / 284 3.8 260.1 12.3X + Hive built-in ORC 831 / 842 1.3 792.8 4.0X */ sqlBenchmark.run() } From 91b3d662fd99ad099b3d1226a8ecb261a6db0ae0 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 8 Jan 2018 19:22:31 -0800 Subject: [PATCH 19/23] Address comments. --- .../execution/datasources/orc/OrcColumnarBatchReader.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java index caffdbd7cacf5..744bf3fb5e71a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java @@ -472,7 +472,7 @@ private void putNonNullFloats(int count, DoubleColumnVector fromColumn, Writable private void putNonNullDecimals(int count, DecimalColumnVector fromColumn, WritableColumnVector toColumn, DecimalType type) { DecimalColumnVector data = fromColumn; - if (type.precision() > Decimal.MAX_INT_DIGITS()) { + if (type.precision() > Decimal.MAX_LONG_DIGITS()) { WritableColumnVector arrayData = toColumn.getChildColumn(0); arrayData.reserve(count * 16); } @@ -590,6 +590,10 @@ private void putByteArrays(int count, BytesColumnVector fromColumn, WritableColu private void putDecimals(int count, DecimalColumnVector fromColumn, WritableColumnVector toColumn, DecimalType type) { HiveDecimalWritable[] vector = fromColumn.vector; + if (type.precision() > Decimal.MAX_LONG_DIGITS()) { + WritableColumnVector arrayData = toColumn.getChildColumn(0); + arrayData.reserve(count * 16); + } for (int index = 0; index < count; index++) { if (fromColumn.isNull[index]) { toColumn.putNull(index); From db025552700f174686ddea9f6ea6f13078a64079 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 8 Jan 2018 19:54:12 -0800 Subject: [PATCH 20/23] Use recordReader.getProgress --- .../sql/execution/datasources/orc/OrcColumnarBatchReader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java index 744bf3fb5e71a..fe8f51dabef2c 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java @@ -115,7 +115,7 @@ public ColumnarBatch getCurrentValue() throws IOException, InterruptedException @Override public float getProgress() throws IOException, InterruptedException { - return (float) rowsReturned / totalRowCount; + return recordReader.getProgress(); } @Override From 8abbbc3946550a24c435afab9ebf77ddb91ad002 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 8 Jan 2018 20:34:19 -0800 Subject: [PATCH 21/23] Revert "Address comments." This reverts commit 91b3d662fd99ad099b3d1226a8ecb261a6db0ae0. --- .../execution/datasources/orc/OrcColumnarBatchReader.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java index fe8f51dabef2c..4edfed4bc640b 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java @@ -472,7 +472,7 @@ private void putNonNullFloats(int count, DoubleColumnVector fromColumn, Writable private void putNonNullDecimals(int count, DecimalColumnVector fromColumn, WritableColumnVector toColumn, DecimalType type) { DecimalColumnVector data = fromColumn; - if (type.precision() > Decimal.MAX_LONG_DIGITS()) { + if (type.precision() > Decimal.MAX_INT_DIGITS()) { WritableColumnVector arrayData = toColumn.getChildColumn(0); arrayData.reserve(count * 16); } @@ -590,10 +590,6 @@ private void putByteArrays(int count, BytesColumnVector fromColumn, WritableColu private void putDecimals(int count, DecimalColumnVector fromColumn, WritableColumnVector toColumn, DecimalType type) { HiveDecimalWritable[] vector = fromColumn.vector; - if (type.precision() > Decimal.MAX_LONG_DIGITS()) { - WritableColumnVector arrayData = toColumn.getChildColumn(0); - arrayData.reserve(count * 16); - } for (int index = 0; index < count; index++) { if (fromColumn.isNull[index]) { toColumn.putNull(index); From 42182e397d6486cc7e50c3088ea09b69da408695 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 8 Jan 2018 20:34:27 -0800 Subject: [PATCH 22/23] Revert "Split functions." This reverts commit 8fc2162c3be968324c40a8717e4ddcc5cf173ec9. --- .../orc/OrcColumnarBatchReader.java | 403 +++++++----------- .../spark/sql/hive/orc/OrcReadBenchmark.scala | 114 ++--- 2 files changed, 222 insertions(+), 295 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java index 4edfed4bc640b..c3e29c76e5289 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java @@ -49,21 +49,6 @@ * After creating, `initialize` and `initBatch` should be called sequentially. */ public class OrcColumnarBatchReader extends RecordReader { - /** - * The default size of batch. We use this value for both ORC and Spark consistently - * because they have different default values like the following. - * - * - ORC's VectorizedRowBatch.DEFAULT_SIZE = 1024 - * - Spark's ColumnarBatch.DEFAULT_BATCH_SIZE = 4 * 1024 - */ - public static final int DEFAULT_SIZE = 4 * 1024; - - /** - * Returns the number of micros since epoch from an element of TimestampColumnVector. - */ - private static long fromTimestampColumnVector(TimestampColumnVector vector, int index) { - return vector.time[index] * 1000L + vector.nanos[index] / 1000L; - } // ORC File Reader private Reader reader; @@ -268,7 +253,14 @@ private void putRepeatingValues( } else if (type instanceof DoubleType) { toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]); } else if (type instanceof StringType || type instanceof BinaryType) { - putByteArrays(batchSize, toColumn, ((BytesColumnVector)fromColumn).vector[0]); + BytesColumnVector data = (BytesColumnVector)fromColumn; + WritableColumnVector arrayData = toColumn.getChildColumn(0); + int size = data.vector[0].length; + arrayData.reserve(size); + arrayData.putBytes(0, size, data.vector[0], 0); + for (int index = 0; index < batchSize; index++) { + toColumn.putArray(index, 0, size); + } } else if (type instanceof DecimalType) { DecimalType decimalType = (DecimalType)type; putDecimalWritables( @@ -290,31 +282,68 @@ private void putNonNullValues( WritableColumnVector toColumn) { DataType type = field.dataType(); if (type instanceof BooleanType) { - putNonNullBooleans(batchSize, (LongColumnVector) fromColumn, toColumn); + long[] data = ((LongColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + toColumn.putBoolean(index, data[index] == 1); + } } else if (type instanceof ByteType) { - putNonNullBytes(batchSize, (LongColumnVector) fromColumn, toColumn); + long[] data = ((LongColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + toColumn.putByte(index, (byte)data[index]); + } } else if (type instanceof ShortType) { - putNonNullShorts(batchSize, (LongColumnVector) fromColumn, toColumn); + long[] data = ((LongColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + toColumn.putShort(index, (short)data[index]); + } } else if (type instanceof IntegerType || type instanceof DateType) { - putNonNullInts(batchSize, (LongColumnVector) fromColumn, toColumn); + long[] data = ((LongColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + toColumn.putInt(index, (int)data[index]); + } } else if (type instanceof LongType) { toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector, 0); } else if (type instanceof TimestampType) { - putNonNullTimestamps(batchSize, (TimestampColumnVector) fromColumn, toColumn); + TimestampColumnVector data = ((TimestampColumnVector)fromColumn); + for (int index = 0; index < batchSize; index++) { + toColumn.putLong(index, fromTimestampColumnVector(data, index)); + } } else if (type instanceof FloatType) { - putNonNullFloats(batchSize, (DoubleColumnVector) fromColumn, toColumn); + double[] data = ((DoubleColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + toColumn.putFloat(index, (float)data[index]); + } } else if (type instanceof DoubleType) { toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0); } else if (type instanceof StringType || type instanceof BinaryType) { - putNonNullByteArray(batchSize, (BytesColumnVector) fromColumn, toColumn); + BytesColumnVector data = ((BytesColumnVector)fromColumn); + WritableColumnVector arrayData = toColumn.getChildColumn(0); + int totalNumBytes = IntStream.of(data.length).sum(); + arrayData.reserve(totalNumBytes); + for (int index = 0, pos = 0; index < batchSize; pos += data.length[index], index++) { + arrayData.putBytes(pos, data.length[index], data.vector[index], data.start[index]); + toColumn.putArray(index, pos, data.length[index]); + } } else if (type instanceof DecimalType) { - putNonNullDecimals(batchSize, (DecimalColumnVector) fromColumn, toColumn, (DecimalType) type); + DecimalType decimalType = (DecimalType)type; + DecimalColumnVector data = ((DecimalColumnVector)fromColumn); + if (decimalType.precision() > Decimal.MAX_INT_DIGITS()) { + WritableColumnVector arrayData = toColumn.getChildColumn(0); + arrayData.reserve(batchSize * 16); + } + for (int index = 0; index < batchSize; index++) { + putDecimalWritable( + toColumn, + index, + decimalType.precision(), + decimalType.scale(), + data.vector[index]); + } } else { throw new UnsupportedOperationException("Unsupported Data Type: " + type); } } - private void putValues( int batchSize, StructField field, @@ -322,34 +351,129 @@ private void putValues( WritableColumnVector toColumn) { DataType type = field.dataType(); if (type instanceof BooleanType) { - putBooleans(batchSize, (LongColumnVector)fromColumn, toColumn); + long[] vector = ((LongColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + toColumn.putBoolean(index, vector[index] == 1); + } + } } else if (type instanceof ByteType) { - putBytes(batchSize, (LongColumnVector)fromColumn, toColumn); + long[] vector = ((LongColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + toColumn.putByte(index, (byte)vector[index]); + } + } } else if (type instanceof ShortType) { - putShorts(batchSize, (LongColumnVector)fromColumn, toColumn); + long[] vector = ((LongColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + toColumn.putShort(index, (short)vector[index]); + } + } } else if (type instanceof IntegerType || type instanceof DateType) { - putInts(batchSize, (LongColumnVector)fromColumn, toColumn); + long[] vector = ((LongColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + toColumn.putInt(index, (int)vector[index]); + } + } } else if (type instanceof LongType) { - putLongs(batchSize, (LongColumnVector)fromColumn, toColumn); + long[] vector = ((LongColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + toColumn.putLong(index, vector[index]); + } + } } else if (type instanceof TimestampType) { - putTimestamps(batchSize, (TimestampColumnVector)fromColumn, toColumn); + TimestampColumnVector vector = ((TimestampColumnVector)fromColumn); + for (int index = 0; index < batchSize; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + toColumn.putLong(index, fromTimestampColumnVector(vector, index)); + } + } } else if (type instanceof FloatType) { - putFloats(batchSize, (DoubleColumnVector)fromColumn, toColumn); + double[] vector = ((DoubleColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + toColumn.putFloat(index, (float)vector[index]); + } + } } else if (type instanceof DoubleType) { - putDoubles(batchSize, (DoubleColumnVector)fromColumn, toColumn); + double[] vector = ((DoubleColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + toColumn.putDouble(index, vector[index]); + } + } } else if (type instanceof StringType || type instanceof BinaryType) { - putByteArrays(batchSize, (BytesColumnVector)fromColumn, toColumn); + BytesColumnVector vector = (BytesColumnVector)fromColumn; + WritableColumnVector arrayData = toColumn.getChildColumn(0); + int totalNumBytes = IntStream.of(vector.length).sum(); + arrayData.reserve(totalNumBytes); + for (int index = 0, pos = 0; index < batchSize; pos += vector.length[index], index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + arrayData.putBytes(pos, vector.length[index], vector.vector[index], vector.start[index]); + toColumn.putArray(index, pos, vector.length[index]); + } + } } else if (type instanceof DecimalType) { - putDecimals(batchSize, (DecimalColumnVector)fromColumn, toColumn, (DecimalType) type); + DecimalType decimalType = (DecimalType)type; + HiveDecimalWritable[] vector = ((DecimalColumnVector)fromColumn).vector; + for (int index = 0; index < batchSize; index++) { + if (fromColumn.isNull[index]) { + toColumn.putNull(index); + } else { + putDecimalWritable( + toColumn, + index, + decimalType.precision(), + decimalType.scale(), + vector[index]); + } + } } else { throw new UnsupportedOperationException("Unsupported Data Type: " + type); } } - // -------------------------------------------------------------------------- - // Put a value - // -------------------------------------------------------------------------- + /** + * The default size of batch. We use this value for both ORC and Spark consistently + * because they have different default values like the following. + * + * - ORC's VectorizedRowBatch.DEFAULT_SIZE = 1024 + * - Spark's ColumnarBatch.DEFAULT_BATCH_SIZE = 4 * 1024 + */ + public static final int DEFAULT_SIZE = 4 * 1024; + + /** + * Returns the number of micros since epoch from an element of TimestampColumnVector. + */ + private static long fromTimestampColumnVector(TimestampColumnVector vector, int index) { + return vector.time[index] * 1000L + vector.nanos[index] / 1000L; + } + /** + * Put a `HiveDecimalWritable` to a `WritableColumnVector`. + */ private static void putDecimalWritable( WritableColumnVector toColumn, int index, @@ -373,20 +497,9 @@ private static void putDecimalWritable( } } - // -------------------------------------------------------------------------- - // Put repeating values - // -------------------------------------------------------------------------- - - private void putByteArrays(int count, WritableColumnVector toColumn, byte[] bytes) { - WritableColumnVector arrayData = toColumn.getChildColumn(0); - int size = bytes.length; - arrayData.reserve(size); - arrayData.putBytes(0, size, bytes, 0); - for (int index = 0; index < count; index++) { - toColumn.putArray(index, 0, size); - } - } - + /** + * Put `HiveDecimalWritable`s to a `WritableColumnVector`. + */ private static void putDecimalWritables( WritableColumnVector toColumn, int size, @@ -412,190 +525,4 @@ private static void putDecimalWritables( } } } - - // -------------------------------------------------------------------------- - // Put non-null values - // -------------------------------------------------------------------------- - - private void putNonNullBooleans(int count, LongColumnVector fromColumn, WritableColumnVector toColumn) { - long[] data = fromColumn.vector; - for (int index = 0; index < count; index++) { - toColumn.putBoolean(index, data[index] == 1); - } - } - - private void putNonNullByteArray(int count, BytesColumnVector fromColumn, WritableColumnVector toColumn) { - BytesColumnVector data = fromColumn; - WritableColumnVector arrayData = toColumn.getChildColumn(0); - int totalNumBytes = IntStream.of(data.length).sum(); - arrayData.reserve(totalNumBytes); - for (int index = 0, pos = 0; index < count; pos += data.length[index], index++) { - arrayData.putBytes(pos, data.length[index], data.vector[index], data.start[index]); - toColumn.putArray(index, pos, data.length[index]); - } - } - - private void putNonNullBytes(int count, LongColumnVector fromColumn, WritableColumnVector toColumn) { - long[] data = fromColumn.vector; - for (int index = 0; index < count; index++) { - toColumn.putByte(index, (byte)data[index]); - } - } - - private void putNonNullShorts(int count, LongColumnVector fromColumn, WritableColumnVector toColumn) { - long[] data = fromColumn.vector; - for (int index = 0; index < count; index++) { - toColumn.putShort(index, (short)data[index]); - } - } - - private void putNonNullInts(int count, LongColumnVector fromColumn, WritableColumnVector toColumn) { - long[] data = fromColumn.vector; - for (int index = 0; index < count; index++) { - toColumn.putInt(index, (int)data[index]); - } - } - - private void putNonNullTimestamps(int count, TimestampColumnVector fromColumn, WritableColumnVector toColumn) { - TimestampColumnVector data = fromColumn; - for (int index = 0; index < count; index++) { - toColumn.putLong(index, fromTimestampColumnVector(data, index)); - } - } - - private void putNonNullFloats(int count, DoubleColumnVector fromColumn, WritableColumnVector toColumn) { - double[] data = fromColumn.vector; - for (int index = 0; index < count; index++) { - toColumn.putFloat(index, (float)data[index]); - } - } - - private void putNonNullDecimals(int count, DecimalColumnVector fromColumn, WritableColumnVector toColumn, DecimalType type) { - DecimalColumnVector data = fromColumn; - if (type.precision() > Decimal.MAX_INT_DIGITS()) { - WritableColumnVector arrayData = toColumn.getChildColumn(0); - arrayData.reserve(count * 16); - } - for (int index = 0; index < count; index++) { - putDecimalWritable(toColumn, index, type.precision(), type.scale(), data.vector[index]); - } - } - - // -------------------------------------------------------------------------- - // Put values - // -------------------------------------------------------------------------- - - private void putBooleans(int count, LongColumnVector fromColumn, WritableColumnVector toColumn) { - long[] vector = fromColumn.vector; - for (int index = 0; index < count; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putBoolean(index, vector[index] == 1); - } - } - } - - private void putBytes(int count, LongColumnVector fromColumn, WritableColumnVector toColumn) { - long[] vector = fromColumn.vector; - for (int index = 0; index < count; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putByte(index, (byte)vector[index]); - } - } - } - - private void putShorts(int count, LongColumnVector fromColumn, WritableColumnVector toColumn) { - long[] vector = fromColumn.vector; - for (int index = 0; index < count; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putShort(index, (short)vector[index]); - } - } - } - - private void putInts(int count, LongColumnVector fromColumn, WritableColumnVector toColumn) { - long[] vector = fromColumn.vector; - for (int index = 0; index < count; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putInt(index, (int)vector[index]); - } - } - } - - private void putLongs(int count, LongColumnVector fromColumn, WritableColumnVector toColumn) { - long[] vector = fromColumn.vector; - for (int index = 0; index < count; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putLong(index, vector[index]); - } - } - } - - private void putTimestamps(int count, TimestampColumnVector fromColumn, WritableColumnVector toColumn) { - TimestampColumnVector vector = fromColumn; - for (int index = 0; index < count; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putLong(index, fromTimestampColumnVector(vector, index)); - } - } - } - - private void putFloats(int count, DoubleColumnVector fromColumn, WritableColumnVector toColumn) { - double[] vector = fromColumn.vector; - for (int index = 0; index < count; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putFloat(index, (float)vector[index]); - } - } - } - - private void putDoubles(int count, DoubleColumnVector fromColumn, WritableColumnVector toColumn) { - double[] vector = fromColumn.vector; - for (int index = 0; index < count; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putDouble(index, vector[index]); - } - } - } - - private void putByteArrays(int count, BytesColumnVector fromColumn, WritableColumnVector toColumn) { - BytesColumnVector vector = fromColumn; - WritableColumnVector arrayData = toColumn.getChildColumn(0); - int totalNumBytes = IntStream.of(vector.length).sum(); - arrayData.reserve(totalNumBytes); - for (int index = 0, pos = 0; index < count; pos += vector.length[index], index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - arrayData.putBytes(pos, vector.length[index], vector.vector[index], vector.start[index]); - toColumn.putArray(index, pos, vector.length[index]); - } - } - } - - private void putDecimals(int count, DecimalColumnVector fromColumn, WritableColumnVector toColumn, DecimalType type) { - HiveDecimalWritable[] vector = fromColumn.vector; - for (int index = 0; index < count; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - putDecimalWritable(toColumn, index, type.precision(), type.scale(), vector[index]); - } - } - } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala index 0ce89f833b274..e06da64c25e0d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala @@ -110,44 +110,44 @@ object OrcReadBenchmark { } /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2 + Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz SQL Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1202 / 1265 13.1 76.4 1.0X - Native ORC Vectorized 162 / 172 97.1 10.3 7.4X - Hive built-in ORC 1410 / 1428 11.2 89.6 0.9X + Native ORC MR 1180 / 1230 13.3 75.0 1.0X + Native ORC Vectorized 159 / 169 98.8 10.1 7.4X + Hive built-in ORC 1395 / 1396 11.3 88.7 0.8X SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1266 / 1286 12.4 80.5 1.0X - Native ORC Vectorized 165 / 174 95.2 10.5 7.7X - Hive built-in ORC 1701 / 1704 9.2 108.2 0.7X + Native ORC MR 1216 / 1267 12.9 77.3 1.0X + Native ORC Vectorized 163 / 172 96.4 10.4 7.5X + Hive built-in ORC 1649 / 1672 9.5 104.8 0.7X SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1307 / 1307 12.0 83.1 1.0X - Native ORC Vectorized 232 / 248 67.9 14.7 5.6X - Hive built-in ORC 1793 / 1793 8.8 114.0 0.7X + Native ORC MR 1331 / 1332 11.8 84.6 1.0X + Native ORC Vectorized 233 / 245 67.6 14.8 5.7X + Hive built-in ORC 1832 / 1839 8.6 116.5 0.7X SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1360 / 1372 11.6 86.5 1.0X - Native ORC Vectorized 293 / 303 53.8 18.6 4.7X - Hive built-in ORC 1913 / 1933 8.2 121.6 0.7X + Native ORC MR 1382 / 1389 11.4 87.8 1.0X + Native ORC Vectorized 291 / 299 54.1 18.5 4.8X + Hive built-in ORC 1926 / 1936 8.2 122.5 0.7X SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1389 / 1488 11.3 88.3 1.0X - Native ORC Vectorized 340 / 346 46.2 21.6 4.1X - Hive built-in ORC 1976 / 1997 8.0 125.6 0.7X + Native ORC MR 1399 / 1478 11.2 88.9 1.0X + Native ORC Vectorized 324 / 329 48.6 20.6 4.3X + Hive built-in ORC 1938 / 1945 8.1 123.2 0.7X SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1461 / 1465 10.8 92.9 1.0X - Native ORC Vectorized 395 / 406 39.8 25.1 3.7X - Hive built-in ORC 2127 / 2146 7.4 135.2 0.7X + Native ORC MR 1441 / 1470 10.9 91.6 1.0X + Native ORC Vectorized 406 / 408 38.8 25.8 3.6X + Hive built-in ORC 2031 / 2039 7.7 129.1 0.7X */ sqlBenchmark.run() } @@ -181,14 +181,14 @@ object OrcReadBenchmark { } /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2 + Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2843 / 2935 3.7 271.1 1.0X - Native ORC Vectorized 1349 / 1359 7.8 128.6 2.1X - Hive built-in ORC 3862 / 3881 2.7 368.3 0.7X + Native ORC MR 2716 / 2738 3.9 259.0 1.0X + Native ORC Vectorized 1325 / 1325 7.9 126.4 2.0X + Hive built-in ORC 3607 / 3645 2.9 344.0 0.8X */ benchmark.run() } @@ -248,20 +248,20 @@ object OrcReadBenchmark { } /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2 + Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Read data column - Native ORC MR 1545 / 1568 10.2 98.2 1.0X - Read data column - Native ORC Vectorized 300 / 304 52.4 19.1 5.1X - Read data column - Hive built-in ORC 2097 / 2117 7.5 133.3 0.7X - Read partition column - Native ORC MR 1023 / 1026 15.4 65.1 1.5X - Read partition column - Native ORC Vectorized 54 / 56 292.3 3.4 28.7X - Read partition column - Hive built-in ORC 1285 / 1289 12.2 81.7 1.2X - Read both columns - Native ORC MR 1564 / 1565 10.1 99.4 1.0X - Read both columns - Native ORC Vectorized 336 / 340 46.8 21.4 4.6X - Read both columns - Hive built-in ORC 2100 / 2123 7.5 133.5 0.7X + Read data column - Native ORC MR 1544 / 1547 10.2 98.1 1.0X + Read data column - Native ORC Vectorized 298 / 306 52.7 19.0 5.2X + Read data column - Hive built-in ORC 2089 / 2097 7.5 132.8 0.7X + Read partition column - Native ORC MR 1050 / 1051 15.0 66.8 1.5X + Read partition column - Native ORC Vectorized 54 / 57 290.0 3.4 28.5X + Read partition column - Hive built-in ORC 1271 / 1280 12.4 80.8 1.2X + Read both columns - Native ORC MR 1572 / 1605 10.0 100.0 1.0X + Read both columns - Native ORC Vectorized 332 / 338 47.4 21.1 4.6X + Read both columns - Hive built-in ORC 2108 / 2123 7.5 134.0 0.7X */ benchmark.run() } @@ -292,14 +292,14 @@ object OrcReadBenchmark { } /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2 + Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz Repeated String: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1338 / 1340 7.8 127.6 1.0X - Native ORC Vectorized 342 / 350 30.6 32.6 3.9X - Hive built-in ORC 2036 / 2117 5.2 194.2 0.7X + Native ORC MR 1297 / 1327 8.1 123.7 1.0X + Native ORC Vectorized 317 / 327 33.1 30.2 4.1X + Hive built-in ORC 1970 / 1973 5.3 187.9 0.7X */ benchmark.run() } @@ -337,26 +337,26 @@ object OrcReadBenchmark { } /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2 + Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz String with Nulls Scan (0.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2510 / 2527 4.2 239.3 1.0X - Native ORC Vectorized 1012 / 1012 10.4 96.5 2.5X - Hive built-in ORC 3967 / 3984 2.6 378.3 0.6X + Native ORC MR 2531 / 2542 4.1 241.4 1.0X + Native ORC Vectorized 947 / 952 11.1 90.3 2.7X + Hive built-in ORC 4012 / 4034 2.6 382.6 0.6X String with Nulls Scan (0.5%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2374 / 2388 4.4 226.4 1.0X - Native ORC Vectorized 1269 / 1275 8.3 121.0 1.9X - Hive built-in ORC 2994 / 2998 3.5 285.5 0.8X + Native ORC MR 2388 / 2407 4.4 227.8 1.0X + Native ORC Vectorized 1235 / 1236 8.5 117.8 1.9X + Hive built-in ORC 2951 / 2958 3.6 281.4 0.8X String with Nulls Scan (0.95%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1285 / 1312 8.2 122.6 1.0X - Native ORC Vectorized 500 / 508 21.0 47.7 2.6X - Hive built-in ORC 1630 / 1638 6.4 155.5 0.8X + Native ORC MR 1325 / 1346 7.9 126.4 1.0X + Native ORC Vectorized 460 / 468 22.8 43.9 2.9X + Hive built-in ORC 1600 / 1607 6.6 152.6 0.8X */ benchmark.run() } @@ -391,26 +391,26 @@ object OrcReadBenchmark { } /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2 + Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz SQL Single Column Scan from 100 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1107 / 1111 0.9 1055.4 1.0X - Native ORC Vectorized 93 / 102 11.3 88.8 11.9X - Hive built-in ORC 377 / 389 2.8 359.6 2.9X + Native ORC MR 1107 / 1118 0.9 1056.1 1.0X + Native ORC Vectorized 94 / 100 11.1 89.8 11.8X + Hive built-in ORC 382 / 390 2.7 364.0 2.9X SQL Single Column Scan from 200 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2258 / 2270 0.5 2153.8 1.0X - Native ORC Vectorized 160 / 168 6.6 152.6 14.1X - Hive built-in ORC 591 / 597 1.8 563.7 3.8X + Native ORC MR 2278 / 2287 0.5 2172.0 1.0X + Native ORC Vectorized 158 / 165 6.6 150.6 14.4X + Hive built-in ORC 585 / 590 1.8 557.7 3.9X SQL Single Column Scan from 300 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 3364 / 3391 0.3 3208.3 1.0X - Native ORC Vectorized 273 / 284 3.8 260.1 12.3X - Hive built-in ORC 831 / 842 1.3 792.8 4.0X + Native ORC MR 3386 / 3394 0.3 3229.1 1.0X + Native ORC Vectorized 271 / 281 3.9 258.2 12.5X + Hive built-in ORC 843 / 852 1.2 803.6 4.0X */ sqlBenchmark.run() } From 2cf98b6734c806f66e21df50520a465b03d9f060 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 8 Jan 2018 20:35:58 -0800 Subject: [PATCH 23/23] Remove totalRowCount/rowsReturned and address comments.. --- .../orc/OrcColumnarBatchReader.java | 39 +++--- .../spark/sql/hive/orc/OrcReadBenchmark.scala | 114 +++++++++--------- 2 files changed, 74 insertions(+), 79 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java index c3e29c76e5289..5c28d0e6e507a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java @@ -50,6 +50,15 @@ */ public class OrcColumnarBatchReader extends RecordReader { + /** + * The default size of batch. We use this value for both ORC and Spark consistently + * because they have different default values like the following. + * + * - ORC's VectorizedRowBatch.DEFAULT_SIZE = 1024 + * - Spark's ColumnarBatch.DEFAULT_BATCH_SIZE = 4 * 1024 + */ + public static final int DEFAULT_SIZE = 4 * 1024; + // ORC File Reader private Reader reader; @@ -73,11 +82,6 @@ public class OrcColumnarBatchReader extends RecordReader { // Writable column vectors of the result columnar batch. private WritableColumnVector[] columnVectors; - // The number of rows read and considered to be returned. - private long rowsReturned = 0L; - - private long totalRowCount = 0L; - /** * The memory mode of the columnarBatch */ @@ -138,7 +142,6 @@ public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptCont Reader.Options options = OrcInputFormat.buildOptions(conf, reader, fileSplit.getStart(), fileSplit.getLength()); recordReader = reader.rows(options); - totalRowCount = reader.getNumberOfRows(); } /** @@ -193,19 +196,16 @@ public void initBatch( * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns. */ private boolean nextBatch() throws IOException { - if (rowsReturned >= totalRowCount) { - return false; + for (WritableColumnVector vector : columnVectors) { + vector.reset(); } + columnarBatch.setNumRows(0); recordReader.nextBatch(batch); int batchSize = batch.size; if (batchSize == 0) { return false; } - rowsReturned += batchSize; - for (WritableColumnVector vector : columnVectors) { - vector.reset(); - } columnarBatch.setNumRows(batchSize); for (int i = 0; i < requiredFields.length; i++) { StructField field = requiredFields[i]; @@ -327,7 +327,7 @@ private void putNonNullValues( } else if (type instanceof DecimalType) { DecimalType decimalType = (DecimalType)type; DecimalColumnVector data = ((DecimalColumnVector)fromColumn); - if (decimalType.precision() > Decimal.MAX_INT_DIGITS()) { + if (decimalType.precision() > Decimal.MAX_LONG_DIGITS()) { WritableColumnVector arrayData = toColumn.getChildColumn(0); arrayData.reserve(batchSize * 16); } @@ -438,6 +438,10 @@ private void putValues( } else if (type instanceof DecimalType) { DecimalType decimalType = (DecimalType)type; HiveDecimalWritable[] vector = ((DecimalColumnVector)fromColumn).vector; + if (decimalType.precision() > Decimal.MAX_LONG_DIGITS()) { + WritableColumnVector arrayData = toColumn.getChildColumn(0); + arrayData.reserve(batchSize * 16); + } for (int index = 0; index < batchSize; index++) { if (fromColumn.isNull[index]) { toColumn.putNull(index); @@ -455,15 +459,6 @@ private void putValues( } } - /** - * The default size of batch. We use this value for both ORC and Spark consistently - * because they have different default values like the following. - * - * - ORC's VectorizedRowBatch.DEFAULT_SIZE = 1024 - * - Spark's ColumnarBatch.DEFAULT_BATCH_SIZE = 4 * 1024 - */ - public static final int DEFAULT_SIZE = 4 * 1024; - /** * Returns the number of micros since epoch from an element of TimestampColumnVector. */ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala index e06da64c25e0d..37ed846acd1eb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala @@ -110,44 +110,44 @@ object OrcReadBenchmark { } /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 + Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz SQL Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1180 / 1230 13.3 75.0 1.0X - Native ORC Vectorized 159 / 169 98.8 10.1 7.4X - Hive built-in ORC 1395 / 1396 11.3 88.7 0.8X + Native ORC MR 1192 / 1221 13.2 75.8 1.0X + Native ORC Vectorized 161 / 170 97.5 10.3 7.4X + Hive built-in ORC 1399 / 1413 11.2 89.0 0.9X SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1216 / 1267 12.9 77.3 1.0X - Native ORC Vectorized 163 / 172 96.4 10.4 7.5X - Hive built-in ORC 1649 / 1672 9.5 104.8 0.7X + Native ORC MR 1287 / 1333 12.2 81.8 1.0X + Native ORC Vectorized 164 / 172 95.6 10.5 7.8X + Hive built-in ORC 1629 / 1650 9.7 103.6 0.8X SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1331 / 1332 11.8 84.6 1.0X - Native ORC Vectorized 233 / 245 67.6 14.8 5.7X - Hive built-in ORC 1832 / 1839 8.6 116.5 0.7X + Native ORC MR 1304 / 1388 12.1 82.9 1.0X + Native ORC Vectorized 227 / 240 69.3 14.4 5.7X + Hive built-in ORC 1866 / 1867 8.4 118.6 0.7X SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1382 / 1389 11.4 87.8 1.0X - Native ORC Vectorized 291 / 299 54.1 18.5 4.8X - Hive built-in ORC 1926 / 1936 8.2 122.5 0.7X + Native ORC MR 1331 / 1357 11.8 84.6 1.0X + Native ORC Vectorized 289 / 297 54.4 18.4 4.6X + Hive built-in ORC 1922 / 1929 8.2 122.2 0.7X SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1399 / 1478 11.2 88.9 1.0X - Native ORC Vectorized 324 / 329 48.6 20.6 4.3X - Hive built-in ORC 1938 / 1945 8.1 123.2 0.7X + Native ORC MR 1410 / 1428 11.2 89.7 1.0X + Native ORC Vectorized 328 / 335 48.0 20.8 4.3X + Hive built-in ORC 1929 / 2012 8.2 122.6 0.7X SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1441 / 1470 10.9 91.6 1.0X - Native ORC Vectorized 406 / 408 38.8 25.8 3.6X - Hive built-in ORC 2031 / 2039 7.7 129.1 0.7X + Native ORC MR 1467 / 1485 10.7 93.3 1.0X + Native ORC Vectorized 402 / 411 39.1 25.6 3.6X + Hive built-in ORC 2023 / 2042 7.8 128.6 0.7X */ sqlBenchmark.run() } @@ -181,14 +181,14 @@ object OrcReadBenchmark { } /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 + Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2716 / 2738 3.9 259.0 1.0X - Native ORC Vectorized 1325 / 1325 7.9 126.4 2.0X - Hive built-in ORC 3607 / 3645 2.9 344.0 0.8X + Native ORC MR 2729 / 2744 3.8 260.2 1.0X + Native ORC Vectorized 1318 / 1344 8.0 125.7 2.1X + Hive built-in ORC 3731 / 3782 2.8 355.8 0.7X */ benchmark.run() } @@ -248,20 +248,20 @@ object OrcReadBenchmark { } /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 + Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Read data column - Native ORC MR 1544 / 1547 10.2 98.1 1.0X - Read data column - Native ORC Vectorized 298 / 306 52.7 19.0 5.2X - Read data column - Hive built-in ORC 2089 / 2097 7.5 132.8 0.7X - Read partition column - Native ORC MR 1050 / 1051 15.0 66.8 1.5X - Read partition column - Native ORC Vectorized 54 / 57 290.0 3.4 28.5X - Read partition column - Hive built-in ORC 1271 / 1280 12.4 80.8 1.2X - Read both columns - Native ORC MR 1572 / 1605 10.0 100.0 1.0X - Read both columns - Native ORC Vectorized 332 / 338 47.4 21.1 4.6X - Read both columns - Hive built-in ORC 2108 / 2123 7.5 134.0 0.7X + Read data column - Native ORC MR 1531 / 1536 10.3 97.4 1.0X + Read data column - Native ORC Vectorized 295 / 298 53.3 18.8 5.2X + Read data column - Hive built-in ORC 2125 / 2126 7.4 135.1 0.7X + Read partition column - Native ORC MR 1049 / 1062 15.0 66.7 1.5X + Read partition column - Native ORC Vectorized 54 / 57 290.1 3.4 28.2X + Read partition column - Hive built-in ORC 1282 / 1291 12.3 81.5 1.2X + Read both columns - Native ORC MR 1594 / 1598 9.9 101.3 1.0X + Read both columns - Native ORC Vectorized 332 / 336 47.4 21.1 4.6X + Read both columns - Hive built-in ORC 2145 / 2187 7.3 136.4 0.7X */ benchmark.run() } @@ -292,14 +292,14 @@ object OrcReadBenchmark { } /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 + Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz Repeated String: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1297 / 1327 8.1 123.7 1.0X - Native ORC Vectorized 317 / 327 33.1 30.2 4.1X - Hive built-in ORC 1970 / 1973 5.3 187.9 0.7X + Native ORC MR 1325 / 1328 7.9 126.4 1.0X + Native ORC Vectorized 320 / 330 32.8 30.5 4.1X + Hive built-in ORC 1971 / 1972 5.3 188.0 0.7X */ benchmark.run() } @@ -337,26 +337,26 @@ object OrcReadBenchmark { } /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 + Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz String with Nulls Scan (0.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2531 / 2542 4.1 241.4 1.0X - Native ORC Vectorized 947 / 952 11.1 90.3 2.7X - Hive built-in ORC 4012 / 4034 2.6 382.6 0.6X + Native ORC MR 2553 / 2554 4.1 243.4 1.0X + Native ORC Vectorized 953 / 954 11.0 90.9 2.7X + Hive built-in ORC 3875 / 3898 2.7 369.6 0.7X String with Nulls Scan (0.5%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2388 / 2407 4.4 227.8 1.0X - Native ORC Vectorized 1235 / 1236 8.5 117.8 1.9X - Hive built-in ORC 2951 / 2958 3.6 281.4 0.8X + Native ORC MR 2389 / 2408 4.4 227.8 1.0X + Native ORC Vectorized 1208 / 1209 8.7 115.2 2.0X + Hive built-in ORC 2940 / 2952 3.6 280.4 0.8X String with Nulls Scan (0.95%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1325 / 1346 7.9 126.4 1.0X - Native ORC Vectorized 460 / 468 22.8 43.9 2.9X - Hive built-in ORC 1600 / 1607 6.6 152.6 0.8X + Native ORC MR 1295 / 1311 8.1 123.5 1.0X + Native ORC Vectorized 449 / 457 23.4 42.8 2.9X + Hive built-in ORC 1649 / 1660 6.4 157.3 0.8X */ benchmark.run() } @@ -391,26 +391,26 @@ object OrcReadBenchmark { } /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.1 + Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.2 Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz SQL Single Column Scan from 100 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 1107 / 1118 0.9 1056.1 1.0X - Native ORC Vectorized 94 / 100 11.1 89.8 11.8X - Hive built-in ORC 382 / 390 2.7 364.0 2.9X + Native ORC MR 1103 / 1124 1.0 1052.0 1.0X + Native ORC Vectorized 92 / 100 11.4 87.9 12.0X + Hive built-in ORC 383 / 390 2.7 365.4 2.9X SQL Single Column Scan from 200 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 2278 / 2287 0.5 2172.0 1.0X - Native ORC Vectorized 158 / 165 6.6 150.6 14.4X - Hive built-in ORC 585 / 590 1.8 557.7 3.9X + Native ORC MR 2245 / 2250 0.5 2141.0 1.0X + Native ORC Vectorized 157 / 165 6.7 150.2 14.3X + Hive built-in ORC 587 / 593 1.8 559.4 3.8X SQL Single Column Scan from 300 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ - Native ORC MR 3386 / 3394 0.3 3229.1 1.0X - Native ORC Vectorized 271 / 281 3.9 258.2 12.5X - Hive built-in ORC 843 / 852 1.2 803.6 4.0X + Native ORC MR 3343 / 3350 0.3 3188.3 1.0X + Native ORC Vectorized 265 / 280 3.9 253.2 12.6X + Hive built-in ORC 828 / 842 1.3 789.8 4.0X */ sqlBenchmark.run() }