-
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 2 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 |
|---|---|---|
|
|
@@ -34,6 +34,7 @@ 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.types._ | ||
| import org.apache.spark.util.{SerializableConfiguration, ThreadUtils} | ||
|
|
||
| object OrcUtils extends Logging { | ||
|
|
||
|
|
@@ -82,14 +83,95 @@ 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] | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Read single ORC file schema using native version of ORC | ||
|
||
| */ | ||
| def singleFileSchemaReader(file: String, conf: Configuration, ignoreCorruptFiles: Boolean) | ||
| : Option[StructType] = { | ||
|
||
| OrcUtils.readSchema(new Path(file), conf, ignoreCorruptFiles) | ||
| .map(s => CatalystSqlParser.parseDataType(s.toString).asInstanceOf[StructType]) | ||
| } | ||
|
|
||
| /** | ||
| * Figures out a merged ORC schema with a distributed Spark job. | ||
| */ | ||
| def mergeSchemasInParallel( | ||
| sparkSession: SparkSession, | ||
| files: Seq[FileStatus], | ||
| singleFileSchemaReader: (String, Configuration, Boolean) => Option[StructType]) | ||
| : Option[StructType] = { | ||
|
||
| val serializedConf = new SerializableConfiguration(sparkSession.sessionState.newHadoopConf()) | ||
|
|
||
| val filePaths = files.map(_.getPath.toString) | ||
|
|
||
| // 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(filePaths.size, 1), | ||
| sparkSession.sparkContext.defaultParallelism) | ||
|
|
||
| val ignoreCorruptFiles = sparkSession.sessionState.conf.ignoreCorruptFiles | ||
|
|
||
| // Issues a Spark job to read ORC schema in parallel. | ||
| val partiallyMergedSchemas = | ||
|
||
| sparkSession | ||
| .sparkContext | ||
| .parallelize(filePaths, numParallelism) | ||
| .mapPartitions { iterator => | ||
| // Reads Orc schema in multi-threaded manner. | ||
| val partFiles = iterator.toSeq | ||
| val schemas = ThreadUtils.parmap(partFiles, "readingOrcSchemas", 8) { currentFile => | ||
| singleFileSchemaReader(currentFile, serializedConf.value, ignoreCorruptFiles) | ||
| }.flatten | ||
|
|
||
| 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) | ||
| } | ||
| } | ||
|
|
||
| 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) { | ||
| OrcUtils.mergeSchemasInParallel(sparkSession, files, OrcUtils.singleFileSchemaReader) | ||
| } 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 |
|---|---|---|
|
|
@@ -23,18 +23,19 @@ import java.sql.Timestamp | |
| import java.util.Locale | ||
|
|
||
| import org.apache.hadoop.conf.Configuration | ||
| import org.apache.hadoop.fs.Path | ||
| import org.apache.hadoop.fs.{FileSystem, Path} | ||
| import org.apache.orc.OrcConf.COMPRESS | ||
| import org.apache.orc.OrcFile | ||
| import org.apache.orc.OrcProto.ColumnEncoding.Kind.{DICTIONARY_V2, DIRECT, DIRECT_V2} | ||
| import org.apache.orc.OrcProto.Stream.Kind | ||
| import org.apache.orc.impl.RecordReaderImpl | ||
| import org.scalatest.BeforeAndAfterAll | ||
|
|
||
| import org.apache.spark.SPARK_VERSION_SHORT | ||
| import org.apache.spark.{SPARK_VERSION_SHORT, SparkException} | ||
| import org.apache.spark.sql.{Row, SPARK_VERSION_METADATA_KEY} | ||
| import org.apache.spark.sql.internal.SQLConf | ||
| import org.apache.spark.sql.test.SharedSQLContext | ||
| import org.apache.spark.sql.types.{LongType, StructField, StructType} | ||
| import org.apache.spark.util.Utils | ||
|
|
||
| case class OrcData(intField: Int, stringField: String) | ||
|
|
@@ -188,6 +189,49 @@ abstract class OrcSuite extends OrcTest with BeforeAndAfterAll { | |
| } | ||
| } | ||
|
|
||
| protected def testMergeSchemasInParallel( | ||
| ignoreCorruptFiles: Boolean, | ||
| singleFileSchemaReader: (String, Configuration, Boolean) => Option[StructType]): Unit = { | ||
| withSQLConf( | ||
| SQLConf.IGNORE_CORRUPT_FILES.key -> ignoreCorruptFiles.toString, | ||
| SQLConf.ORC_IMPLEMENTATION.key -> orcImp) { | ||
| withTempDir { dir => | ||
| val fs = FileSystem.get(spark.sessionState.newHadoopConf()) | ||
| val basePath = dir.getCanonicalPath | ||
|
|
||
| val path1 = new Path(basePath, "first") | ||
| val path2 = new Path(basePath, "second") | ||
| val path3 = new Path(basePath, "third") | ||
|
|
||
| spark.range(1).toDF("a").coalesce(1).write.orc(path1.toString) | ||
| spark.range(1, 2).toDF("b").coalesce(1).write.orc(path2.toString) | ||
| spark.range(2, 3).toDF("a").coalesce(1).write.json(path3.toString) | ||
|
|
||
| val fileStatuses = | ||
| Seq(fs.listStatus(path1), fs.listStatus(path2), fs.listStatus(path3)).flatten | ||
|
|
||
| val schema = OrcUtils.mergeSchemasInParallel( | ||
| spark, | ||
| fileStatuses, | ||
| singleFileSchemaReader) | ||
|
|
||
| assert(schema.isDefined == true) | ||
|
||
| assert(schema.get == StructType(Seq( | ||
| StructField("a", LongType, true), | ||
| StructField("b", LongType, true)))) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| protected def testMergeSchemasInParallel( | ||
| singleFileSchemaReader: (String, Configuration, Boolean) => Option[StructType]): Unit = { | ||
| testMergeSchemasInParallel(true, singleFileSchemaReader) | ||
| val exception = intercept[SparkException] { | ||
| testMergeSchemasInParallel(false, singleFileSchemaReader) | ||
| }.getCause | ||
| assert(exception.getCause.getMessage.contains("Could not read footer for file")) | ||
| } | ||
|
|
||
| test("create temporary orc table") { | ||
| checkAnswer(sql("SELECT COUNT(*) FROM normal_orc_source"), Row(10)) | ||
|
|
||
|
|
@@ -377,4 +421,8 @@ class OrcSourceSuite extends OrcSuite with SharedSQLContext { | |
| test("Enforce direct encoding column-wise selectively") { | ||
| testSelectiveDictionaryEncoding(isSelective = true) | ||
| } | ||
|
|
||
| test("SPARK-11412 read and merge orc schemas in parallel") { | ||
| testMergeSchemasInParallel(OrcUtils.singleFileSchemaReader) | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -101,6 +101,19 @@ private[hive] object OrcFileOperator extends Logging { | |
| } | ||
| } | ||
|
|
||
| /** | ||
| * Read single ORC file schema using Hive ORC library | ||
| */ | ||
| def singleFileSchemaReader(file: String, conf: Configuration, ignoreCorruptFiles: Boolean) | ||
| : Option[StructType] = { | ||
|
||
| getFileReader(file, Some(conf), ignoreCorruptFiles).map(reader => { | ||
| val readerInspector = reader.getObjectInspector.asInstanceOf[StructObjectInspector] | ||
| val schema = readerInspector.getTypeName | ||
| logDebug(s"Reading schema from file $file, got Hive schema string: $schema") | ||
| CatalystSqlParser.parseDataType(schema).asInstanceOf[StructType] | ||
| }) | ||
| } | ||
|
|
||
| def getObjectInspector( | ||
| path: String, conf: Option[Configuration]): Option[StructObjectInspector] = { | ||
| getFileReader(path, conf).map(_.getObjectInspector.asInstanceOf[StructObjectInspector]) | ||
|
|
||
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