-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-11412][SQL] Support merge schema for ORC #24043
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
083f6ec
7d833b0
d9a0ff2
70bc31b
1d10350
da35351
e527c19
ac11880
b8d216f
9353214
43f7b58
bc4618f
2ea9eb3
50c3906
a6fc2d0
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,106 @@ | ||
| /* | ||
| * 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 | ||
|
|
||
| import org.apache.hadoop.conf.Configuration | ||
| import org.apache.hadoop.fs.{FileStatus, Path} | ||
|
|
||
| import org.apache.spark.SparkException | ||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.sql.SparkSession | ||
| import org.apache.spark.sql.types.StructType | ||
| import org.apache.spark.util.SerializableConfiguration | ||
|
|
||
| object SchemaMergeUtils extends Logging { | ||
| /** | ||
| * Figures out a merged Parquet/ORC schema with a distributed Spark job. | ||
| */ | ||
| def mergeSchemasInParallel( | ||
| sparkSession: SparkSession, | ||
| files: Seq[FileStatus], | ||
| schemaReader: (Seq[FileStatus], Configuration, Boolean) => Seq[StructType]) | ||
| : Option[StructType] = { | ||
| val serializedConf = new SerializableConfiguration(sparkSession.sessionState.newHadoopConf()) | ||
|
|
||
| // !! HACK ALERT !! | ||
| // Here is a hack for Parquet, but it can be used by Orc as well. | ||
| // | ||
| // Parquet requires `FileStatus`es to read footers. | ||
| // Here we try to send cached `FileStatus`es to executor side to avoid fetching them again. | ||
| // However, `FileStatus` is not `Serializable` | ||
| // but only `Writable`. What makes it worse, for some reason, `FileStatus` doesn't play well | ||
| // with `SerializableWritable[T]` and always causes a weird `IllegalStateException`. These | ||
| // facts virtually prevents us to serialize `FileStatus`es. | ||
| // | ||
| // Since Parquet only relies on path and length information of those `FileStatus`es to read | ||
| // footers, here we just extract them (which can be easily serialized), send them to executor | ||
| // side, and resemble fake `FileStatus`es there. | ||
| val partialFileStatusInfo = files.map(f => (f.getPath.toString, f.getLen)) | ||
|
|
||
| // Set the number of partitions to prevent following schema reads from generating many tasks | ||
| // in case of a small number of orc files. | ||
| val numParallelism = Math.min(Math.max(partialFileStatusInfo.size, 1), | ||
| sparkSession.sparkContext.defaultParallelism) | ||
|
|
||
| val ignoreCorruptFiles = sparkSession.sessionState.conf.ignoreCorruptFiles | ||
|
|
||
| // Issues a Spark job to read Parquet/ORC schema in parallel. | ||
| val partiallyMergedSchemas = | ||
| sparkSession | ||
| .sparkContext | ||
| .parallelize(partialFileStatusInfo, numParallelism) | ||
| .mapPartitions { iterator => | ||
| // Resembles fake `FileStatus`es with serialized path and length information. | ||
| val fakeFileStatuses = iterator.map { case (path, length) => | ||
| new FileStatus(length, false, 0, 0, 0, 0, null, null, null, new Path(path)) | ||
| }.toSeq | ||
|
|
||
| val schemas = schemaReader(fakeFileStatuses, serializedConf.value, ignoreCorruptFiles) | ||
|
|
||
| if (schemas.isEmpty) { | ||
| Iterator.empty | ||
| } else { | ||
| var mergedSchema = schemas.head | ||
| schemas.tail.foreach { schema => | ||
| try { | ||
| mergedSchema = mergedSchema.merge(schema) | ||
| } catch { case cause: SparkException => | ||
| throw new SparkException( | ||
| s"Failed merging schema:\n${schema.treeString}", cause) | ||
| } | ||
| } | ||
| Iterator.single(mergedSchema) | ||
| } | ||
| }.collect() | ||
|
|
||
| if (partiallyMergedSchemas.isEmpty) { | ||
| None | ||
| } else { | ||
| var finalSchema = partiallyMergedSchemas.head | ||
| partiallyMergedSchemas.tail.foreach { schema => | ||
| try { | ||
| finalSchema = finalSchema.merge(schema) | ||
| } catch { case cause: SparkException => | ||
| throw new SparkException( | ||
| s"Failed merging schema:\n${schema.treeString}", cause) | ||
| } | ||
| } | ||
| Some(finalSchema) | ||
| } | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -33,7 +33,9 @@ import org.apache.spark.sql.{SPARK_VERSION_METADATA_KEY, SparkSession} | |
| import org.apache.spark.sql.catalyst.analysis.caseSensitiveResolution | ||
| import org.apache.spark.sql.catalyst.parser.CatalystSqlParser | ||
| import org.apache.spark.sql.catalyst.util.quoteIdentifier | ||
| import org.apache.spark.sql.execution.datasources.SchemaMergeUtils | ||
| import org.apache.spark.sql.types._ | ||
| import org.apache.spark.util.{SerializableConfiguration, ThreadUtils} | ||
|
|
||
| object OrcUtils extends Logging { | ||
|
|
||
|
|
@@ -82,14 +84,36 @@ object OrcUtils extends Logging { | |
| : Option[StructType] = { | ||
| val ignoreCorruptFiles = sparkSession.sessionState.conf.ignoreCorruptFiles | ||
| val conf = sparkSession.sessionState.newHadoopConf() | ||
| // TODO: We need to support merge schema. Please see SPARK-11412. | ||
| files.toIterator.map(file => readSchema(file.getPath, conf, ignoreCorruptFiles)).collectFirst { | ||
| case Some(schema) => | ||
| logDebug(s"Reading schema from file $files, got Hive schema string: $schema") | ||
| CatalystSqlParser.parseDataType(schema.toString).asInstanceOf[StructType] | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Reads ORC file schemas in multi-threaded manner, using native version of ORC. | ||
| * This is visible for testing. | ||
| */ | ||
| def readOrcSchemasInParallel( | ||
| files: Seq[FileStatus], conf: Configuration, ignoreCorruptFiles: Boolean): Seq[StructType] = { | ||
| ThreadUtils.parmap(files, "readingOrcSchemas", 8) { currentFile => | ||
| OrcUtils.readSchema(currentFile.getPath, conf, ignoreCorruptFiles) | ||
| .map(s => CatalystSqlParser.parseDataType(s.toString).asInstanceOf[StructType]) | ||
| }.flatten | ||
| } | ||
|
|
||
| def inferSchema(sparkSession: SparkSession, files: Seq[FileStatus], options: Map[String, String]) | ||
|
Member
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. I think we can keep the name |
||
| : Option[StructType] = { | ||
| val orcOptions = new OrcOptions(options, sparkSession.sessionState.conf) | ||
| if (orcOptions.mergeSchema) { | ||
| SchemaMergeUtils.mergeSchemasInParallel( | ||
| sparkSession, files, OrcUtils.readOrcSchemasInParallel) | ||
| } else { | ||
| OrcUtils.readSchema(sparkSession, files) | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Returns the requested column ids from the given ORC file. Column id can be -1, which means the | ||
| * requested column doesn't exist in the ORC file. Returns None if the given ORC file is empty. | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,6 +17,7 @@ | |
|
|
||
| package org.apache.spark.sql.execution.datasources | ||
|
|
||
| import org.apache.spark.SparkConf | ||
| import org.apache.spark.sql.internal.SQLConf | ||
|
|
||
| /** | ||
|
|
@@ -32,6 +33,7 @@ import org.apache.spark.sql.internal.SQLConf | |
| * | ||
| * -> OrcReadSchemaSuite | ||
| * -> VectorizedOrcReadSchemaSuite | ||
| * -> MergedOrcReadSchemaSuite | ||
| * | ||
| * -> ParquetReadSchemaSuite | ||
| * -> VectorizedParquetReadSchemaSuite | ||
|
|
@@ -134,6 +136,25 @@ class VectorizedOrcReadSchemaSuite | |
| } | ||
| } | ||
|
|
||
| class MergedOrcReadSchemaSuite | ||
| extends ReadSchemaSuite | ||
| with AddColumnIntoTheMiddleTest | ||
| with HideColumnInTheMiddleTest | ||
| with AddNestedColumnTest | ||
| with HideNestedColumnTest | ||
| with ChangePositionTest | ||
| with BooleanTypeTest | ||
| with IntegralTypeTest | ||
| with ToDoubleTypeTest { | ||
|
|
||
| override val format: String = "orc" | ||
|
|
||
|
Member
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. We can use And drop the
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. done |
||
| override protected def sparkConf: SparkConf = | ||
| super | ||
| .sparkConf | ||
| .set(SQLConf.ORC_SCHEMA_MERGING_ENABLED.key, "true") | ||
| } | ||
|
|
||
| class ParquetReadSchemaSuite | ||
| extends ReadSchemaSuite | ||
| with AddColumnIntoTheMiddleTest | ||
|
|
||
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.
We should at least add a test case for this new option
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.
added a new test case in
OrcSuite