Github user dongjoon-hyun commented on a diff in the pull request: https://github.com/apache/spark/pull/19651#discussion_r150430312 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcSerializer.scala --- @@ -0,0 +1,205 @@ +/* + * 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.io._ +import org.apache.orc.mapred.{OrcList, OrcMap, OrcStruct, OrcTimestamp} +import org.apache.orc.storage.common.`type`.HiveDecimal +import org.apache.orc.storage.serde2.io.{DateWritable, HiveDecimalWritable} + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{SpecializedGetters, SpecificInternalRow} +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.execution.datasources.orc.OrcUtils.{getTypeDescription, withNullSafe} +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +private[orc] class OrcSerializer(dataSchema: StructType) { + + private[this] lazy val orcStruct: OrcStruct = createOrcValue(dataSchema).asInstanceOf[OrcStruct] + + private[this] lazy val length = dataSchema.length + + private[this] val writers = dataSchema.map(_.dataType).map(makeWriter).toArray + + def serialize(row: InternalRow): OrcStruct = { + var i = 0 + while (i < length) { + if (row.isNullAt(i)) { + orcStruct.setFieldValue(i, null) + } else { + writers(i)(row, i) + } + i += 1 + } + orcStruct + } + + private[this] def makeWriter(dataType: DataType): (SpecializedGetters, Int) => Unit = { + dataType match { + case BooleanType => + (row: SpecializedGetters, ordinal: Int) => + orcStruct.setFieldValue(ordinal, new BooleanWritable(row.getBoolean(ordinal))) + + case ByteType => + (row: SpecializedGetters, ordinal: Int) => + orcStruct.setFieldValue(ordinal, new ByteWritable(row.getByte(ordinal))) + + case ShortType => + (row: SpecializedGetters, ordinal: Int) => + orcStruct.setFieldValue(ordinal, new ShortWritable(row.getShort(ordinal))) + + case IntegerType => + (row: SpecializedGetters, ordinal: Int) => + orcStruct.setFieldValue(ordinal, new IntWritable(row.getInt(ordinal))) + + case LongType => + (row: SpecializedGetters, ordinal: Int) => + orcStruct.setFieldValue(ordinal, new LongWritable(row.getLong(ordinal))) + + case FloatType => + (row: SpecializedGetters, ordinal: Int) => + orcStruct.setFieldValue(ordinal, new FloatWritable(row.getFloat(ordinal))) + + case DoubleType => + (row: SpecializedGetters, ordinal: Int) => + orcStruct.setFieldValue(ordinal, new DoubleWritable(row.getDouble(ordinal))) + + case _ => + val wrapper = getWritableWrapper(dataType) --- End diff -- Please see the previous @cloud-fan 's comment. This function is added to avoid boxing issue.
--- --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org