-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-20682][SPARK-15474][SPARK-21791] Add new ORCFileFormat based on ORC 1.4.1 #19651
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
Changes from 19 commits
Commits
Show all changes
22 commits
Select commit
Hold shift + click to select a range
b342196
[SPARK-20682][SPARK-15474][SPARK-21791] Add new ORCFileFormat based o…
dongjoon-hyun ca78ac7
Address comments.
dongjoon-hyun 9d18834
Recover OrcFileFormat back, avoid function serialization, add TODO.
dongjoon-hyun 6971cdf
Simplify `isSearchableType`
dongjoon-hyun b373495
Avoid boxing for primitive types.
dongjoon-hyun de8b509
Use getMissingColumnNames instead of getMissingSchema
dongjoon-hyun 726406f
Move withNullSafe to OrcUtils
dongjoon-hyun 4097457
Handle top-level columns in a while-loop and split the logic for Stru…
dongjoon-hyun 8e0d392
fix
dongjoon-hyun 9e3ac1a
Address comments.
dongjoon-hyun a3ebfbf
fix
dongjoon-hyun cc40fba
Move out column name handling logic.
dongjoon-hyun f482179
Use Updater like Parquet.
dongjoon-hyun e13dfa3
fix
dongjoon-hyun fdab6a7
remove outdate comment.
dongjoon-hyun daef4ba
refactor
cloud-fan f143e17
address comment
cloud-fan 8a34731
Revert the change on HiveInspectors.scala.
dongjoon-hyun 74cb053
Merge branch 'PR-2' into SPARK-20682
dongjoon-hyun eae50b3
bug fix
cloud-fan 520837f
Merge pull request #3 from cloud-fan/orc
dongjoon-hyun 71be008
Restore old ORC implementation.
dongjoon-hyun File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
1 change: 1 addition & 0 deletions
1
...core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -36,6 +36,7 @@ import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap | |
| import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat | ||
| import org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider | ||
| import org.apache.spark.sql.execution.datasources.json.JsonFileFormat | ||
| import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat | ||
| import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat | ||
| import org.apache.spark.sql.execution.streaming._ | ||
| import org.apache.spark.sql.sources._ | ||
|
|
@@ -536,7 +537,7 @@ object DataSource extends Logging { | |
| val parquet = classOf[ParquetFileFormat].getCanonicalName | ||
| val csv = classOf[CSVFileFormat].getCanonicalName | ||
| val libsvm = "org.apache.spark.ml.source.libsvm.LibSVMFileFormat" | ||
| val orc = "org.apache.spark.sql.hive.orc.OrcFileFormat" | ||
| val orc = classOf[OrcFileFormat].getCanonicalName | ||
|
||
|
|
||
| Map( | ||
| "org.apache.spark.sql.jdbc" -> jdbc, | ||
|
|
||
243 changes: 243 additions & 0 deletions
243
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcDeserializer.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,243 @@ | ||
| /* | ||
| * 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.serde2.io.{DateWritable, HiveDecimalWritable} | ||
|
|
||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.catalyst.expressions.{SpecificInternalRow, UnsafeArrayData} | ||
| import org.apache.spark.sql.catalyst.util._ | ||
| import org.apache.spark.sql.types._ | ||
| import org.apache.spark.unsafe.types.UTF8String | ||
|
|
||
| /** | ||
| * A deserializer to deserialize ORC structs to Spark rows. | ||
| */ | ||
| class OrcDeserializer( | ||
| dataSchema: StructType, | ||
| requiredSchema: StructType, | ||
| requestedColIds: Array[Int]) { | ||
|
|
||
| private val resultRow = new SpecificInternalRow(requiredSchema.map(_.dataType)) | ||
|
|
||
| private val fieldWriters: Array[WritableComparable[_] => Unit] = { | ||
| requiredSchema.zipWithIndex | ||
| // The value of missing columns are always null, do not need writers. | ||
| .filterNot { case (_, index) => requestedColIds(index) == -1 } | ||
| .map { case (f, index) => | ||
| val writer = newWriter(f.dataType, new RowUpdater(resultRow)) | ||
| (value: WritableComparable[_]) => writer(index, value) | ||
| }.toArray | ||
| } | ||
|
|
||
| private val validColIds = requestedColIds.filterNot(_ == -1) | ||
|
|
||
| def deserialize(orcStruct: OrcStruct): InternalRow = { | ||
| var i = 0 | ||
| while (i < validColIds.length) { | ||
| val value = orcStruct.getFieldValue(validColIds(i)) | ||
| if (value == null) { | ||
| resultRow.setNullAt(i) | ||
| } else { | ||
| fieldWriters(i)(value) | ||
| } | ||
| i += 1 | ||
| } | ||
| resultRow | ||
| } | ||
|
|
||
| /** | ||
| * Creates a writer to write ORC values to Catalyst data structure at the given ordinal. | ||
| */ | ||
| private def newWriter( | ||
| dataType: DataType, updater: CatalystDataUpdater): (Int, WritableComparable[_]) => Unit = | ||
| dataType match { | ||
| case NullType => (ordinal, _) => | ||
| updater.setNullAt(ordinal) | ||
|
|
||
| case BooleanType => (ordinal, value) => | ||
| updater.setBoolean(ordinal, value.asInstanceOf[BooleanWritable].get) | ||
|
|
||
| case ByteType => (ordinal, value) => | ||
| updater.setByte(ordinal, value.asInstanceOf[ByteWritable].get) | ||
|
|
||
| case ShortType => (ordinal, value) => | ||
| updater.setShort(ordinal, value.asInstanceOf[ShortWritable].get) | ||
|
|
||
| case IntegerType => (ordinal, value) => | ||
| updater.setInt(ordinal, value.asInstanceOf[IntWritable].get) | ||
|
|
||
| case LongType => (ordinal, value) => | ||
| updater.setLong(ordinal, value.asInstanceOf[LongWritable].get) | ||
|
|
||
| case FloatType => (ordinal, value) => | ||
| updater.setFloat(ordinal, value.asInstanceOf[FloatWritable].get) | ||
|
|
||
| case DoubleType => (ordinal, value) => | ||
| updater.setDouble(ordinal, value.asInstanceOf[DoubleWritable].get) | ||
|
|
||
| case StringType => (ordinal, value) => | ||
| updater.set(ordinal, UTF8String.fromBytes(value.asInstanceOf[Text].copyBytes)) | ||
|
|
||
| case BinaryType => (ordinal, value) => | ||
| val binary = value.asInstanceOf[BytesWritable] | ||
| val bytes = new Array[Byte](binary.getLength) | ||
| System.arraycopy(binary.getBytes, 0, bytes, 0, binary.getLength) | ||
| updater.set(ordinal, bytes) | ||
|
|
||
| case DateType => (ordinal, value) => | ||
| updater.setInt(ordinal, DateTimeUtils.fromJavaDate(value.asInstanceOf[DateWritable].get)) | ||
|
|
||
| case TimestampType => (ordinal, value) => | ||
| updater.setLong(ordinal, DateTimeUtils.fromJavaTimestamp(value.asInstanceOf[OrcTimestamp])) | ||
|
|
||
| case DecimalType.Fixed(precision, scale) => (ordinal, value) => | ||
| val decimal = value.asInstanceOf[HiveDecimalWritable].getHiveDecimal() | ||
| val v = Decimal(decimal.bigDecimalValue, decimal.precision(), decimal.scale()) | ||
| v.changePrecision(precision, scale) | ||
| updater.set(ordinal, v) | ||
|
|
||
| case st: StructType => (ordinal, value) => | ||
| val result = new SpecificInternalRow(st) | ||
| val fieldUpdater = new RowUpdater(result) | ||
| val fieldConverters = st.map(_.dataType).map { dt => | ||
| newWriter(dt, fieldUpdater) | ||
| }.toArray | ||
| val orcStruct = value.asInstanceOf[OrcStruct] | ||
|
|
||
| var i = 0 | ||
| while (i < st.length) { | ||
| val value = orcStruct.getFieldValue(i) | ||
| if (value == null) { | ||
| result.setNullAt(i) | ||
| } else { | ||
| fieldConverters(i)(i, value) | ||
| } | ||
| i += 1 | ||
| } | ||
|
|
||
| updater.set(ordinal, result) | ||
|
|
||
| case ArrayType(elementType, _) => (ordinal, value) => | ||
| val orcArray = value.asInstanceOf[OrcList[WritableComparable[_]]] | ||
| val length = orcArray.size() | ||
| val result = createArrayData(elementType, length) | ||
| val elementUpdater = new ArrayDataUpdater(result) | ||
| val elementConverter = newWriter(elementType, elementUpdater) | ||
|
|
||
| var i = 0 | ||
| while (i < length) { | ||
| val value = orcArray.get(i) | ||
| if (value == null) { | ||
| result.setNullAt(i) | ||
| } else { | ||
| elementConverter(i, value) | ||
| } | ||
| i += 1 | ||
| } | ||
|
|
||
| updater.set(ordinal, result) | ||
|
|
||
| case MapType(keyType, valueType, _) => (ordinal, value) => | ||
| val orcMap = value.asInstanceOf[OrcMap[WritableComparable[_], WritableComparable[_]]] | ||
| val length = orcMap.size() | ||
| val keyArray = createArrayData(keyType, length) | ||
| val keyUpdater = new ArrayDataUpdater(keyArray) | ||
| val keyConverter = newWriter(keyType, keyUpdater) | ||
| val valueArray = createArrayData(valueType, length) | ||
| val valueUpdater = new ArrayDataUpdater(valueArray) | ||
| val valueConverter = newWriter(valueType, valueUpdater) | ||
|
|
||
| var i = 0 | ||
| val it = orcMap.entrySet().iterator() | ||
| while (it.hasNext) { | ||
| val entry = it.next() | ||
| keyConverter(i, entry.getKey) | ||
| val value = entry.getValue | ||
| if (value == null) { | ||
| valueArray.setNullAt(i) | ||
| } else { | ||
| valueConverter(i, value) | ||
| } | ||
| i += 1 | ||
| } | ||
|
|
||
| updater.set(ordinal, new ArrayBasedMapData(keyArray, valueArray)) | ||
|
|
||
| case udt: UserDefinedType[_] => newWriter(udt.sqlType, updater) | ||
|
|
||
| case _ => | ||
| throw new UnsupportedOperationException(s"$dataType is not supported yet.") | ||
| } | ||
|
|
||
| private def createArrayData(elementType: DataType, length: Int): ArrayData = elementType match { | ||
| case BooleanType => UnsafeArrayData.fromPrimitiveArray(new Array[Boolean](length)) | ||
| case ByteType => UnsafeArrayData.fromPrimitiveArray(new Array[Byte](length)) | ||
| case ShortType => UnsafeArrayData.fromPrimitiveArray(new Array[Short](length)) | ||
| case IntegerType => UnsafeArrayData.fromPrimitiveArray(new Array[Int](length)) | ||
| case LongType => UnsafeArrayData.fromPrimitiveArray(new Array[Long](length)) | ||
| case FloatType => UnsafeArrayData.fromPrimitiveArray(new Array[Float](length)) | ||
| case DoubleType => UnsafeArrayData.fromPrimitiveArray(new Array[Double](length)) | ||
| case _ => new GenericArrayData(new Array[Any](length)) | ||
| } | ||
|
|
||
| /** | ||
| * A base interface for updating values inside catalyst data structure like `InternalRow` and | ||
| * `ArrayData`. | ||
| */ | ||
| sealed trait CatalystDataUpdater { | ||
| def set(ordinal: Int, value: Any): Unit | ||
|
|
||
| def setNullAt(ordinal: Int): Unit = set(ordinal, null) | ||
| def setBoolean(ordinal: Int, value: Boolean): Unit = set(ordinal, value) | ||
| def setByte(ordinal: Int, value: Byte): Unit = set(ordinal, value) | ||
| def setShort(ordinal: Int, value: Short): Unit = set(ordinal, value) | ||
| def setInt(ordinal: Int, value: Int): Unit = set(ordinal, value) | ||
| def setLong(ordinal: Int, value: Long): Unit = set(ordinal, value) | ||
| def setDouble(ordinal: Int, value: Double): Unit = set(ordinal, value) | ||
| def setFloat(ordinal: Int, value: Float): Unit = set(ordinal, value) | ||
| } | ||
|
|
||
| final class RowUpdater(row: InternalRow) extends CatalystDataUpdater { | ||
| override def setNullAt(ordinal: Int): Unit = row.setNullAt(ordinal) | ||
| override def set(ordinal: Int, value: Any): Unit = row.update(ordinal, value) | ||
|
|
||
| override def setBoolean(ordinal: Int, value: Boolean): Unit = row.setBoolean(ordinal, value) | ||
| override def setByte(ordinal: Int, value: Byte): Unit = row.setByte(ordinal, value) | ||
| override def setShort(ordinal: Int, value: Short): Unit = row.setShort(ordinal, value) | ||
| override def setInt(ordinal: Int, value: Int): Unit = row.setInt(ordinal, value) | ||
| override def setLong(ordinal: Int, value: Long): Unit = row.setLong(ordinal, value) | ||
| override def setDouble(ordinal: Int, value: Double): Unit = row.setDouble(ordinal, value) | ||
| override def setFloat(ordinal: Int, value: Float): Unit = row.setFloat(ordinal, value) | ||
| } | ||
|
|
||
| final class ArrayDataUpdater(array: ArrayData) extends CatalystDataUpdater { | ||
| override def setNullAt(ordinal: Int): Unit = array.setNullAt(ordinal) | ||
| override def set(ordinal: Int, value: Any): Unit = array.update(ordinal, value) | ||
|
|
||
| override def setBoolean(ordinal: Int, value: Boolean): Unit = array.setBoolean(ordinal, value) | ||
| override def setByte(ordinal: Int, value: Byte): Unit = array.setByte(ordinal, value) | ||
| override def setShort(ordinal: Int, value: Short): Unit = array.setShort(ordinal, value) | ||
| override def setInt(ordinal: Int, value: Int): Unit = array.setInt(ordinal, value) | ||
| override def setLong(ordinal: Int, value: Long): Unit = array.setLong(ordinal, value) | ||
| override def setDouble(ordinal: Int, value: Double): Unit = array.setDouble(ordinal, value) | ||
| override def setFloat(ordinal: Int, value: Float): Unit = array.setFloat(ordinal, value) | ||
| } | ||
| } |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is there any issue here, @gatorsmile ?