Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/19651#discussion_r152521905 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcDeserializer.scala --- @@ -0,0 +1,216 @@ +/* + * 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 scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +import org.apache.hadoop.io._ +import org.apache.orc.mapred.{OrcList, OrcMap, OrcStruct, OrcTimestamp} +import org.apache.orc.storage.serde2.io.{DateWritable, HiveDecimalWritable} + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.SpecificInternalRow +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.execution.datasources.orc.OrcUtils.withNullSafe +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +private[orc] class OrcDeserializer( + dataSchema: StructType, + requiredSchema: StructType, + missingColumnNames: Seq[String]) { + + private[this] val mutableRow = new SpecificInternalRow(requiredSchema.map(_.dataType)) + + private[this] val length = requiredSchema.length + + private[this] val unwrappers = requiredSchema.map(_.dataType).map(unwrapperFor).toArray + + def deserialize(orcStruct: OrcStruct): InternalRow = { + var i = 0 + val names = orcStruct.getSchema.getFieldNames + while (i < length) { + val name = requiredSchema(i).name + val writable = if (missingColumnNames.contains(name)) { + null + } else { + if (names.contains(name)) { + orcStruct.getFieldValue(name) + } else { + orcStruct.getFieldValue("_col" + dataSchema.fieldIndex(name)) + } + } + if (writable == null) { + mutableRow.setNullAt(i) + } else { + unwrappers(i)(writable, mutableRow, i) + } + i += 1 + } + mutableRow + } + + private[this] def unwrapperFor(dataType: DataType): (Any, InternalRow, Int) => Unit = + dataType match { + case NullType => + (value: Any, row: InternalRow, ordinal: Int) => + row.setNullAt(ordinal) + + case BooleanType => + (value: Any, row: InternalRow, ordinal: Int) => + row.setBoolean(ordinal, value.asInstanceOf[BooleanWritable].get) + + case ByteType => + (value: Any, row: InternalRow, ordinal: Int) => + row.setByte(ordinal, value.asInstanceOf[ByteWritable].get) + + case ShortType => + (value: Any, row: InternalRow, ordinal: Int) => + row.setShort(ordinal, value.asInstanceOf[ShortWritable].get) + + case IntegerType => + (value: Any, row: InternalRow, ordinal: Int) => + row.setInt(ordinal, value.asInstanceOf[IntWritable].get) + + case LongType => + (value: Any, row: InternalRow, ordinal: Int) => + row.setLong(ordinal, value.asInstanceOf[LongWritable].get) + + case FloatType => + (value: Any, row: InternalRow, ordinal: Int) => + row.setFloat(ordinal, value.asInstanceOf[FloatWritable].get) + + case DoubleType => + (value: Any, row: InternalRow, ordinal: Int) => + row.setDouble(ordinal, value.asInstanceOf[DoubleWritable].get) + + case _ => + val unwrapper = getValueUnwrapper(dataType) + (value: Any, row: InternalRow, ordinal: Int) => + row(ordinal) = unwrapper(value) --- End diff -- it's not only for performance, but also remove duplicated code.
--- --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org