-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-2883][SQL] Spark Support for ORCFile with New Framework #6135
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
Changes from all commits
a96e8d9
90ded0b
e7f7178
a76d5b8
dc1bfa1
8b885d6
4dbea6e
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,62 @@ | ||
| /* | ||
| * 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 org.apache.hadoop.hive.serde2.objectinspector._ | ||
| import org.apache.hadoop.hive.serde2.objectinspector.primitive._ | ||
|
|
||
| import org.apache.spark.sql.catalyst.expressions.MutableRow | ||
| import org.apache.spark.sql.hive.{HiveInspectors, HiveShim} | ||
|
|
||
| /** | ||
| * We can consolidate TableReader.unwrappers and HiveInspectors.wrapperFor to use | ||
| * this class. | ||
| * | ||
| */ | ||
| private[hive] object HadoopTypeConverter extends HiveInspectors { | ||
| /** | ||
| * Builds specific unwrappers ahead of time according to object inspector | ||
| * types to avoid pattern matching and branching costs per row. | ||
| */ | ||
| def unwrappers(fieldRefs: Seq[StructField]): Seq[(Any, MutableRow, Int) => Unit] = fieldRefs.map { | ||
| _.getFieldObjectInspector match { | ||
| case oi: BooleanObjectInspector => | ||
| (value: Any, row: MutableRow, ordinal: Int) => row.setBoolean(ordinal, oi.get(value)) | ||
| case oi: ByteObjectInspector => | ||
| (value: Any, row: MutableRow, ordinal: Int) => row.setByte(ordinal, oi.get(value)) | ||
| case oi: ShortObjectInspector => | ||
| (value: Any, row: MutableRow, ordinal: Int) => row.setShort(ordinal, oi.get(value)) | ||
| case oi: IntObjectInspector => | ||
| (value: Any, row: MutableRow, ordinal: Int) => row.setInt(ordinal, oi.get(value)) | ||
| case oi: LongObjectInspector => | ||
| (value: Any, row: MutableRow, ordinal: Int) => row.setLong(ordinal, oi.get(value)) | ||
| case oi: FloatObjectInspector => | ||
| (value: Any, row: MutableRow, ordinal: Int) => row.setFloat(ordinal, oi.get(value)) | ||
| case oi: DoubleObjectInspector => | ||
| (value: Any, row: MutableRow, ordinal: Int) => row.setDouble(ordinal, oi.get(value)) | ||
| case oi => | ||
| (value: Any, row: MutableRow, ordinal: Int) => row(ordinal) = unwrap(value, oi) | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Wraps with Hive types based on object inspector. | ||
| */ | ||
| def wrappers(oi: ObjectInspector): Any => Any = wrapperFor(oi) | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,71 @@ | ||
| /* | ||
| * 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 org.apache.hadoop.conf.Configuration | ||
| import org.apache.hadoop.fs.Path | ||
| import org.apache.hadoop.hive.ql.io.orc.{OrcFile, Reader} | ||
| import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector | ||
|
|
||
| import org.apache.spark.Logging | ||
| import org.apache.spark.deploy.SparkHadoopUtil | ||
| import org.apache.spark.sql.hive.HiveMetastoreTypes | ||
| import org.apache.spark.sql.types.StructType | ||
|
|
||
| private[orc] object OrcFileOperator extends Logging{ | ||
|
|
||
| def getFileReader(pathStr: String, config: Option[Configuration] = None ): Reader = { | ||
| val conf = config.getOrElse(new Configuration) | ||
| val fspath = new Path(pathStr) | ||
| val fs = fspath.getFileSystem(conf) | ||
| val orcFiles = listOrcFiles(pathStr, conf) | ||
| OrcFile.createReader(fs, orcFiles(0)) | ||
| } | ||
|
|
||
| def readSchema(path: String, conf: Option[Configuration]): StructType = { | ||
| val reader = getFileReader(path, conf) | ||
| val readerInspector: StructObjectInspector = reader.getObjectInspector | ||
| .asInstanceOf[StructObjectInspector] | ||
| val schema = readerInspector.getTypeName | ||
| HiveMetastoreTypes.toDataType(schema).asInstanceOf[StructType] | ||
| } | ||
|
|
||
| def getObjectInspector(path: String, conf: Option[Configuration]): StructObjectInspector = { | ||
| val reader = getFileReader(path, conf) | ||
| val readerInspector: StructObjectInspector = reader.getObjectInspector | ||
| .asInstanceOf[StructObjectInspector] | ||
| readerInspector | ||
| } | ||
|
|
||
| def listOrcFiles(pathStr: String, conf: Configuration): Seq[Path] = { | ||
| val origPath = new Path(pathStr) | ||
| val fs = origPath.getFileSystem(conf) | ||
| val path = origPath.makeQualified(fs) | ||
| val paths = SparkHadoopUtil.get.listLeafStatuses(fs, origPath) | ||
| .filterNot(_.isDir) | ||
| .map(_.getPath) | ||
| .filterNot(_.getName.startsWith("_")) | ||
| .filterNot(_.getName.startsWith(".")) | ||
|
|
||
| if (paths == null || paths.size == 0) { | ||
| throw new IllegalArgumentException( | ||
| s"orcFileOperator: path $path does not have valid orc files matching the pattern") | ||
| } | ||
| paths | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,76 @@ | ||
| /* | ||
| * 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 org.apache.hadoop.hive.ql.io.sarg.SearchArgument | ||
| import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.Builder | ||
| import org.apache.spark.Logging | ||
| import org.apache.spark.sql.sources._ | ||
|
|
||
| /** | ||
| * It may be optimized by push down partial filters. But we are conservative here. | ||
| * Because if some filters fail to be parsed, the tree may be corrupted, | ||
| * and cannot be used anymore. | ||
| */ | ||
| private[orc] object OrcFilters extends Logging { | ||
| def createFilter(expr: Array[Filter]): Option[SearchArgument] = { | ||
| if (expr.nonEmpty) { | ||
| expr.foldLeft(Some(SearchArgument.FACTORY.newBuilder().startAnd()): Option[Builder]) { | ||
| (maybeBuilder, e) => createFilter(e, maybeBuilder) | ||
| }.map(_.end().build()) | ||
| } else { | ||
| None | ||
| } | ||
| } | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This method can be simplified to: def createFilter(expr: Array[Filter]): Option[SearchArgument] = {
expr.headOption.map { _ =>
val andBuilder = SearchArgument.FACTORY.newBuilder()
expr.foldLeft(andBuilder.startAnd()) { (builder, e) =>
createFilter(e, builder).getOrElse(builder)
}.end().build()
}
}(I omitted the null checking since
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Actually let's make it less cryptic... def createFilter(expr: Array[Filter]): Option[SearchArgument] = {
if (expr.nonEmpty) {
val and = expr.foldLeft(SearchArgument.FACTORY.newBuilder().startAnd()) {
(builder, e) => createFilter(e, builder).getOrElse(builder)
}.end()
Some(and.build())
} else {
None
}
} |
||
|
|
||
| private def createFilter(expression: Filter, maybeBuilder: Option[Builder]): Option[Builder] = { | ||
| maybeBuilder.flatMap { builder => | ||
| expression match { | ||
| case p@And(left, right) => | ||
| for { | ||
| lhs <- createFilter(left, Some(builder.startAnd())) | ||
| rhs <- createFilter(right, Some(lhs)) | ||
| } yield rhs.end() | ||
| case p@Or(left, right) => | ||
| for { | ||
| lhs <- createFilter(left, Some(builder.startOr())) | ||
| rhs <- createFilter(right, Some(lhs)) | ||
| } yield rhs.end() | ||
| case p@Not(child) => | ||
| createFilter(child, Some(builder.startNot())).map(_.end()) | ||
| case p@EqualTo(attribute, value) => | ||
| Some(builder.equals(attribute, value)) | ||
| case p@LessThan(attribute, value) => | ||
| Some(builder.lessThan(attribute, value)) | ||
| case p@LessThanOrEqual(attribute, value) => | ||
| Some(builder.lessThanEquals(attribute, value)) | ||
| case p@GreaterThan(attribute, value) => | ||
| Some(builder.startNot().lessThanEquals(attribute, value).end()) | ||
| case p@GreaterThanOrEqual(attribute, value) => | ||
| Some(builder.startNot().lessThan(attribute, value).end()) | ||
| case p@IsNull(attribute) => | ||
| Some(builder.isNull(attribute)) | ||
| case p@IsNotNull(attribute) => | ||
| Some(builder.startNot().isNull(attribute).end()) | ||
| case p@In(attribute, values) => | ||
| Some(builder.in(attribute, values)) | ||
| case _ => None | ||
| } | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,158 @@ | ||
| /* | ||
| * 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.util.Objects | ||
|
|
||
| import org.apache.hadoop.fs.Path | ||
| import org.apache.hadoop.hive.ql.io.orc.{OrcSerde, OrcOutputFormat} | ||
| import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, StructObjectInspector} | ||
| import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfoUtils, TypeInfo} | ||
| import org.apache.hadoop.io.{Writable, NullWritable} | ||
| import org.apache.hadoop.mapred.{RecordWriter, Reporter, JobConf} | ||
| import org.apache.hadoop.mapreduce.{TaskID, TaskAttemptContext} | ||
|
|
||
| import org.apache.spark.Logging | ||
| import org.apache.spark.annotation.DeveloperApi | ||
| import org.apache.spark.mapred.SparkHadoopMapRedUtil | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.sql.hive.HiveMetastoreTypes | ||
| import org.apache.spark.sql.types.StructType | ||
| import org.apache.spark.sql.{Row, SQLContext} | ||
| import org.apache.spark.sql.sources._ | ||
|
|
||
| /* Implicit conversions */ | ||
| import scala.collection.JavaConversions._ | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Please reorder imports. |
||
|
|
||
|
|
||
| private[sql] class DefaultSource extends FSBasedRelationProvider { | ||
| def createRelation( | ||
| sqlContext: SQLContext, | ||
| paths: Array[String], | ||
| schema: Option[StructType], | ||
| partitionColumns: Option[StructType], | ||
| parameters: Map[String, String]): FSBasedRelation ={ | ||
| val partitionSpec = partitionColumns.map(PartitionSpec(_, Seq.empty[Partition])) | ||
| OrcRelation(paths, parameters, | ||
| schema, partitionSpec)(sqlContext) | ||
| } | ||
| } | ||
|
|
||
|
|
||
| private[sql] class OrcOutputWriter extends OutputWriter with SparkHadoopMapRedUtil { | ||
|
|
||
| var taskAttemptContext: TaskAttemptContext = _ | ||
| var serializer: OrcSerde = _ | ||
| var wrappers: Array[Any => Any] = _ | ||
| var created = false | ||
| var path: String = _ | ||
| var dataSchema: StructType = _ | ||
| var fieldOIs: Array[ObjectInspector] = _ | ||
| var structOI: StructObjectInspector = _ | ||
| var outputData: Array[Any] = _ | ||
| lazy val recordWriter: RecordWriter[NullWritable, Writable] = { | ||
| created = true | ||
| val conf = taskAttemptContext.getConfiguration | ||
| val taskId: TaskID = taskAttemptContext.getTaskAttemptID.getTaskID | ||
| val partition: Int = taskId.getId | ||
| val filename = f"part-r-$partition%05d-${System.currentTimeMillis}%015d.orc" | ||
| val file = new Path(path, filename) | ||
| val fs = file.getFileSystem(conf) | ||
| val outputFormat = new OrcOutputFormat() | ||
| outputFormat.getRecordWriter(fs, | ||
| conf.asInstanceOf[JobConf], | ||
| file.toUri.getPath, Reporter.NULL) | ||
| .asInstanceOf[org.apache.hadoop.mapred.RecordWriter[NullWritable, Writable]] | ||
| } | ||
|
|
||
| override def init(path: String, | ||
| dataSchema: StructType, | ||
| context: TaskAttemptContext): Unit = { | ||
| this.path = path | ||
| taskAttemptContext = context | ||
| val orcSchema = HiveMetastoreTypes.toMetastoreType(dataSchema) | ||
| serializer = new OrcSerde | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Seems that we also need to call
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Here we initialize the ObjectInspector per file bases. The other approach is that we send the schema from the driver side. In that case, it may become complicated if we want to support schema merge in the future.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Just saw your new code.My misunderstanding. ObjectInspector is also specified in serialize, but doing initialization does look more elegant. |
||
| val typeInfo: TypeInfo = | ||
| TypeInfoUtils.getTypeInfoFromTypeString(orcSchema) | ||
| structOI = TypeInfoUtils | ||
| .getStandardJavaObjectInspectorFromTypeInfo(typeInfo) | ||
| .asInstanceOf[StructObjectInspector] | ||
| fieldOIs = structOI | ||
| .getAllStructFieldRefs.map(_.getFieldObjectInspector).toArray | ||
| outputData = new Array[Any](fieldOIs.length) | ||
| wrappers = fieldOIs.map(HadoopTypeConverter.wrappers) | ||
| } | ||
|
|
||
| override def write(row: Row): Unit = { | ||
| var i = 0 | ||
| while (i < row.length) { | ||
| outputData(i) = wrappers(i)(row(i)) | ||
| i += 1 | ||
| } | ||
| val writable = serializer.serialize(outputData, structOI) | ||
| recordWriter.write(NullWritable.get(), writable) | ||
| } | ||
|
|
||
| override def close(): Unit = { | ||
| if (created) { | ||
| recordWriter.close(Reporter.NULL) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| @DeveloperApi | ||
| private[sql] case class OrcRelation( | ||
| override val paths: Array[String], | ||
| parameters: Map[String, String], | ||
| maybeSchema: Option[StructType] = None, | ||
| maybePartitionSpec: Option[PartitionSpec] = None)( | ||
| @transient val sqlContext: SQLContext) | ||
| extends FSBasedRelation(paths, maybePartitionSpec) | ||
| with Logging { | ||
| override val dataSchema: StructType = | ||
| maybeSchema.getOrElse(OrcFileOperator.readSchema(paths(0), | ||
| Some(sqlContext.sparkContext.hadoopConfiguration))) | ||
|
|
||
| override def outputWriterClass: Class[_ <: OutputWriter] = classOf[OrcOutputWriter] | ||
|
|
||
| override def needConversion: Boolean = false | ||
|
|
||
| override def equals(other: Any): Boolean = other match { | ||
| case that: OrcRelation => | ||
| paths.toSet == that.paths.toSet && | ||
| dataSchema == that.dataSchema && | ||
| schema == that.schema && | ||
| partitionColumns == that.partitionColumns | ||
| case _ => false | ||
| } | ||
|
|
||
| override def hashCode(): Int = { | ||
| Objects.hashCode( | ||
| paths.toSet, | ||
| dataSchema, | ||
| schema, | ||
| maybePartitionSpec) | ||
| } | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. New line here. |
||
|
|
||
| override def buildScan(requiredColumns: Array[String], | ||
| filters: Array[Filter], | ||
| inputPaths: Array[String]): RDD[Row] = { | ||
| val output = StructType(requiredColumns.map(dataSchema(_))).toAttributes | ||
| OrcTableScan(output, this, filters, inputPaths).execute() | ||
| } | ||
| } | ||
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.
Change TableReader.unwrapper method to remove this duplicated code?