From 09cb47daf993159fe619359560a578d6859b2d00 Mon Sep 17 00:00:00 2001 From: Wang Gengliang Date: Thu, 29 Mar 2018 16:33:23 +0800 Subject: [PATCH 01/25] orc v2 --- .../apache/spark/sql/internal/SQLConf.scala | 9 + .../orc/OrcColumnarBatchReader.java | 65 +++-- .../apache/spark/sql/DataFrameReader.scala | 8 + .../sql/execution/DataSourceScanExec.scala | 106 ++------- .../sql/execution/PartitionedFileUtil.scala | 107 +++++++++ .../execution/datasources/DataSource.scala | 2 +- .../datasources/DataSourceStrategy.scala | 18 +- .../datasources/FilePartitionUtil.scala | 225 ++++++++++++++++++ .../datasources/HadoopFsRelation.scala | 22 +- .../datasources/PartitioningUtils.scala | 78 ++++++ .../datasources/orc/OrcFileFormat.scala | 11 +- .../datasources/orc/OrcFilters.scala | 16 +- .../execution/datasources/orc/OrcUtils.scala | 2 + .../datasources/v2/EmptyDataReader.scala | 34 +++ .../v2/orc/OrcColumnarBatchDataReader.scala | 113 +++++++++ .../datasources/v2/orc/OrcDataSourceV2.scala | 184 ++++++++++++++ .../v2/orc/OrcUnsafeRowReaderFactory.scala | 114 +++++++++ .../internal/BaseSessionStateBuilder.scala | 1 + .../datasources/orc/OrcFilterSuite.scala | 56 +++-- .../orc/OrcPartitionDiscoverySuite.scala | 7 + .../datasources/orc/OrcQuerySuite.scala | 7 +- 21 files changed, 1013 insertions(+), 172 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartitionUtil.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyDataReader.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcColumnarBatchDataReader.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcUnsafeRowReaderFactory.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 0b1965c438e2..105282dd61cd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1229,6 +1229,13 @@ object SQLConf { .booleanConf .createWithDefault(true) + val ENABLED_V2_DATA_SOURCE_READERS = buildConf("spark.sql.enabledV2DataSourceReaders") + .internal() + .doc("A comma-separated list of data source short names for which " + + "DataSourceReader is enabled. Reads from these sources will use V2 Sources") + .stringConf + .createWithDefault("orc") + object PartitionOverwriteMode extends Enumeration { val STATIC, DYNAMIC = Value } @@ -1611,6 +1618,8 @@ class SQLConf extends Serializable with Logging { def disabledV2StreamingMicroBatchReaders: String = getConf(DISABLED_V2_STREAMING_MICROBATCH_READERS) + def enabledV2DataSourceReader: String = getConf(ENABLED_V2_DATA_SOURCE_READERS) + def concatBinaryAsString: Boolean = getConf(CONCAT_BINARY_AS_STRING) def eltOutputAsString: Boolean = getConf(ELT_OUTPUT_AS_STRING) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java index dcebdc39f0aa..fb090dde866f 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java @@ -62,6 +62,12 @@ public class OrcColumnarBatchReader extends RecordReader { */ private int[] requestedColIds; + /** + * The column IDs of the ORC file partition schema which are required by this reader. + * -1 means this required column doesn't exist in the ORC partition columns. + */ + private int[] requestedPartitionColds; + // Record reader from ORC row batch. private org.apache.orc.RecordReader recordReader; @@ -146,22 +152,19 @@ public void initialize( */ public void initBatch( TypeDescription orcSchema, - int[] requestedColIds, StructField[] requiredFields, - StructType partitionSchema, + int[] requestedColIds, + int[] requestedPartitionColIds, InternalRow partitionValues) { batch = orcSchema.createRowBatch(capacity); assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`. - this.requiredFields = requiredFields; this.requestedColIds = requestedColIds; + this.requestedPartitionColds = requestedPartitionColIds; assert(requiredFields.length == requestedColIds.length); + assert(requiredFields.length == requestedPartitionColIds.length); StructType resultSchema = new StructType(requiredFields); - for (StructField f : partitionSchema.fields()) { - resultSchema = resultSchema.add(f); - } - if (copyToSpark) { if (MEMORY_MODE == MemoryMode.OFF_HEAP) { columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema); @@ -171,20 +174,16 @@ public void initBatch( // Initialize the missing columns once. for (int i = 0; i < requiredFields.length; i++) { - if (requestedColIds[i] == -1) { + if (requestedPartitionColIds[i] != -1) { + ColumnVectorUtils.populate(columnVectors[i], + partitionValues, requestedPartitionColIds[i]); + columnVectors[i].setIsConstant(); + } else if (requestedColIds[i] == -1) { columnVectors[i].putNulls(0, capacity); columnVectors[i].setIsConstant(); } } - if (partitionValues.numFields() > 0) { - int partitionIdx = requiredFields.length; - for (int i = 0; i < partitionValues.numFields(); i++) { - ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i); - columnVectors[i + partitionIdx].setIsConstant(); - } - } - columnarBatch = new ColumnarBatch(columnVectors); } else { // Just wrap the ORC column vector instead of copying it to Spark column vector. @@ -192,26 +191,22 @@ public void initBatch( for (int i = 0; i < requiredFields.length; i++) { DataType dt = requiredFields[i].dataType(); - int colId = requestedColIds[i]; - // Initialize the missing columns once. - if (colId == -1) { - OnHeapColumnVector missingCol = new OnHeapColumnVector(capacity, dt); - missingCol.putNulls(0, capacity); - missingCol.setIsConstant(); - orcVectorWrappers[i] = missingCol; - } else { - orcVectorWrappers[i] = new OrcColumnVector(dt, batch.cols[colId]); - } - } - - if (partitionValues.numFields() > 0) { - int partitionIdx = requiredFields.length; - for (int i = 0; i < partitionValues.numFields(); i++) { - DataType dt = partitionSchema.fields()[i].dataType(); + if (requestedPartitionColIds[i] != -1) { OnHeapColumnVector partitionCol = new OnHeapColumnVector(capacity, dt); - ColumnVectorUtils.populate(partitionCol, partitionValues, i); + ColumnVectorUtils.populate(partitionCol, partitionValues, requestedPartitionColIds[i]); partitionCol.setIsConstant(); - orcVectorWrappers[partitionIdx + i] = partitionCol; + orcVectorWrappers[i] = partitionCol; + } else { + int colId = requestedColIds[i]; + // Initialize the missing columns once. + if (colId == -1) { + OnHeapColumnVector missingCol = new OnHeapColumnVector(capacity, dt); + missingCol.putNulls(0, capacity); + missingCol.setIsConstant(); + orcVectorWrappers[i] = missingCol; + } else { + orcVectorWrappers[i] = new OrcColumnVector(dt, batch.cols[colId]); + } } } @@ -233,7 +228,7 @@ private boolean nextBatch() throws IOException { if (!copyToSpark) { for (int i = 0; i < requiredFields.length; i++) { - if (requestedColIds[i] != -1) { + if (requestedColIds[i] != -1 && requestedPartitionColds[i] == -1) { ((OrcColumnVector) orcVectorWrappers[i]).setBatchSize(batchSize); } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 53f44888ebaf..afb262dce284 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -29,6 +29,7 @@ import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JacksonParser, JSONOptions} +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.{DataSource, FailureSafeParser} import org.apache.spark.sql.execution.datasources.csv._ @@ -36,6 +37,7 @@ import org.apache.spark.sql.execution.datasources.jdbc._ import org.apache.spark.sql.execution.datasources.json.TextInputJsonDataSource import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils +import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema} import org.apache.spark.sql.types.{StringType, StructType} import org.apache.spark.unsafe.types.UTF8String @@ -190,6 +192,12 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { "read files of Hive data source directly.") } + val allPaths = CaseInsensitiveMap(extraOptions.toMap).get("path") ++ paths + val orcV2 = OrcDataSourceV2.satisfy(sparkSession, source, allPaths.toSeq) + if (orcV2.isDefined) { + option("path", allPaths.head) + source = orcV2.get + } val cls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf) if (classOf[DataSourceV2].isAssignableFrom(cls)) { val ds = cls.newInstance().asInstanceOf[DataSourceV2] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 08ff33afbba3..345af08d478c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -368,8 +368,7 @@ case class FileSourceScanExec( val bucketed = selectedPartitions.flatMap { p => p.files.map { f => - val hosts = getBlockHosts(getBlockLocations(f), 0, f.getLen) - PartitionedFile(p.values, f.getPath.toUri.toString, 0, f.getLen, hosts) + PartitionedFileUtil.getPartitionedFile(f, f.getPath, p.values) } }.groupBy { f => BucketingUtils @@ -396,107 +395,34 @@ case class FileSourceScanExec( readFile: (PartitionedFile) => Iterator[InternalRow], selectedPartitions: Seq[PartitionDirectory], fsRelation: HadoopFsRelation): RDD[InternalRow] = { - val defaultMaxSplitBytes = - fsRelation.sparkSession.sessionState.conf.filesMaxPartitionBytes + val maxSplitBytes = PartitionedFileUtil.maxSplitBytes(relation.sparkSession, selectedPartitions) val openCostInBytes = fsRelation.sparkSession.sessionState.conf.filesOpenCostInBytes - val defaultParallelism = fsRelation.sparkSession.sparkContext.defaultParallelism - val totalBytes = selectedPartitions.flatMap(_.files.map(_.getLen + openCostInBytes)).sum - val bytesPerCore = totalBytes / defaultParallelism - - val maxSplitBytes = Math.min(defaultMaxSplitBytes, Math.max(openCostInBytes, bytesPerCore)) logInfo(s"Planning scan with bin packing, max size: $maxSplitBytes bytes, " + s"open cost is considered as scanning $openCostInBytes bytes.") val splitFiles = selectedPartitions.flatMap { partition => partition.files.flatMap { file => - val blockLocations = getBlockLocations(file) - if (fsRelation.fileFormat.isSplitable( - fsRelation.sparkSession, fsRelation.options, file.getPath)) { - (0L until file.getLen by maxSplitBytes).map { offset => - val remaining = file.getLen - offset - val size = if (remaining > maxSplitBytes) maxSplitBytes else remaining - val hosts = getBlockHosts(blockLocations, offset, size) - PartitionedFile( - partition.values, file.getPath.toUri.toString, offset, size, hosts) - } - } else { - val hosts = getBlockHosts(blockLocations, 0, file.getLen) - Seq(PartitionedFile( - partition.values, file.getPath.toUri.toString, 0, file.getLen, hosts)) - } + // getPath() is very expensive so we only want to call it once in this block: + val filePath = file.getPath + val isSplitable = relation.fileFormat.isSplitable( + relation.sparkSession, relation.options, filePath) + PartitionedFileUtil.splitFiles( + sparkSession = relation.sparkSession, + file = file, + filePath = filePath, + isSplitable = isSplitable, + maxSplitBytes = maxSplitBytes, + partitionValues = partition.values + ) } }.toArray.sortBy(_.length)(implicitly[Ordering[Long]].reverse) - val partitions = new ArrayBuffer[FilePartition] - val currentFiles = new ArrayBuffer[PartitionedFile] - var currentSize = 0L - - /** Close the current partition and move to the next. */ - def closePartition(): Unit = { - if (currentFiles.nonEmpty) { - val newPartition = - FilePartition( - partitions.size, - currentFiles.toArray.toSeq) // Copy to a new Array. - partitions += newPartition - } - currentFiles.clear() - currentSize = 0 - } - - // Assign files to partitions using "Next Fit Decreasing" - splitFiles.foreach { file => - if (currentSize + file.length > maxSplitBytes) { - closePartition() - } - // Add the given file to the current partition. - currentSize += file.length + openCostInBytes - currentFiles += file - } - closePartition() + val partitions = + FilePartitionUtil.getFilePartition(relation.sparkSession, splitFiles, maxSplitBytes) new FileScanRDD(fsRelation.sparkSession, readFile, partitions) } - private def getBlockLocations(file: FileStatus): Array[BlockLocation] = file match { - case f: LocatedFileStatus => f.getBlockLocations - case f => Array.empty[BlockLocation] - } - - // Given locations of all blocks of a single file, `blockLocations`, and an `(offset, length)` - // pair that represents a segment of the same file, find out the block that contains the largest - // fraction the segment, and returns location hosts of that block. If no such block can be found, - // returns an empty array. - private def getBlockHosts( - blockLocations: Array[BlockLocation], offset: Long, length: Long): Array[String] = { - val candidates = blockLocations.map { - // The fragment starts from a position within this block - case b if b.getOffset <= offset && offset < b.getOffset + b.getLength => - b.getHosts -> (b.getOffset + b.getLength - offset).min(length) - - // The fragment ends at a position within this block - case b if offset <= b.getOffset && offset + length < b.getLength => - b.getHosts -> (offset + length - b.getOffset).min(length) - - // The fragment fully contains this block - case b if offset <= b.getOffset && b.getOffset + b.getLength <= offset + length => - b.getHosts -> b.getLength - - // The fragment doesn't intersect with this block - case b => - b.getHosts -> 0L - }.filter { case (hosts, size) => - size > 0L - } - - if (candidates.isEmpty) { - Array.empty[String] - } else { - val (hosts, _) = candidates.maxBy { case (_, size) => size } - hosts - } - } - override def doCanonicalize(): FileSourceScanExec = { FileSourceScanExec( relation, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala new file mode 100644 index 000000000000..06585271c165 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala @@ -0,0 +1,107 @@ +/* + * 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 + +import org.apache.hadoop.fs.{BlockLocation, FileStatus, LocatedFileStatus, Path} + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.{PartitionedFile, _} + +object PartitionedFileUtil { + def splitFiles( + sparkSession: SparkSession, + file: FileStatus, + filePath: Path, + isSplitable: Boolean, + maxSplitBytes: Long, + partitionValues: InternalRow): Seq[PartitionedFile] = { + if (isSplitable) { + (0L until file.getLen by maxSplitBytes).map { offset => + val remaining = file.getLen - offset + val size = if (remaining > maxSplitBytes) maxSplitBytes else remaining + val hosts = getBlockHosts(getBlockLocations(file), offset, size) + PartitionedFile(partitionValues, filePath.toUri.toString, offset, size, hosts) + } + } else { + Seq(getPartitionedFile(file, filePath, partitionValues)) + } + } + + def getPartitionedFile( + file: FileStatus, + filePath: Path, + partitionValues: InternalRow + ): PartitionedFile = { + val hosts = getBlockHosts(getBlockLocations(file), 0, file.getLen) + PartitionedFile(partitionValues, filePath.toUri.toString, 0, file.getLen, hosts) + } + + def maxSplitBytes( + sparkSession: SparkSession, + selectedPartitions: Seq[PartitionDirectory]): Long = { + val defaultMaxSplitBytes = sparkSession.sessionState.conf.filesMaxPartitionBytes + val openCostInBytes = sparkSession.sessionState.conf.filesOpenCostInBytes + val defaultParallelism = sparkSession.sparkContext.defaultParallelism + val totalBytes = selectedPartitions.flatMap(_.files.map(_.getLen + openCostInBytes)).sum + val bytesPerCore = totalBytes / defaultParallelism + + Math.min(defaultMaxSplitBytes, Math.max(openCostInBytes, bytesPerCore)) + } + + private def getBlockLocations(file: FileStatus): Array[BlockLocation] = file match { + case f: LocatedFileStatus => f.getBlockLocations + case f => Array.empty[BlockLocation] + } + // Given locations of all blocks of a single file, `blockLocations`, and an `(offset, length)` + // pair that represents a segment of the same file, find out the block that contains the largest + // fraction the segment, and returns location hosts of that block. If no such block can be found, + // returns an empty array. + private def getBlockHosts( + blockLocations: Array[BlockLocation], + offset: Long, + length: Long): Array[String] = { + val candidates = blockLocations.map { + // The fragment starts from a position within this block + case b if b.getOffset <= offset && offset < b.getOffset + b.getLength => + b.getHosts -> (b.getOffset + b.getLength - offset).min(length) + + // The fragment ends at a position within this block + case b if offset <= b.getOffset && offset + length < b.getLength => + b.getHosts -> (offset + length - b.getOffset).min(length) + + // The fragment fully contains this block + case b if offset <= b.getOffset && b.getOffset + b.getLength <= offset + length => + b.getHosts -> b.getLength + + // The fragment doesn't intersect with this block + case b => + b.getHosts -> 0L + }.filter { case (hosts, size) => + size > 0L + } + + if (candidates.isEmpty) { + Array.empty[String] + } else { + val (hosts, _) = candidates.maxBy { case (_, size) => size } + hosts + } + } +} + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index f16d824201e7..72c170a41314 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -93,6 +93,7 @@ case class DataSource( DataSource.lookupDataSource(className, sparkSession.sessionState.conf) lazy val sourceInfo: SourceInfo = sourceSchema() private val caseInsensitiveOptions = CaseInsensitiveMap(options) + private val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis private val equality = sparkSession.sessionState.conf.resolver bucketSpec.map { bucket => @@ -426,7 +427,6 @@ case class DataSource( s"got: ${allPaths.mkString(", ")}") } - val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis PartitioningUtils.validatePartitionColumn(data.schema, partitionColumns, caseSensitive) val fileIndex = catalogTable.map(_.identifier).map { tableIdent => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 3f41612c0806..cf45c9e996e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -37,6 +37,8 @@ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{RowDataSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ @@ -213,7 +215,21 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast } } - +class FallBackToOrcV1(sparkSession: SparkSession) extends Rule[LogicalPlan] { + private def convertToOrcV1(v2Relation: DataSourceV2Relation): LogicalPlan = { + val v1 = DataSource.apply( + sparkSession = sparkSession, + paths = Seq.empty, + userSpecifiedSchema = v2Relation.userSpecifiedSchema, + className = "org.apache.spark.sql.execution.datasources.orc", + options = v2Relation.options).resolveRelation() + LogicalRelation(v1) + } + override def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case i @InsertIntoTable(d: DataSourceV2Relation, _, _, _, _) + if d.source.isInstanceOf[OrcDataSourceV2] => i.copy(table = convertToOrcV1(d)) + } +} /** * Replaces [[UnresolvedCatalogRelation]] with concrete relation logical plans. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartitionUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartitionUtil.scala new file mode 100644 index 000000000000..c8640776a67c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartitionUtil.scala @@ -0,0 +1,225 @@ +/* + * 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 java.io.{FileNotFoundException, IOException} + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.TaskContext +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.InputFileBlockHolder +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.NextIterator + +object FilePartitionUtil extends Logging { + + def getFilePartition( + sparkSession: SparkSession, + partitionedFiles: Seq[PartitionedFile], + maxSplitBytes: Long): Seq[FilePartition] = { + val partitions = new ArrayBuffer[FilePartition] + val currentFiles = new ArrayBuffer[PartitionedFile] + var currentSize = 0L + + /** Close the current partition and move to the next. */ + def closePartition(): Unit = { + if (currentFiles.nonEmpty) { + val newPartition = + FilePartition( + partitions.size, + currentFiles.toArray.toSeq) // Copy to a new Array. + partitions += newPartition + } + currentFiles.clear() + currentSize = 0 + } + + val openCostInBytes = sparkSession.sessionState.conf.filesOpenCostInBytes + // Assign files to partitions using "Next Fit Decreasing" + partitionedFiles.foreach { file => + if (currentSize + file.length > maxSplitBytes) { + closePartition() + } + // Add the given file to the current partition. + currentSize += file.length + openCostInBytes + currentFiles += file + } + closePartition() + partitions + } + + def compute( + split: FilePartition, + context: TaskContext, + readFunction: (PartitionedFile) => Iterator[InternalRow], + ignoreCorruptFiles: Boolean = false, + ignoreMissingFiles: Boolean = false): Iterator[InternalRow] = { + val iterator = new Iterator[Object] with AutoCloseable { + private val inputMetrics = context.taskMetrics().inputMetrics + private val existingBytesRead = inputMetrics.bytesRead + + // Find a function that will return the FileSystem bytes read by this thread. Do this before + // apply readFunction, because it might read some bytes. + private val getBytesReadCallback = + SparkHadoopUtil.get.getFSBytesReadOnThreadCallback() + + // We get our input bytes from thread-local Hadoop FileSystem statistics. + // If we do a coalesce, however, we are likely to compute multiple partitions in the same + // task and in the same thread, in which case we need to avoid override values written by + // previous partitions (SPARK-13071). + private def updateBytesRead(): Unit = { + inputMetrics.setBytesRead(existingBytesRead + getBytesReadCallback()) + } + + // If we can't get the bytes read from the FS stats, fall back to the file size, + // which may be inaccurate. + private def updateBytesReadWithFileSize(): Unit = { + if (currentFile != null) { + inputMetrics.incBytesRead(currentFile.length) + } + } + + private[this] val files = split.asInstanceOf[FilePartition].files.toIterator + private[this] var currentFile: PartitionedFile = null + private[this] var currentIterator: Iterator[Object] = null + + def hasNext: Boolean = { + // Kill the task in case it has been marked as killed. This logic is from + // InterruptibleIterator, but we inline it here instead of wrapping the iterator in order + // to avoid performance overhead. + context.killTaskIfInterrupted() + (currentIterator != null && currentIterator.hasNext) || nextIterator() + } + def next(): Object = { + val nextElement = currentIterator.next() + // TODO: we should have a better separation of row based and batch based scan, so that we + // don't need to run this `if` for every record. + if (nextElement.isInstanceOf[ColumnarBatch]) { + inputMetrics.incRecordsRead(nextElement.asInstanceOf[ColumnarBatch].numRows()) + } else { + inputMetrics.incRecordsRead(1) + } + if (inputMetrics.recordsRead % SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS == 0) { + updateBytesRead() + } + nextElement + } + + private def readCurrentFile(): Iterator[InternalRow] = { + try { + readFunction(currentFile) + } catch { + case e: FileNotFoundException => + throw new FileNotFoundException( + e.getMessage + "\n" + + "It is possible the underlying files have been updated. " + + "You can explicitly invalidate the cache in Spark by " + + "running 'REFRESH TABLE tableName' command in SQL or " + + "by recreating the Dataset/DataFrame involved.") + } + } + + /** Advances to the next file. Returns true if a new non-empty iterator is available. */ + private def nextIterator(): Boolean = { + updateBytesReadWithFileSize() + if (files.hasNext) { + currentFile = files.next() + logInfo(s"Reading File $currentFile") + // Sets InputFileBlockHolder for the file block's information + InputFileBlockHolder.set(currentFile.filePath, currentFile.start, currentFile.length) + + if (ignoreMissingFiles || ignoreCorruptFiles) { + currentIterator = new NextIterator[Object] { + // The readFunction may read some bytes before consuming the iterator, e.g., + // vectorized Parquet reader. Here we use lazy val to delay the creation of + // iterator so that we will throw exception in `getNext`. + private lazy val internalIter = readCurrentFile() + + override def getNext(): AnyRef = { + try { + if (internalIter.hasNext) { + internalIter.next() + } else { + finished = true + null + } + } catch { + case e: FileNotFoundException if ignoreMissingFiles => + logWarning(s"Skipped missing file: $currentFile", e) + finished = true + null + // Throw FileNotFoundException even if `ignoreCorruptFiles` is true + case e: FileNotFoundException if !ignoreMissingFiles => throw e + case e @ (_: RuntimeException | _: IOException) if ignoreCorruptFiles => + logWarning( + s"Skipped the rest of the content in the corrupted file: $currentFile", e) + finished = true + null + } + } + + override def close(): Unit = {} + } + } else { + currentIterator = readCurrentFile() + } + + hasNext + } else { + currentFile = null + InputFileBlockHolder.unset() + false + } + } + + override def close(): Unit = { + updateBytesRead() + updateBytesReadWithFileSize() + InputFileBlockHolder.unset() + } + } + + // Register an on-task-completion callback to close the input stream. + context.addTaskCompletionListener(_ => iterator.close()) + + iterator.asInstanceOf[Iterator[InternalRow]] // This is an erasure hack. + } + + def getPreferredLocations(split: FilePartition): Seq[String] = { + val files = split.files + + // Computes total number of bytes can be retrieved from each host. + val hostToNumBytes = mutable.HashMap.empty[String, Long] + files.foreach { file => + file.locations.filter(_ != "localhost").foreach { host => + hostToNumBytes(host) = hostToNumBytes.getOrElse(host, 0L) + file.length + } + } + + // Takes the first 3 hosts with the most data to be retrieved + hostToNumBytes.toSeq.sortBy { + case (host, numBytes) => numBytes + }.reverse.take(3).map { + case (host, numBytes) => host + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala index b2f73b7f8d1f..d278802e6c9f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala @@ -52,28 +52,12 @@ case class HadoopFsRelation( override def sqlContext: SQLContext = sparkSession.sqlContext - private def getColName(f: StructField): String = { - if (sparkSession.sessionState.conf.caseSensitiveAnalysis) { - f.name - } else { - f.name.toLowerCase(Locale.ROOT) - } - } - - val overlappedPartCols = mutable.Map.empty[String, StructField] - partitionSchema.foreach { partitionField => - if (dataSchema.exists(getColName(_) == getColName(partitionField))) { - overlappedPartCols += getColName(partitionField) -> partitionField - } - } - // When data and partition schemas have overlapping columns, the output // schema respects the order of the data schema for the overlapping columns, and it // respects the data types of the partition schema. - val schema: StructType = { - StructType(dataSchema.map(f => overlappedPartCols.getOrElse(getColName(f), f)) ++ - partitionSchema.filterNot(f => overlappedPartCols.contains(getColName(f)))) - } + val (schema: StructType, overlappedPartCols: Map[String, StructField]) = + PartitioningUtils.mergeDataAndPartitionSchema(dataSchema, + partitionSchema, sparkSession.sessionState.conf.caseSensitiveAnalysis) def partitionSchemaOption: Option[StructType] = if (partitionSchema.isEmpty) None else Some(partitionSchema) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index 1edf27619ad7..f4377fc18ad9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -21,6 +21,7 @@ import java.lang.{Double => JDouble, Long => JLong} import java.math.{BigDecimal => JBigDecimal} import java.util.{Locale, TimeZone} +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.util.Try @@ -493,6 +494,83 @@ object PartitioningUtils { }).asNullable } + def mergeDataAndPartitionSchema( + dataSchema: StructType, + partitionSchema: StructType, + caseSensitive: Boolean): (StructType, Map[String, StructField]) = { + val equality = columnNameEquality(caseSensitive) + val overlappedPartCols = mutable.Map.empty[String, StructField] + partitionSchema.foreach { partitionField => + val partitionFieldName = getColName(partitionField, caseSensitive) + if (dataSchema.exists(getColName(_, caseSensitive) == partitionFieldName)) { + overlappedPartCols += partitionFieldName -> partitionField + } + } + + // When data and partition schemas have overlapping columns, the output + // schema respects the order of the data schema for the overlapping columns, and it + // respects the data types of the partition schema. + val fullSchema = + StructType(dataSchema.map(f => overlappedPartCols.getOrElse(getColName(f, caseSensitive), f)) ++ + partitionSchema.filterNot(f => overlappedPartCols.contains(getColName(f, caseSensitive)))) + (fullSchema, overlappedPartCols.toMap) + } + + def requestedPartitionColumnIds( + partitionSchema: StructType, + requiredSchema: StructType, + caseSensitive: Boolean): Array[Int] = { + val columnNameMap = + partitionSchema.fields.map(getColName(_, caseSensitive)).zipWithIndex.toMap + requiredSchema.fields.map { field => + columnNameMap.getOrElse(getColName(field, caseSensitive), -1) + } + } + + /** + * Returns a new StructType that is a copy of the original StructType, removing any items that + * also appear in other StructType. The order is preserved from the original StructType. + */ + def subtractSchema(original: StructType, other: StructType, isCaseSensitive: Boolean) + : StructType = { + val otherNameSet = other.fields.map(getColName(_, isCaseSensitive)).toSet + val fields = original.fields.filterNot { field => + otherNameSet.contains(getColName(field, isCaseSensitive)) + } + + StructType(fields) + } + + /** + * In the read path, only managed tables by Hive provide the partition columns properly when + * initializing this class. All other file based data sources will try to infer the partitioning, + * and then cast the inferred types to user specified dataTypes if the partition columns exist + * inside `userSpecifiedSchema`, otherwise we can hit data corruption bugs like SPARK-18510, or + * inconsistent data types as reported in SPARK-21463. + * @param fileIndex A FileIndex that will perform partition inference + * @return The PartitionSchema resolved from inference and cast according to `userSpecifiedSchema` + */ + def combineInferredAndUserSpecifiedPartitionSchema( + fileIndex: FileIndex, + userSpecifiedSchema: Option[StructType] = None, + caseSensitive: Boolean): StructType = { + val equality = columnNameEquality(caseSensitive) + val resolved = fileIndex.partitionSchema.map { partitionField => + // SPARK-18510: try to get schema from userSpecifiedSchema, otherwise fallback to inferred + userSpecifiedSchema.flatMap(_.find(f => equality(f.name, partitionField.name))).getOrElse( + partitionField) + } + StructType(resolved) + } + + private def getColName(f: StructField, caseSensitive: Boolean): String = { + if (caseSensitive) { + f.name + } else { + f.name.toLowerCase(Locale.ROOT) + } + } + private def columnNameEquality(caseSensitive: Boolean): (String, String) => Boolean = { if (caseSensitive) { org.apache.spark.sql.catalyst.analysis.caseSensitiveResolution diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala index 1de2ca2914c4..8b81aec14e46 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala @@ -168,13 +168,13 @@ class OrcFileFormat val reader = OrcFile.createReader(filePath, readerOptions) val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( - isCaseSensitive, dataSchema, requiredSchema, reader, conf) + isCaseSensitive, dataSchema, resultSchema, reader, conf) if (requestedColIdsOrEmptyFile.isEmpty) { Iterator.empty } else { val requestedColIds = requestedColIdsOrEmptyFile.get - assert(requestedColIds.length == requiredSchema.length, + assert(requestedColIds.length == resultSchema.length, "[BUG] requested column IDs do not match required schema") val taskConf = new Configuration(conf) taskConf.set(OrcConf.INCLUDE_COLUMNS.getAttribute, @@ -193,13 +193,14 @@ class OrcFileFormat // after opening a file. val iter = new RecordReaderIterator(batchReader) Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => iter.close())) - + val requestedPartitionColIds = + Array.fill(requiredSchema.length)(-1) ++ Range(0, partitionSchema.length) batchReader.initialize(fileSplit, taskAttemptContext) batchReader.initBatch( reader.getSchema, + resultSchema.fields, requestedColIds, - requiredSchema.fields, - partitionSchema, + requestedPartitionColIds, file.partitionValues) iter.asInstanceOf[Iterator[InternalRow]] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 4f44ae4fa1d7..7cac51828f8f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -54,7 +54,7 @@ import org.apache.spark.sql.types._ * builder methods mentioned above can only be found in test code, where all tested filters are * known to be convertible. */ -private[orc] object OrcFilters { +object OrcFilters { /** * Create ORC filter as a SearchArgument instance. @@ -64,19 +64,23 @@ private[orc] object OrcFilters { // First, tries to convert each filter individually to see whether it's convertible, and then // collect all convertible ones to build the final `SearchArgument`. - val convertibleFilters = for { - filter <- filters - _ <- buildSearchArgument(dataTypeMap, filter, SearchArgumentFactory.newBuilder()) - } yield filter + val convertible = convertibleFilters(schema, filters) for { // Combines all convertible filters using `And` to produce a single conjunction - conjunction <- convertibleFilters.reduceOption(org.apache.spark.sql.sources.And) + conjunction <- convertible.reduceOption(org.apache.spark.sql.sources.And) // Then tries to build a single ORC `SearchArgument` for the conjunction predicate builder <- buildSearchArgument(dataTypeMap, conjunction, SearchArgumentFactory.newBuilder()) } yield builder.build() } + def convertibleFilters(schema: StructType, filters: Seq[Filter]): Seq[Filter] = { + val dataTypeMap = schema.map(f => f.name -> f.dataType).toMap + for { + filter <- filters + _ <- buildSearchArgument(dataTypeMap, filter, SearchArgumentFactory.newBuilder()) + } yield filter + } /** * Return true if this is a searchable type in ORC. * Both CharType and VarcharType are cleaned at AstBuilder. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala index 460194ba61c8..0dfc6e301502 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.datasources.orc +import java.util.Locale + import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyDataReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyDataReader.scala new file mode 100644 index 000000000000..59e0da89cc7d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyDataReader.scala @@ -0,0 +1,34 @@ +/* + * 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.v2 + +import java.io.IOException + +import org.apache.spark.sql.sources.v2.reader.DataReader + +/** + * A [[DataReader]] with empty output. + */ +class EmptyDataReader[T] extends DataReader[T] { + override def next(): Boolean = false + + override def get(): T = + throw new IOException("No records should be returned from EmptyDataReader") + + override def close(): Unit = {} +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcColumnarBatchDataReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcColumnarBatchDataReader.scala new file mode 100644 index 000000000000..0527ad39537f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcColumnarBatchDataReader.scala @@ -0,0 +1,113 @@ +/* + * 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.v2.orc + +import java.io.IOException +import java.net.URI + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType} +import org.apache.hadoop.mapreduce.lib.input.FileSplit +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl +import org.apache.orc.{OrcConf, OrcFile} + +import org.apache.spark.TaskContext +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.orc.{OrcColumnarBatchReader, OrcUtils} +import org.apache.spark.sql.sources.v2.reader.{DataReader, DataReaderFactory} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.SerializableConfiguration + +case class OrcBatchDataReaderFactory( + file: FilePartition, + dataSchema: StructType, + partitionSchema: StructType, + readSchema: StructType, + enableOffHeapColumnVector: Boolean, + copyToSpark: Boolean, + capacity: Int, + broadcastedConf: Broadcast[SerializableConfiguration], + isCaseSensitive: Boolean) + extends DataReaderFactory[ColumnarBatch] { + private val readFunction = (file: PartitionedFile) => { + val conf = broadcastedConf.value.value + val filePath = new Path(new URI(file.filePath)) + val fileSplit = new FileSplit(filePath, file.start, file.length, Array.empty) + val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) + val fs = filePath.getFileSystem(conf) + val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) + val reader = OrcFile.createReader(filePath, readerOptions) + val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( + isCaseSensitive, dataSchema, readSchema, reader, conf) + if (requestedColIdsOrEmptyFile.isEmpty) { + Iterator.empty + } else { + val requestedColIds = requestedColIdsOrEmptyFile.get + assert(requestedColIds.length == readSchema.length, + "[BUG] requested column IDs do not match required schema") + val taskContext = Option(TaskContext.get()) + val batchReader = new OrcColumnarBatchReader( + enableOffHeapColumnVector && taskContext.isDefined, copyToSpark, capacity) + + // SPARK-23399 Register a task completion listener first to call `close()` in all cases. + // There is a possibility that `initialize` and `initBatch` hit some errors (like OOM) + // after opening a file. + val iter = new RecordReaderIterator(batchReader) + Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => iter.close())) + Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => batchReader.close())) + + val taskConf = new Configuration(conf) + taskConf.set(OrcConf.INCLUDE_COLUMNS.getAttribute, + requestedColIds.filter(_ != -1).sorted.mkString(",")) + val taskAttemptContext = new TaskAttemptContextImpl(taskConf, attemptId) + batchReader.initialize(fileSplit, taskAttemptContext) + + val partitionColIds = PartitioningUtils.requestedPartitionColumnIds( + partitionSchema, readSchema, isCaseSensitive) + batchReader.initBatch( + reader.getSchema, + readSchema.fields, + requestedColIds, + partitionColIds, + file.partitionValues) + iter.asInstanceOf[Iterator[InternalRow]] + } + } + + override def createDataReader(): DataReader[ColumnarBatch] = { + val taskContext = TaskContext.get() + val iter = FilePartitionUtil.compute(file, taskContext, readFunction) + OrcColumnarBatchDataReader(iter) + } + + override def preferredLocations(): Array[String] = { + FilePartitionUtil.getPreferredLocations(file).toArray + } +} + +case class OrcColumnarBatchDataReader(iter: Iterator[InternalRow]) + extends DataReader[ColumnarBatch] { + override def next(): Boolean = iter.hasNext + + override def get(): ColumnarBatch = iter.next().asInstanceOf[ColumnarBatch] + + override def close(): Unit = {} +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala new file mode 100644 index 000000000000..4445e509b091 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala @@ -0,0 +1,184 @@ +/* + * 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.v2.orc + +import java.util.{List => JList, Locale} + +import scala.collection.JavaConverters._ + +import org.apache.orc.mapreduce.OrcInputFormat + +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.expressions.{Expression, UnsafeRow} +import org.apache.spark.sql.execution.PartitionedFileUtil +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.orc.{OrcFilters, OrcUtils} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema} +import org.apache.spark.sql.sources.v2.reader._ +import org.apache.spark.sql.types.{AtomicType, StructType} +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.SerializableConfiguration + +class OrcDataSourceV2 extends DataSourceV2 with ReadSupport with ReadSupportWithSchema { + override def createReader(options: DataSourceOptions): DataSourceReader = { + new OrcDataSourceReader(options, None) + } + + override def createReader(schema: StructType, options: DataSourceOptions): DataSourceReader = { + new OrcDataSourceReader(options, Some(schema)) + } +} + +class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: Option[StructType]) + extends DataSourceReader + with SupportsScanColumnarBatch + with SupportsScanUnsafeRow + with SupportsPushDownCatalystFilters + with SupportsPushDownRequiredColumns { + + private val sparkSession = SparkSession.getActiveSession + .getOrElse(SparkSession.getDefaultSession.get) + private val hadoopConf = + sparkSession.sessionState.newHadoopConfWithOptions(options.asMap().asScala.toMap) + private val isCaseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis + private val fileIndex = { + val filePath = options.get("path") + if (!filePath.isPresent) { + throw new AnalysisException("ORC data source requires a" + + " path (e.g. data backed by a local or distributed file system).") + } + val rootPathsSpecified = + DataSource.checkAndGlobPathIfNecessary(hadoopConf, filePath.get, checkFilesExist = true) + new InMemoryFileIndex(sparkSession, rootPathsSpecified, options.asMap().asScala.toMap, None) + } + + private val partitionSchema = PartitioningUtils.combineInferredAndUserSpecifiedPartitionSchema( + fileIndex, userSpecifiedSchema, isCaseSensitive) + + private val dataSchema = userSpecifiedSchema.getOrElse { + val files = fileIndex.allFiles() + OrcUtils.readSchema(sparkSession, files).getOrElse { + throw new AnalysisException( + s"Unable to infer schema for Orc. It must be specified manually.") + } + } + private val (fullSchema, _) = + PartitioningUtils.mergeDataAndPartitionSchema(dataSchema, partitionSchema, isCaseSensitive) + private val broadcastedConf = + sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) + private var pushedFiltersArray: Array[Expression] = Array.empty + private var partitionKeyFilters: Array[Expression] = Array.empty + private var requiredSchema = fullSchema + + private def partitions: Seq[FilePartition] = { + val selectedPartitions = fileIndex.listFiles(partitionKeyFilters, Seq.empty) + val maxSplitBytes = PartitionedFileUtil.maxSplitBytes(sparkSession, selectedPartitions) + val splitFiles = selectedPartitions.flatMap { partition => + partition.files.flatMap { file => + PartitionedFileUtil.splitFiles( + sparkSession = sparkSession, + file = file, + filePath = file.getPath, + isSplitable = true, + maxSplitBytes = maxSplitBytes, + partitionValues = partition.values + ) + }.toArray.sortBy(_.length)(implicitly[Ordering[Long]].reverse) + } + FilePartitionUtil.getFilePartition(sparkSession, splitFiles, maxSplitBytes) + } + + + override def readSchema(): StructType = { + requiredSchema + } + + override def pruneColumns(requiredSchema: StructType): Unit = { + this.requiredSchema = requiredSchema + } + + override def createBatchDataReaderFactories(): JList[DataReaderFactory[ColumnarBatch]] = { + val sqlConf = sparkSession.sessionState.conf + val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled + val capacity = sqlConf.orcVectorizedReaderBatchSize + val copyToSpark = sparkSession.sessionState.conf.getConf(SQLConf.ORC_COPY_BATCH_TO_SPARK) + + partitions.map { partitionedFile => + new OrcBatchDataReaderFactory(partitionedFile, dataSchema, partitionSchema, + readSchema(), enableOffHeapColumnVector, + copyToSpark, capacity, broadcastedConf, isCaseSensitive) + .asInstanceOf[DataReaderFactory[ColumnarBatch]] + }.asJava + } + + override def createUnsafeRowReaderFactories: JList[DataReaderFactory[UnsafeRow]] = { + partitions.map { partitionedFile => + new OrcUnsafeRowReaderFactory(partitionedFile, dataSchema, partitionSchema, + readSchema(), broadcastedConf, isCaseSensitive) + .asInstanceOf[DataReaderFactory[UnsafeRow]] + }.asJava + } + + override def enableBatchRead(): Boolean = { + val conf = sparkSession.sessionState.conf + val schema = readSchema() + conf.orcVectorizedReaderEnabled && conf.wholeStageEnabled && + schema.length <= conf.wholeStageMaxNumFields && + schema.forall(_.dataType.isInstanceOf[AtomicType]) + } + + override def pushCatalystFilters(filters: Array[Expression]): Array[Expression] = { + val partitionColumnNames = partitionSchema.toAttributes.map(_.name).toSet + val (partitionKeyFilters, otherFilters) = filters.partition { + _.references.map(_.name).toSet.subsetOf(partitionColumnNames) + } + this.partitionKeyFilters = partitionKeyFilters + pushedFiltersArray = partitionKeyFilters + if (sparkSession.sessionState.conf.orcFilterPushDown) { + val dataFilters = otherFilters.map { f => + (DataSourceStrategy.translateFilter(f), f) + }.collect { case (optionalFilter, catalystFilter) if optionalFilter.isDefined => + (optionalFilter.get, catalystFilter) + }.toMap + val pushedDataFilters = + OrcFilters.convertibleFilters(fullSchema, dataFilters.keys.toSeq).map(dataFilters).toArray + pushedFiltersArray ++= pushedDataFilters + OrcFilters.createFilter(fullSchema, dataFilters.keys.toSeq).foreach { f => + OrcInputFormat.setSearchArgument(hadoopConf, f, fullSchema.fieldNames) + } + } + otherFilters + } + + override def pushedCatalystFilters(): Array[Expression] = { + pushedFiltersArray + } +} + +object OrcDataSourceV2 { + def satisfy(sparkSession: SparkSession, source: String, paths: Seq[String]): Option[String] = { + val enabledV2Readers = sparkSession.sqlContext.conf.enabledV2DataSourceReader.split(",") + val isNative = sparkSession.sqlContext.conf.getConf(SQLConf.ORC_IMPLEMENTATION) == "native" + if (source.toLowerCase(Locale.ROOT) == "orc" && isNative && + enabledV2Readers.contains(source) && paths.length == 1) { + Some("org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2") + } else { + None + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcUnsafeRowReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcUnsafeRowReaderFactory.scala new file mode 100644 index 000000000000..ac036be21896 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcUnsafeRowReaderFactory.scala @@ -0,0 +1,114 @@ +/* + * 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.v2.orc + +import java.net.URI + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType} +import org.apache.hadoop.mapreduce.lib.input.FileSplit +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl +import org.apache.orc.{OrcConf, OrcFile} +import org.apache.orc.mapred.OrcStruct +import org.apache.orc.mapreduce.OrcInputFormat + +import org.apache.spark.TaskContext +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.orc.{OrcDeserializer, OrcUtils} +import org.apache.spark.sql.sources.v2.reader.{DataReader, DataReaderFactory} +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.SerializableConfiguration + +case class OrcUnsafeRowReaderFactory( + file: FilePartition, + dataSchema: StructType, + partitionSchema: StructType, + readSchema: StructType, + broadcastedConf: Broadcast[SerializableConfiguration], + isCaseSensitive: Boolean) + extends DataReaderFactory[UnsafeRow] { + private val readFunction = (file: PartitionedFile) => { + val conf = broadcastedConf.value.value + + val filePath = new Path(new URI(file.filePath)) + + val fs = filePath.getFileSystem(conf) + val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) + val reader = OrcFile.createReader(filePath, readerOptions) + + val requiredSchema = + PartitioningUtils.subtractSchema(readSchema, partitionSchema, isCaseSensitive) + val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( + isCaseSensitive, dataSchema, requiredSchema, reader, conf) + if (requestedColIdsOrEmptyFile.isEmpty) { + Iterator.empty + } else { + val requestedColIds = requestedColIdsOrEmptyFile.get + assert(requestedColIds.length == requiredSchema.length, + "[BUG] requested column IDs o not match required schema") + val taskConf = new Configuration(conf) + taskConf.set(OrcConf.INCLUDE_COLUMNS.getAttribute, + requestedColIds.filter(_ != -1).sorted.mkString(",")) + + val fileSplit = new FileSplit(filePath, file.start, file.length, Array.empty) + val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) + val taskAttemptContext = new TaskAttemptContextImpl(taskConf, attemptId) + + val taskContext = Option(TaskContext.get()) + val orcRecordReader = new OrcInputFormat[OrcStruct] + .createRecordReader(fileSplit, taskAttemptContext) + val iter = new RecordReaderIterator[OrcStruct](orcRecordReader) + taskContext.foreach(_.addTaskCompletionListener(_ => iter.close())) + + val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes + val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) + val deserializer = new OrcDeserializer(dataSchema, requiredSchema, requestedColIds) + + if (partitionSchema.length == 0) { + iter.map(value => unsafeProjection(deserializer.deserialize(value))) + } else { + val joinedRow = new JoinedRow() + iter.map(value => + unsafeProjection(joinedRow(deserializer.deserialize(value), file.partitionValues))) + } + } + } + + override def createDataReader(): DataReader[UnsafeRow] = { + val taskContext = TaskContext.get() + val iter = FilePartitionUtil.compute(file, taskContext, readFunction) + OrcUnsafeRowDataReader(iter) + } + + override def preferredLocations(): Array[String] = { + FilePartitionUtil.getPreferredLocations(file).toArray + } +} + +case class OrcUnsafeRowDataReader(iter: Iterator[InternalRow]) + extends DataReader[UnsafeRow] { + override def next(): Boolean = iter.hasNext + + override def get(): UnsafeRow = iter.next().asInstanceOf[UnsafeRow] + + override def close(): Unit = {} +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index 3a0db7e16c23..10f713c586c6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -158,6 +158,7 @@ abstract class BaseSessionStateBuilder( override val extendedResolutionRules: Seq[Rule[LogicalPlan]] = new FindDataSourceTable(session) +: new ResolveSQLOnFile(session) +: + new FallBackToOrcV1(session) +: customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index 8680b86517b1..d4dd09c18eb0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -29,6 +29,10 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 +import org.apache.spark.sql.sources.v2.{DataSourceOptions, ReadSupport, ReadSupportWithSchema} +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownCatalystFilters} import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -50,18 +54,30 @@ class OrcFilterSuite extends OrcTest with SharedSQLContext { .select(output.map(e => Column(e)): _*) .where(Column(predicate)) - var maybeRelation: Option[HadoopFsRelation] = None + var maybeDataReader: Option[DataSourceReader] = None val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => - maybeRelation = Some(orcRelation) + case PhysicalOperation(_, filters, + DataSourceV2Relation(orcRelation: OrcDataSourceV2, options, _, _, userSpecifiedSchema)) => + val dataSourceOptions = new DataSourceOptions(options.asJava) + val dataReader = if (userSpecifiedSchema.isDefined) { + orcRelation.asInstanceOf[ReadSupportWithSchema] + .createReader(userSpecifiedSchema.get, dataSourceOptions) + } else { + orcRelation.asInstanceOf[ReadSupport].createReader(dataSourceOptions) + } + maybeDataReader = Some(dataReader) filters }.flatten.reduceLeftOption(_ && _) assert(maybeAnalyzedPredicate.isDefined, "No filter is analyzed from the given query") - val (_, selectedFilters, _) = - DataSourceStrategy.selectFilters(maybeRelation.get, maybeAnalyzedPredicate.toSeq) - assert(selectedFilters.nonEmpty, "No filter is pushed down") + val pushDownCatalystFiltersReader = + maybeDataReader.get.asInstanceOf[SupportsPushDownCatalystFilters] + pushDownCatalystFiltersReader.pushCatalystFilters(Array(maybeAnalyzedPredicate.get)) + val selectedCatalystFilters = + pushDownCatalystFiltersReader.pushedCatalystFilters() + assert(selectedCatalystFilters.nonEmpty, "No filter is pushed down") + val selectedFilters = selectedCatalystFilters.flatMap(DataSourceStrategy.translateFilter) val maybeFilter = OrcFilters.createFilter(query.schema, selectedFilters) assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $selectedFilters") checker(maybeFilter.get) @@ -94,20 +110,32 @@ class OrcFilterSuite extends OrcTest with SharedSQLContext { .select(output.map(e => Column(e)): _*) .where(Column(predicate)) - var maybeRelation: Option[HadoopFsRelation] = None + var maybeDataReader: Option[DataSourceReader] = None val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => - maybeRelation = Some(orcRelation) + case PhysicalOperation(_, filters, + DataSourceV2Relation(orcRelation: OrcDataSourceV2, options, _, _, userSpecifiedSchema)) => + val dataSourceOptions = new DataSourceOptions(options.asJava) + val dataReader = if (userSpecifiedSchema.isDefined) { + orcRelation.asInstanceOf[ReadSupportWithSchema] + .createReader(userSpecifiedSchema.get, dataSourceOptions) + } else { + orcRelation.asInstanceOf[ReadSupport].createReader(dataSourceOptions) + } + maybeDataReader = Some(dataReader) filters }.flatten.reduceLeftOption(_ && _) assert(maybeAnalyzedPredicate.isDefined, "No filter is analyzed from the given query") - val (_, selectedFilters, _) = - DataSourceStrategy.selectFilters(maybeRelation.get, maybeAnalyzedPredicate.toSeq) - assert(selectedFilters.nonEmpty, "No filter is pushed down") + val pushDownCatalystFiltersReader = + maybeDataReader.get.asInstanceOf[SupportsPushDownCatalystFilters] + pushDownCatalystFiltersReader.pushCatalystFilters(Array(maybeAnalyzedPredicate.get)) + val selectedCatalystFilters = + pushDownCatalystFiltersReader.pushedCatalystFilters() + assert(selectedCatalystFilters.nonEmpty, "No filter is pushed down") + val selectedFilters = selectedCatalystFilters.flatMap(DataSourceStrategy.translateFilter) val maybeFilter = OrcFilters.createFilter(query.schema, selectedFilters) - assert(maybeFilter.isEmpty, s"Could generate filter predicate for $selectedFilters") + assert(maybeFilter.isEmpty, s"Couldn't generate filter predicate for $selectedFilters") } test("filter pushdown - integer") { @@ -340,7 +368,7 @@ class OrcFilterSuite extends OrcTest with SharedSQLContext { } } - test("no filter pushdown - non-supported types") { + ignore("no filter pushdown - non-supported types") { implicit class IntToBinary(int: Int) { def b: Array[Byte] = int.toString.getBytes(StandardCharsets.UTF_8) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala index d1911ea7f32a..49aaa27085d9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala @@ -19,7 +19,9 @@ package org.apache.spark.sql.execution.datasources.orc import java.io.File +import org.apache.spark.SparkConf import org.apache.spark.sql._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext // The data where the partitioning key exists only in the directory structure. @@ -227,3 +229,8 @@ abstract class OrcPartitionDiscoveryTest extends OrcTest { } class OrcPartitionDiscoverySuite extends OrcPartitionDiscoveryTest with SharedSQLContext + +class OrcV1PartitionDiscoverySuite extends OrcPartitionDiscoveryTest with SharedSQLContext { + override protected def sparkConf: SparkConf = + super.sparkConf.set(SQLConf.ENABLED_V2_DATA_SOURCE_READERS, "") +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala index f58c331f33ca..aa48a30c81c7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala @@ -31,7 +31,7 @@ import org.apache.orc.OrcConf.COMPRESS import org.apache.orc.mapred.OrcStruct import org.apache.orc.mapreduce.OrcInputFormat -import org.apache.spark.SparkException +import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, RecordReaderIterator} @@ -656,3 +656,8 @@ class OrcQuerySuite extends OrcQueryTest with SharedSQLContext { } } } + +class OrcV1QuerySuite extends OrcQuerySuite { + override protected def sparkConf: SparkConf = + super.sparkConf.set(SQLConf.ENABLED_V2_DATA_SOURCE_READERS, "") +} From c6aac35ece4f9e82ba9c21e70aa78adf55ef9472 Mon Sep 17 00:00:00 2001 From: Wang Gengliang Date: Thu, 29 Mar 2018 19:53:50 +0800 Subject: [PATCH 02/25] revise code --- .../apache/spark/sql/DataFrameReader.scala | 2 + .../sql/execution/DataSourceScanExec.scala | 2 +- .../datasources/DataSourceStrategy.scala | 6 + .../datasources/FilePartitionUtil.scala | 27 ++- .../execution/datasources/FileScanRDD.scala | 178 +----------------- .../v2/orc/OrcColumnarBatchDataReader.scala | 2 +- .../datasources/v2/orc/OrcDataSourceV2.scala | 2 +- .../v2/orc/OrcUnsafeRowReaderFactory.scala | 2 +- 8 files changed, 40 insertions(+), 181 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index afb262dce284..78867c8ac2d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -192,6 +192,8 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { "read files of Hive data source directly.") } + // SPARK-23817 Since datasource V2 didn't support reading multiple files yet, + // ORC V2 is only used when loading single file path. val allPaths = CaseInsensitiveMap(extraOptions.toMap).get("path") ++ paths val orcV2 = OrcDataSourceV2.satisfy(sparkSession, source, allPaths.toSeq) if (orcV2.isDefined) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 345af08d478c..9a91cab095d1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -418,7 +418,7 @@ case class FileSourceScanExec( }.toArray.sortBy(_.length)(implicitly[Ordering[Long]].reverse) val partitions = - FilePartitionUtil.getFilePartition(relation.sparkSession, splitFiles, maxSplitBytes) + FilePartitionUtil.getFilePartitions(relation.sparkSession, splitFiles, maxSplitBytes) new FileScanRDD(fsRelation.sparkSession, readFile, partitions) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index cf45c9e996e1..fe18b977e072 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -215,6 +215,11 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast } } +/** + * Replaces [[OrcDataSourceV2]] with [[DataSource]] if parent node is [[InsertIntoTable]]. + * This is because [[OrcDataSourceV2]] doesn't support writing data yet. + * @param sparkSession + */ class FallBackToOrcV1(sparkSession: SparkSession) extends Rule[LogicalPlan] { private def convertToOrcV1(v2Relation: DataSourceV2Relation): LogicalPlan = { val v1 = DataSource.apply( @@ -230,6 +235,7 @@ class FallBackToOrcV1(sparkSession: SparkSession) extends Rule[LogicalPlan] { if d.source.isInstanceOf[OrcDataSourceV2] => i.copy(table = convertToOrcV1(d)) } } + /** * Replaces [[UnresolvedCatalogRelation]] with concrete relation logical plans. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartitionUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartitionUtil.scala index c8640776a67c..e9210c195ed1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartitionUtil.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartitionUtil.scala @@ -21,18 +21,21 @@ import java.io.{FileNotFoundException, IOException} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import org.apache.parquet.io.ParquetDecodingException + import org.apache.spark.TaskContext import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.rdd.InputFileBlockHolder import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.QueryExecutionException import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.NextIterator object FilePartitionUtil extends Logging { - def getFilePartition( + def getFilePartitions( sparkSession: SparkSession, partitionedFiles: Seq[PartitionedFile], maxSplitBytes: Long): Seq[FilePartition] = { @@ -183,7 +186,23 @@ object FilePartitionUtil extends Logging { currentIterator = readCurrentFile() } - hasNext + try { + hasNext + } catch { + case e: SchemaColumnConvertNotSupportedException => + val message = "Parquet column cannot be converted in " + + s"file ${currentFile.filePath}. Column: ${e.getColumn}, " + + s"Expected: ${e.getLogicalType}, Found: ${e.getPhysicalType}" + throw new QueryExecutionException(message, e) + case e: ParquetDecodingException => + if (e.getMessage.contains("Can not read value at")) { + val message = "Encounter error while reading parquet files. " + + "One possible cause: Parquet column cannot be converted in the " + + "corresponding files. Details: " + throw new QueryExecutionException(message, e) + } + throw e + } } else { currentFile = null InputFileBlockHolder.unset() @@ -204,7 +223,7 @@ object FilePartitionUtil extends Logging { iterator.asInstanceOf[Iterator[InternalRow]] // This is an erasure hack. } - def getPreferredLocations(split: FilePartition): Seq[String] = { + def getPreferredLocations(split: FilePartition): Array[String] = { val files = split.files // Computes total number of bytes can be retrieved from each host. @@ -220,6 +239,6 @@ object FilePartitionUtil extends Logging { case (host, numBytes) => numBytes }.reverse.take(3).map { case (host, numBytes) => host - } + }.toArray } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index 28c36b6020d3..ddc3ec7c23d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -17,20 +17,10 @@ package org.apache.spark.sql.execution.datasources -import java.io.{FileNotFoundException, IOException} - -import scala.collection.mutable - -import org.apache.parquet.io.ParquetDecodingException - -import org.apache.spark.{Partition => RDDPartition, TaskContext, TaskKilledException} -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.rdd.{InputFileBlockHolder, RDD} +import org.apache.spark.{Partition => RDDPartition, TaskContext} +import org.apache.spark.rdd.RDD import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.QueryExecutionException -import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.util.NextIterator /** * A part (i.e. "block") of a single file that should be read, along with partition column values @@ -72,171 +62,13 @@ class FileScanRDD( private val ignoreMissingFiles = sparkSession.sessionState.conf.ignoreMissingFiles override def compute(split: RDDPartition, context: TaskContext): Iterator[InternalRow] = { - val iterator = new Iterator[Object] with AutoCloseable { - private val inputMetrics = context.taskMetrics().inputMetrics - private val existingBytesRead = inputMetrics.bytesRead - - // Find a function that will return the FileSystem bytes read by this thread. Do this before - // apply readFunction, because it might read some bytes. - private val getBytesReadCallback = - SparkHadoopUtil.get.getFSBytesReadOnThreadCallback() - - // We get our input bytes from thread-local Hadoop FileSystem statistics. - // If we do a coalesce, however, we are likely to compute multiple partitions in the same - // task and in the same thread, in which case we need to avoid override values written by - // previous partitions (SPARK-13071). - private def updateBytesRead(): Unit = { - inputMetrics.setBytesRead(existingBytesRead + getBytesReadCallback()) - } - - // If we can't get the bytes read from the FS stats, fall back to the file size, - // which may be inaccurate. - private def updateBytesReadWithFileSize(): Unit = { - if (currentFile != null) { - inputMetrics.incBytesRead(currentFile.length) - } - } - - private[this] val files = split.asInstanceOf[FilePartition].files.toIterator - private[this] var currentFile: PartitionedFile = null - private[this] var currentIterator: Iterator[Object] = null - - def hasNext: Boolean = { - // Kill the task in case it has been marked as killed. This logic is from - // InterruptibleIterator, but we inline it here instead of wrapping the iterator in order - // to avoid performance overhead. - context.killTaskIfInterrupted() - (currentIterator != null && currentIterator.hasNext) || nextIterator() - } - def next(): Object = { - val nextElement = currentIterator.next() - // TODO: we should have a better separation of row based and batch based scan, so that we - // don't need to run this `if` for every record. - if (nextElement.isInstanceOf[ColumnarBatch]) { - inputMetrics.incRecordsRead(nextElement.asInstanceOf[ColumnarBatch].numRows()) - } else { - inputMetrics.incRecordsRead(1) - } - if (inputMetrics.recordsRead % SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS == 0) { - updateBytesRead() - } - nextElement - } - - private def readCurrentFile(): Iterator[InternalRow] = { - try { - readFunction(currentFile) - } catch { - case e: FileNotFoundException => - throw new FileNotFoundException( - e.getMessage + "\n" + - "It is possible the underlying files have been updated. " + - "You can explicitly invalidate the cache in Spark by " + - "running 'REFRESH TABLE tableName' command in SQL or " + - "by recreating the Dataset/DataFrame involved.") - } - } - - /** Advances to the next file. Returns true if a new non-empty iterator is available. */ - private def nextIterator(): Boolean = { - updateBytesReadWithFileSize() - if (files.hasNext) { - currentFile = files.next() - logInfo(s"Reading File $currentFile") - // Sets InputFileBlockHolder for the file block's information - InputFileBlockHolder.set(currentFile.filePath, currentFile.start, currentFile.length) - - if (ignoreMissingFiles || ignoreCorruptFiles) { - currentIterator = new NextIterator[Object] { - // The readFunction may read some bytes before consuming the iterator, e.g., - // vectorized Parquet reader. Here we use lazy val to delay the creation of - // iterator so that we will throw exception in `getNext`. - private lazy val internalIter = readCurrentFile() - - override def getNext(): AnyRef = { - try { - if (internalIter.hasNext) { - internalIter.next() - } else { - finished = true - null - } - } catch { - case e: FileNotFoundException if ignoreMissingFiles => - logWarning(s"Skipped missing file: $currentFile", e) - finished = true - null - // Throw FileNotFoundException even if `ignoreCorruptFiles` is true - case e: FileNotFoundException if !ignoreMissingFiles => throw e - case e @ (_: RuntimeException | _: IOException) if ignoreCorruptFiles => - logWarning( - s"Skipped the rest of the content in the corrupted file: $currentFile", e) - finished = true - null - } - } - - override def close(): Unit = {} - } - } else { - currentIterator = readCurrentFile() - } - - try { - hasNext - } catch { - case e: SchemaColumnConvertNotSupportedException => - val message = "Parquet column cannot be converted in " + - s"file ${currentFile.filePath}. Column: ${e.getColumn}, " + - s"Expected: ${e.getLogicalType}, Found: ${e.getPhysicalType}" - throw new QueryExecutionException(message, e) - case e: ParquetDecodingException => - if (e.getMessage.contains("Can not read value at")) { - val message = "Encounter error while reading parquet files. " + - "One possible cause: Parquet column cannot be converted in the " + - "corresponding files. Details: " - throw new QueryExecutionException(message, e) - } - throw e - } - } else { - currentFile = null - InputFileBlockHolder.unset() - false - } - } - - override def close(): Unit = { - updateBytesRead() - updateBytesReadWithFileSize() - InputFileBlockHolder.unset() - } - } - - // Register an on-task-completion callback to close the input stream. - context.addTaskCompletionListener(_ => iterator.close()) - - iterator.asInstanceOf[Iterator[InternalRow]] // This is an erasure hack. + FilePartitionUtil.compute(split.asInstanceOf[FilePartition], context, + readFunction, ignoreCorruptFiles, ignoreMissingFiles) } override protected def getPartitions: Array[RDDPartition] = filePartitions.toArray override protected def getPreferredLocations(split: RDDPartition): Seq[String] = { - val files = split.asInstanceOf[FilePartition].files - - // Computes total number of bytes can be retrieved from each host. - val hostToNumBytes = mutable.HashMap.empty[String, Long] - files.foreach { file => - file.locations.filter(_ != "localhost").foreach { host => - hostToNumBytes(host) = hostToNumBytes.getOrElse(host, 0L) + file.length - } - } - - // Takes the first 3 hosts with the most data to be retrieved - hostToNumBytes.toSeq.sortBy { - case (host, numBytes) => numBytes - }.reverse.take(3).map { - case (host, numBytes) => host - } + FilePartitionUtil.getPreferredLocations(split.asInstanceOf[FilePartition]) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcColumnarBatchDataReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcColumnarBatchDataReader.scala index 0527ad39537f..c08c1978c964 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcColumnarBatchDataReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcColumnarBatchDataReader.scala @@ -99,7 +99,7 @@ case class OrcBatchDataReaderFactory( } override def preferredLocations(): Array[String] = { - FilePartitionUtil.getPreferredLocations(file).toArray + FilePartitionUtil.getPreferredLocations(file) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala index 4445e509b091..44c78c03403d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala @@ -100,7 +100,7 @@ class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: Optio ) }.toArray.sortBy(_.length)(implicitly[Ordering[Long]].reverse) } - FilePartitionUtil.getFilePartition(sparkSession, splitFiles, maxSplitBytes) + FilePartitionUtil.getFilePartitions(sparkSession, splitFiles, maxSplitBytes) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcUnsafeRowReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcUnsafeRowReaderFactory.scala index ac036be21896..49d93761c87c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcUnsafeRowReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcUnsafeRowReaderFactory.scala @@ -100,7 +100,7 @@ case class OrcUnsafeRowReaderFactory( } override def preferredLocations(): Array[String] = { - FilePartitionUtil.getPreferredLocations(file).toArray + FilePartitionUtil.getPreferredLocations(file) } } From 72937ec1e8f1cb80639ac899661cb5b12d970832 Mon Sep 17 00:00:00 2001 From: Wang Gengliang Date: Thu, 29 Mar 2018 21:27:57 +0800 Subject: [PATCH 03/25] refactor constructor --- .../v2/orc/OrcColumnarBatchDataReader.scala | 12 +++--- .../datasources/v2/orc/OrcDataSourceV2.scala | 39 ++++++++++++------- .../v2/orc/OrcUnsafeRowReaderFactory.scala | 9 +++-- 3 files changed, 36 insertions(+), 24 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcColumnarBatchDataReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcColumnarBatchDataReader.scala index c08c1978c964..2a5bf2698eab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcColumnarBatchDataReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcColumnarBatchDataReader.scala @@ -41,11 +41,8 @@ case class OrcBatchDataReaderFactory( dataSchema: StructType, partitionSchema: StructType, readSchema: StructType, - enableOffHeapColumnVector: Boolean, - copyToSpark: Boolean, - capacity: Int, broadcastedConf: Broadcast[SerializableConfiguration], - isCaseSensitive: Boolean) + readerConf: OrcDataReaderFactoryConf) extends DataReaderFactory[ColumnarBatch] { private val readFunction = (file: PartitionedFile) => { val conf = broadcastedConf.value.value @@ -56,7 +53,7 @@ case class OrcBatchDataReaderFactory( val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) val reader = OrcFile.createReader(filePath, readerOptions) val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( - isCaseSensitive, dataSchema, readSchema, reader, conf) + readerConf.isCaseSensitive, dataSchema, readSchema, reader, conf) if (requestedColIdsOrEmptyFile.isEmpty) { Iterator.empty } else { @@ -65,7 +62,8 @@ case class OrcBatchDataReaderFactory( "[BUG] requested column IDs do not match required schema") val taskContext = Option(TaskContext.get()) val batchReader = new OrcColumnarBatchReader( - enableOffHeapColumnVector && taskContext.isDefined, copyToSpark, capacity) + readerConf.enableOffHeapColumnVector && taskContext.isDefined, + readerConf.copyToSpark, readerConf.capacity) // SPARK-23399 Register a task completion listener first to call `close()` in all cases. // There is a possibility that `initialize` and `initBatch` hit some errors (like OOM) @@ -81,7 +79,7 @@ case class OrcBatchDataReaderFactory( batchReader.initialize(fileSplit, taskAttemptContext) val partitionColIds = PartitioningUtils.requestedPartitionColumnIds( - partitionSchema, readSchema, isCaseSensitive) + partitionSchema, readSchema, readerConf.isCaseSensitive) batchReader.initBatch( reader.getSchema, readSchema.fields, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala index 44c78c03403d..ed024631bc5b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala @@ -55,7 +55,18 @@ class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: Optio .getOrElse(SparkSession.getDefaultSession.get) private val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(options.asMap().asScala.toMap) - private val isCaseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis + private val sqlConf = sparkSession.sessionState.conf + private val readerConf = OrcDataReaderFactoryConf( + capacity = sqlConf.orcVectorizedReaderBatchSize, + enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled, + copyToSpark = sqlConf.getConf(SQLConf.ORC_COPY_BATCH_TO_SPARK), + isCaseSensitive = sqlConf.caseSensitiveAnalysis, + ignoreCorruptFiles = sqlConf.ignoreCorruptFiles, + ignoreMissingFiles = sqlConf.ignoreMissingFiles + ) + + private val isCaseSensitive = sqlConf.caseSensitiveAnalysis + private val fileIndex = { val filePath = options.get("path") if (!filePath.isPresent) { @@ -113,15 +124,9 @@ class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: Optio } override def createBatchDataReaderFactories(): JList[DataReaderFactory[ColumnarBatch]] = { - val sqlConf = sparkSession.sessionState.conf - val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled - val capacity = sqlConf.orcVectorizedReaderBatchSize - val copyToSpark = sparkSession.sessionState.conf.getConf(SQLConf.ORC_COPY_BATCH_TO_SPARK) - partitions.map { partitionedFile => new OrcBatchDataReaderFactory(partitionedFile, dataSchema, partitionSchema, - readSchema(), enableOffHeapColumnVector, - copyToSpark, capacity, broadcastedConf, isCaseSensitive) + readSchema(), broadcastedConf, readerConf) .asInstanceOf[DataReaderFactory[ColumnarBatch]] }.asJava } @@ -129,16 +134,15 @@ class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: Optio override def createUnsafeRowReaderFactories: JList[DataReaderFactory[UnsafeRow]] = { partitions.map { partitionedFile => new OrcUnsafeRowReaderFactory(partitionedFile, dataSchema, partitionSchema, - readSchema(), broadcastedConf, isCaseSensitive) + readSchema(), broadcastedConf, readerConf) .asInstanceOf[DataReaderFactory[UnsafeRow]] }.asJava } override def enableBatchRead(): Boolean = { - val conf = sparkSession.sessionState.conf val schema = readSchema() - conf.orcVectorizedReaderEnabled && conf.wholeStageEnabled && - schema.length <= conf.wholeStageMaxNumFields && + sqlConf.orcVectorizedReaderEnabled && sqlConf.wholeStageEnabled && + schema.length <= sqlConf.wholeStageMaxNumFields && schema.forall(_.dataType.isInstanceOf[AtomicType]) } @@ -149,7 +153,7 @@ class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: Optio } this.partitionKeyFilters = partitionKeyFilters pushedFiltersArray = partitionKeyFilters - if (sparkSession.sessionState.conf.orcFilterPushDown) { + if (sqlConf.orcFilterPushDown) { val dataFilters = otherFilters.map { f => (DataSourceStrategy.translateFilter(f), f) }.collect { case (optionalFilter, catalystFilter) if optionalFilter.isDefined => @@ -170,6 +174,15 @@ class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: Optio } } +case class OrcDataReaderFactoryConf( + capacity: Int, + enableOffHeapColumnVector: Boolean, + copyToSpark: Boolean, + isCaseSensitive: Boolean, + ignoreCorruptFiles: Boolean, + ignoreMissingFiles: Boolean +) + object OrcDataSourceV2 { def satisfy(sparkSession: SparkSession, source: String, paths: Seq[String]): Option[String] = { val enabledV2Readers = sparkSession.sqlContext.conf.enabledV2DataSourceReader.split(",") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcUnsafeRowReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcUnsafeRowReaderFactory.scala index 49d93761c87c..87ba96f2c50d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcUnsafeRowReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcUnsafeRowReaderFactory.scala @@ -44,7 +44,7 @@ case class OrcUnsafeRowReaderFactory( partitionSchema: StructType, readSchema: StructType, broadcastedConf: Broadcast[SerializableConfiguration], - isCaseSensitive: Boolean) + readerConf: OrcDataReaderFactoryConf) extends DataReaderFactory[UnsafeRow] { private val readFunction = (file: PartitionedFile) => { val conf = broadcastedConf.value.value @@ -56,9 +56,9 @@ case class OrcUnsafeRowReaderFactory( val reader = OrcFile.createReader(filePath, readerOptions) val requiredSchema = - PartitioningUtils.subtractSchema(readSchema, partitionSchema, isCaseSensitive) + PartitioningUtils.subtractSchema(readSchema, partitionSchema, readerConf.isCaseSensitive) val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( - isCaseSensitive, dataSchema, requiredSchema, reader, conf) + readerConf.isCaseSensitive, dataSchema, requiredSchema, reader, conf) if (requestedColIdsOrEmptyFile.isEmpty) { Iterator.empty } else { @@ -95,7 +95,8 @@ case class OrcUnsafeRowReaderFactory( override def createDataReader(): DataReader[UnsafeRow] = { val taskContext = TaskContext.get() - val iter = FilePartitionUtil.compute(file, taskContext, readFunction) + val iter = FilePartitionUtil.compute(file, taskContext, readFunction, + readerConf.ignoreCorruptFiles, readerConf.ignoreMissingFiles) OrcUnsafeRowDataReader(iter) } From d6128aeed6591826b27e4b16b800fdb8039cf27f Mon Sep 17 00:00:00 2001 From: Wang Gengliang Date: Thu, 29 Mar 2018 23:01:21 +0800 Subject: [PATCH 04/25] fix SQLQuerySuite of hive --- .../spark/sql/execution/datasources/orc/OrcFileFormat.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala index 8b81aec14e46..835dd0315d90 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala @@ -168,12 +168,13 @@ class OrcFileFormat val reader = OrcFile.createReader(filePath, readerOptions) val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( - isCaseSensitive, dataSchema, resultSchema, reader, conf) + isCaseSensitive, dataSchema, requiredSchema, reader, conf) if (requestedColIdsOrEmptyFile.isEmpty) { Iterator.empty } else { - val requestedColIds = requestedColIdsOrEmptyFile.get + val requestedColIds = + requestedColIdsOrEmptyFile.get ++ Array.fill(partitionSchema.length)(-1) assert(requestedColIds.length == resultSchema.length, "[BUG] requested column IDs do not match required schema") val taskConf = new Configuration(conf) From 5d00dd718908ebb86c9c6f6a20d49800b51083c8 Mon Sep 17 00:00:00 2001 From: Wang Gengliang Date: Sat, 31 Mar 2018 08:53:59 +0800 Subject: [PATCH 05/25] address comments --- .../org/apache/spark/sql/internal/SQLConf.scala | 8 ++++---- .../datasources/orc/OrcColumnarBatchReader.java | 14 ++++++++++---- .../datasources/v2/orc/OrcDataSourceV2.scala | 4 ++-- .../orc/OrcPartitionDiscoverySuite.scala | 2 +- .../execution/datasources/orc/OrcQuerySuite.scala | 2 +- 5 files changed, 18 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 105282dd61cd..1d7badb46800 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1229,12 +1229,12 @@ object SQLConf { .booleanConf .createWithDefault(true) - val ENABLED_V2_DATA_SOURCE_READERS = buildConf("spark.sql.enabledV2DataSourceReaders") + val DISABLED_V2_DATA_SOURCE_READERS = buildConf("spark.sql.disabledV2DataSourceReaders") .internal() .doc("A comma-separated list of data source short names for which " + - "DataSourceReader is enabled. Reads from these sources will use V2 Sources") + "DataSourceReader is disabled. Reads from these sources will fall back to the V1 sources") .stringConf - .createWithDefault("orc") + .createWithDefault("") object PartitionOverwriteMode extends Enumeration { val STATIC, DYNAMIC = Value @@ -1618,7 +1618,7 @@ class SQLConf extends Serializable with Logging { def disabledV2StreamingMicroBatchReaders: String = getConf(DISABLED_V2_STREAMING_MICROBATCH_READERS) - def enabledV2DataSourceReader: String = getConf(ENABLED_V2_DATA_SOURCE_READERS) + def disabledV2DataSourceReader: String = getConf(DISABLED_V2_DATA_SOURCE_READERS) def concatBinaryAsString: Boolean = getConf(CONCAT_BINARY_AS_STRING) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java index fb090dde866f..aa5e31460926 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java @@ -66,7 +66,7 @@ public class OrcColumnarBatchReader extends RecordReader { * The column IDs of the ORC file partition schema which are required by this reader. * -1 means this required column doesn't exist in the ORC partition columns. */ - private int[] requestedPartitionColds; + private int[] requestedPartitionColIds; // Record reader from ORC row batch. private org.apache.orc.RecordReader recordReader; @@ -149,6 +149,12 @@ public void initialize( /** * Initialize columnar batch by setting required schema and partition information. * With this information, this creates ColumnarBatch with the full schema. + * + * @param orcSchema Schema from ORC file reader. + * @param requiredFields All the fields that are required to return, including partition fields. + * @param requestedColIds Requested column ids from orcSchema. -1 if not existed. + * @param requestedPartitionColIds Requested column ids from partition schema. -1 if not existed. + * @param partitionValues Values of partition columns. */ public void initBatch( TypeDescription orcSchema, @@ -160,7 +166,7 @@ public void initBatch( assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`. this.requiredFields = requiredFields; this.requestedColIds = requestedColIds; - this.requestedPartitionColds = requestedPartitionColIds; + this.requestedPartitionColIds = requestedPartitionColIds; assert(requiredFields.length == requestedColIds.length); assert(requiredFields.length == requestedPartitionColIds.length); @@ -172,7 +178,7 @@ public void initBatch( columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema); } - // Initialize the missing columns once. + // Initialize the missing columns and partition columns once. for (int i = 0; i < requiredFields.length; i++) { if (requestedPartitionColIds[i] != -1) { ColumnVectorUtils.populate(columnVectors[i], @@ -228,7 +234,7 @@ private boolean nextBatch() throws IOException { if (!copyToSpark) { for (int i = 0; i < requiredFields.length; i++) { - if (requestedColIds[i] != -1 && requestedPartitionColds[i] == -1) { + if (requestedColIds[i] != -1 && requestedPartitionColIds[i] == -1) { ((OrcColumnVector) orcVectorWrappers[i]).setBatchSize(batchSize); } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala index ed024631bc5b..085ef693b943 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala @@ -185,10 +185,10 @@ case class OrcDataReaderFactoryConf( object OrcDataSourceV2 { def satisfy(sparkSession: SparkSession, source: String, paths: Seq[String]): Option[String] = { - val enabledV2Readers = sparkSession.sqlContext.conf.enabledV2DataSourceReader.split(",") + val disabledV2Readers = sparkSession.sqlContext.conf.disabledV2DataSourceReader.split(",") val isNative = sparkSession.sqlContext.conf.getConf(SQLConf.ORC_IMPLEMENTATION) == "native" if (source.toLowerCase(Locale.ROOT) == "orc" && isNative && - enabledV2Readers.contains(source) && paths.length == 1) { + !disabledV2Readers.contains(source) && paths.length == 1) { Some("org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2") } else { None diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala index 49aaa27085d9..bf2b89de20ef 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala @@ -232,5 +232,5 @@ class OrcPartitionDiscoverySuite extends OrcPartitionDiscoveryTest with SharedSQ class OrcV1PartitionDiscoverySuite extends OrcPartitionDiscoveryTest with SharedSQLContext { override protected def sparkConf: SparkConf = - super.sparkConf.set(SQLConf.ENABLED_V2_DATA_SOURCE_READERS, "") + super.sparkConf.set(SQLConf.DISABLED_V2_DATA_SOURCE_READERS, "orc") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala index aa48a30c81c7..3efb977b9c8f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala @@ -659,5 +659,5 @@ class OrcQuerySuite extends OrcQueryTest with SharedSQLContext { class OrcV1QuerySuite extends OrcQuerySuite { override protected def sparkConf: SparkConf = - super.sparkConf.set(SQLConf.ENABLED_V2_DATA_SOURCE_READERS, "") + super.sparkConf.set(SQLConf.DISABLED_V2_DATA_SOURCE_READERS, "orc") } From 2561a724cc8e00cbdcda9318e6cd5e447f9a8f7f Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Tue, 3 Apr 2018 17:02:26 +0800 Subject: [PATCH 06/25] refactor --- .../datasources/v2/EmptyDataReader.scala | 34 ----- .../datasources/v2/FileReaderFactory.scala | 50 +++++++ .../v2/orc/OrcColumnarBatchDataReader.scala | 111 --------------- .../datasources/v2/orc/OrcDataSourceV2.scala | 133 ++++++++++++++---- .../v2/orc/OrcUnsafeRowReaderFactory.scala | 115 --------------- .../orc/OrcPartitionDiscoverySuite.scala | 20 --- 6 files changed, 153 insertions(+), 310 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyDataReader.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileReaderFactory.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcColumnarBatchDataReader.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcUnsafeRowReaderFactory.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyDataReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyDataReader.scala deleted file mode 100644 index 59e0da89cc7d..000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyDataReader.scala +++ /dev/null @@ -1,34 +0,0 @@ -/* - * 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.v2 - -import java.io.IOException - -import org.apache.spark.sql.sources.v2.reader.DataReader - -/** - * A [[DataReader]] with empty output. - */ -class EmptyDataReader[T] extends DataReader[T] { - override def next(): Boolean = false - - override def get(): T = - throw new IOException("No records should be returned from EmptyDataReader") - - override def close(): Unit = {} -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileReaderFactory.scala new file mode 100644 index 000000000000..62430eefd920 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileReaderFactory.scala @@ -0,0 +1,50 @@ +/* + * 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.v2 + +import org.apache.spark.TaskContext +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.{FilePartition, FilePartitionUtil, PartitionedFile} +import org.apache.spark.sql.sources.v2.reader.{DataReader, DataReaderFactory} + +case class FileReaderFactory[T]( + file: FilePartition, + readFunction: (PartitionedFile) => Iterator[InternalRow], + ignoreCorruptFiles: Boolean = false, + ignoreMissingFiles: Boolean = false) + extends DataReaderFactory[T] { + override def createDataReader(): DataReader[T] = { + val taskContext = TaskContext.get() + val iter = FilePartitionUtil.compute(file, taskContext, readFunction, + ignoreCorruptFiles, ignoreMissingFiles) + InternalRowDataReader[T](iter) + } + + override def preferredLocations(): Array[String] = { + FilePartitionUtil.getPreferredLocations(file) + } +} + +case class InternalRowDataReader[T](iter: Iterator[InternalRow]) + extends DataReader[T] { + override def next(): Boolean = iter.hasNext + + override def get(): T = iter.next().asInstanceOf[T] + + override def close(): Unit = {} +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcColumnarBatchDataReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcColumnarBatchDataReader.scala deleted file mode 100644 index 2a5bf2698eab..000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcColumnarBatchDataReader.scala +++ /dev/null @@ -1,111 +0,0 @@ -/* - * 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.v2.orc - -import java.io.IOException -import java.net.URI - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType} -import org.apache.hadoop.mapreduce.lib.input.FileSplit -import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl -import org.apache.orc.{OrcConf, OrcFile} - -import org.apache.spark.TaskContext -import org.apache.spark.broadcast.Broadcast -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.datasources.orc.{OrcColumnarBatchReader, OrcUtils} -import org.apache.spark.sql.sources.v2.reader.{DataReader, DataReaderFactory} -import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.util.SerializableConfiguration - -case class OrcBatchDataReaderFactory( - file: FilePartition, - dataSchema: StructType, - partitionSchema: StructType, - readSchema: StructType, - broadcastedConf: Broadcast[SerializableConfiguration], - readerConf: OrcDataReaderFactoryConf) - extends DataReaderFactory[ColumnarBatch] { - private val readFunction = (file: PartitionedFile) => { - val conf = broadcastedConf.value.value - val filePath = new Path(new URI(file.filePath)) - val fileSplit = new FileSplit(filePath, file.start, file.length, Array.empty) - val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) - val fs = filePath.getFileSystem(conf) - val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) - val reader = OrcFile.createReader(filePath, readerOptions) - val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( - readerConf.isCaseSensitive, dataSchema, readSchema, reader, conf) - if (requestedColIdsOrEmptyFile.isEmpty) { - Iterator.empty - } else { - val requestedColIds = requestedColIdsOrEmptyFile.get - assert(requestedColIds.length == readSchema.length, - "[BUG] requested column IDs do not match required schema") - val taskContext = Option(TaskContext.get()) - val batchReader = new OrcColumnarBatchReader( - readerConf.enableOffHeapColumnVector && taskContext.isDefined, - readerConf.copyToSpark, readerConf.capacity) - - // SPARK-23399 Register a task completion listener first to call `close()` in all cases. - // There is a possibility that `initialize` and `initBatch` hit some errors (like OOM) - // after opening a file. - val iter = new RecordReaderIterator(batchReader) - Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => iter.close())) - Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => batchReader.close())) - - val taskConf = new Configuration(conf) - taskConf.set(OrcConf.INCLUDE_COLUMNS.getAttribute, - requestedColIds.filter(_ != -1).sorted.mkString(",")) - val taskAttemptContext = new TaskAttemptContextImpl(taskConf, attemptId) - batchReader.initialize(fileSplit, taskAttemptContext) - - val partitionColIds = PartitioningUtils.requestedPartitionColumnIds( - partitionSchema, readSchema, readerConf.isCaseSensitive) - batchReader.initBatch( - reader.getSchema, - readSchema.fields, - requestedColIds, - partitionColIds, - file.partitionValues) - iter.asInstanceOf[Iterator[InternalRow]] - } - } - - override def createDataReader(): DataReader[ColumnarBatch] = { - val taskContext = TaskContext.get() - val iter = FilePartitionUtil.compute(file, taskContext, readFunction) - OrcColumnarBatchDataReader(iter) - } - - override def preferredLocations(): Array[String] = { - FilePartitionUtil.getPreferredLocations(file) - } -} - -case class OrcColumnarBatchDataReader(iter: Iterator[InternalRow]) - extends DataReader[ColumnarBatch] { - override def next(): Boolean = iter.hasNext - - override def get(): ColumnarBatch = iter.next().asInstanceOf[ColumnarBatch] - - override def close(): Unit = {} -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala index 085ef693b943..7e715f439ebf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala @@ -16,17 +16,29 @@ */ package org.apache.spark.sql.execution.datasources.v2.orc +import java.net.URI import java.util.{List => JList, Locale} import scala.collection.JavaConverters._ +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType} +import org.apache.hadoop.mapreduce.lib.input.FileSplit +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl +import org.apache.orc.{OrcConf, OrcFile} +import org.apache.orc.mapred.OrcStruct import org.apache.orc.mapreduce.OrcInputFormat +import org.apache.spark.TaskContext import org.apache.spark.sql.{AnalysisException, SparkSession} -import org.apache.spark.sql.catalyst.expressions.{Expression, UnsafeRow} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Expression, JoinedRow, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.execution.PartitionedFileUtil import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.datasources.orc.{OrcFilters, OrcUtils} +import org.apache.spark.sql.execution.datasources.orc.{OrcColumnarBatchReader, OrcDeserializer, OrcFilters, OrcUtils} +import org.apache.spark.sql.execution.datasources.v2.FileReaderFactory import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema} import org.apache.spark.sql.sources.v2.reader._ @@ -56,17 +68,9 @@ class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: Optio private val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(options.asMap().asScala.toMap) private val sqlConf = sparkSession.sessionState.conf - private val readerConf = OrcDataReaderFactoryConf( - capacity = sqlConf.orcVectorizedReaderBatchSize, - enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled, - copyToSpark = sqlConf.getConf(SQLConf.ORC_COPY_BATCH_TO_SPARK), - isCaseSensitive = sqlConf.caseSensitiveAnalysis, - ignoreCorruptFiles = sqlConf.ignoreCorruptFiles, - ignoreMissingFiles = sqlConf.ignoreMissingFiles - ) - private val isCaseSensitive = sqlConf.caseSensitiveAnalysis - + private val ignoreCorruptFiles = sqlConf.ignoreCorruptFiles + private val ignoreMissingFiles = sqlConf.ignoreMissingFiles private val fileIndex = { val filePath = options.get("path") if (!filePath.isPresent) { @@ -90,8 +94,6 @@ class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: Optio } private val (fullSchema, _) = PartitioningUtils.mergeDataAndPartitionSchema(dataSchema, partitionSchema, isCaseSensitive) - private val broadcastedConf = - sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) private var pushedFiltersArray: Array[Expression] = Array.empty private var partitionKeyFilters: Array[Expression] = Array.empty private var requiredSchema = fullSchema @@ -114,7 +116,6 @@ class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: Optio FilePartitionUtil.getFilePartitions(sparkSession, splitFiles, maxSplitBytes) } - override def readSchema(): StructType = { requiredSchema } @@ -123,18 +124,99 @@ class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: Optio this.requiredSchema = requiredSchema } + private def readFunction: PartitionedFile => Iterator[InternalRow] = { + val capacity = sqlConf.orcVectorizedReaderBatchSize + val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled + val copyToSpark = sqlConf.getConf(SQLConf.ORC_COPY_BATCH_TO_SPARK) + val isCaseSensitive = this.isCaseSensitive + val dataSchema = this.dataSchema + val readSchema = this.readSchema() + val partitionSchema = this.partitionSchema + val broadcastedConf = + sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) + val enableVectorizedReader = enableBatchRead() + (file: PartitionedFile) => { + val conf = broadcastedConf.value.value + + val filePath = new Path(new URI(file.filePath)) + + val fs = filePath.getFileSystem(conf) + val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) + val reader = OrcFile.createReader(filePath, readerOptions) + + val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( + isCaseSensitive, dataSchema, readSchema, reader, conf) + + if (requestedColIdsOrEmptyFile.isEmpty) { + Iterator.empty + } else { + val requestedColIds = requestedColIdsOrEmptyFile.get + assert(requestedColIds.length == readSchema.length, + "[BUG] requested column IDs do not match required schema") + val taskConf = new Configuration(conf) + taskConf.set(OrcConf.INCLUDE_COLUMNS.getAttribute, + requestedColIds.filter(_ != -1).sorted.mkString(",")) + + val fileSplit = new FileSplit(filePath, file.start, file.length, Array.empty) + val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) + val taskAttemptContext = new TaskAttemptContextImpl(taskConf, attemptId) + + val taskContext = Option(TaskContext.get()) + if (enableVectorizedReader) { + val batchReader = new OrcColumnarBatchReader( + enableOffHeapColumnVector && taskContext.isDefined, copyToSpark, capacity) + // SPARK-23399 Register a task completion listener first to call `close()` in all cases. + // There is a possibility that `initialize` and `initBatch` hit some errors (like OOM) + // after opening a file. + val iter = new RecordReaderIterator(batchReader) + Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => iter.close())) + batchReader.initialize(fileSplit, taskAttemptContext) + val partitionColIds = PartitioningUtils.requestedPartitionColumnIds( + partitionSchema, readSchema, isCaseSensitive) + + batchReader.initBatch( + reader.getSchema, + readSchema.fields, + requestedColIds, + partitionColIds, + file.partitionValues) + iter.asInstanceOf[Iterator[InternalRow]] + } else { + val requiredDataSchema = + PartitioningUtils.subtractSchema(readSchema, partitionSchema, isCaseSensitive) + val orcRecordReader = new OrcInputFormat[OrcStruct] + .createRecordReader(fileSplit, taskAttemptContext) + val iter = new RecordReaderIterator[OrcStruct](orcRecordReader) + taskContext.foreach(_.addTaskCompletionListener(_ => iter.close())) + + val fullSchema = requiredDataSchema.toAttributes ++ partitionSchema.toAttributes + val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) + val deserializer = new OrcDeserializer(dataSchema, requiredDataSchema, requestedColIds) + + if (partitionSchema.length == 0) { + iter.map(value => unsafeProjection(deserializer.deserialize(value))) + } else { + val joinedRow = new JoinedRow() + iter.map(value => + unsafeProjection(joinedRow(deserializer.deserialize(value), file.partitionValues))) + } + } + } + } + } + override def createBatchDataReaderFactories(): JList[DataReaderFactory[ColumnarBatch]] = { - partitions.map { partitionedFile => - new OrcBatchDataReaderFactory(partitionedFile, dataSchema, partitionSchema, - readSchema(), broadcastedConf, readerConf) + partitions.map { filePartition => + new FileReaderFactory[ColumnarBatch](filePartition, readFunction, ignoreCorruptFiles, + ignoreMissingFiles) .asInstanceOf[DataReaderFactory[ColumnarBatch]] }.asJava } override def createUnsafeRowReaderFactories: JList[DataReaderFactory[UnsafeRow]] = { - partitions.map { partitionedFile => - new OrcUnsafeRowReaderFactory(partitionedFile, dataSchema, partitionSchema, - readSchema(), broadcastedConf, readerConf) + partitions.map { filePartition => + new FileReaderFactory[UnsafeRow](filePartition, readFunction, ignoreCorruptFiles, + ignoreMissingFiles) .asInstanceOf[DataReaderFactory[UnsafeRow]] }.asJava } @@ -174,15 +256,6 @@ class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: Optio } } -case class OrcDataReaderFactoryConf( - capacity: Int, - enableOffHeapColumnVector: Boolean, - copyToSpark: Boolean, - isCaseSensitive: Boolean, - ignoreCorruptFiles: Boolean, - ignoreMissingFiles: Boolean -) - object OrcDataSourceV2 { def satisfy(sparkSession: SparkSession, source: String, paths: Seq[String]): Option[String] = { val disabledV2Readers = sparkSession.sqlContext.conf.disabledV2DataSourceReader.split(",") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcUnsafeRowReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcUnsafeRowReaderFactory.scala deleted file mode 100644 index 87ba96f2c50d..000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcUnsafeRowReaderFactory.scala +++ /dev/null @@ -1,115 +0,0 @@ -/* - * 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.v2.orc - -import java.net.URI - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType} -import org.apache.hadoop.mapreduce.lib.input.FileSplit -import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl -import org.apache.orc.{OrcConf, OrcFile} -import org.apache.orc.mapred.OrcStruct -import org.apache.orc.mapreduce.OrcInputFormat - -import org.apache.spark.TaskContext -import org.apache.spark.broadcast.Broadcast -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeRow} -import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection -import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.datasources.orc.{OrcDeserializer, OrcUtils} -import org.apache.spark.sql.sources.v2.reader.{DataReader, DataReaderFactory} -import org.apache.spark.sql.types.StructType -import org.apache.spark.util.SerializableConfiguration - -case class OrcUnsafeRowReaderFactory( - file: FilePartition, - dataSchema: StructType, - partitionSchema: StructType, - readSchema: StructType, - broadcastedConf: Broadcast[SerializableConfiguration], - readerConf: OrcDataReaderFactoryConf) - extends DataReaderFactory[UnsafeRow] { - private val readFunction = (file: PartitionedFile) => { - val conf = broadcastedConf.value.value - - val filePath = new Path(new URI(file.filePath)) - - val fs = filePath.getFileSystem(conf) - val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) - val reader = OrcFile.createReader(filePath, readerOptions) - - val requiredSchema = - PartitioningUtils.subtractSchema(readSchema, partitionSchema, readerConf.isCaseSensitive) - val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( - readerConf.isCaseSensitive, dataSchema, requiredSchema, reader, conf) - if (requestedColIdsOrEmptyFile.isEmpty) { - Iterator.empty - } else { - val requestedColIds = requestedColIdsOrEmptyFile.get - assert(requestedColIds.length == requiredSchema.length, - "[BUG] requested column IDs o not match required schema") - val taskConf = new Configuration(conf) - taskConf.set(OrcConf.INCLUDE_COLUMNS.getAttribute, - requestedColIds.filter(_ != -1).sorted.mkString(",")) - - val fileSplit = new FileSplit(filePath, file.start, file.length, Array.empty) - val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) - val taskAttemptContext = new TaskAttemptContextImpl(taskConf, attemptId) - - val taskContext = Option(TaskContext.get()) - val orcRecordReader = new OrcInputFormat[OrcStruct] - .createRecordReader(fileSplit, taskAttemptContext) - val iter = new RecordReaderIterator[OrcStruct](orcRecordReader) - taskContext.foreach(_.addTaskCompletionListener(_ => iter.close())) - - val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes - val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) - val deserializer = new OrcDeserializer(dataSchema, requiredSchema, requestedColIds) - - if (partitionSchema.length == 0) { - iter.map(value => unsafeProjection(deserializer.deserialize(value))) - } else { - val joinedRow = new JoinedRow() - iter.map(value => - unsafeProjection(joinedRow(deserializer.deserialize(value), file.partitionValues))) - } - } - } - - override def createDataReader(): DataReader[UnsafeRow] = { - val taskContext = TaskContext.get() - val iter = FilePartitionUtil.compute(file, taskContext, readFunction, - readerConf.ignoreCorruptFiles, readerConf.ignoreMissingFiles) - OrcUnsafeRowDataReader(iter) - } - - override def preferredLocations(): Array[String] = { - FilePartitionUtil.getPreferredLocations(file) - } -} - -case class OrcUnsafeRowDataReader(iter: Iterator[InternalRow]) - extends DataReader[UnsafeRow] { - override def next(): Boolean = iter.hasNext - - override def get(): UnsafeRow = iter.next().asInstanceOf[UnsafeRow] - - override def close(): Unit = {} -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala index bf2b89de20ef..bd0d1ee7503a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala @@ -68,13 +68,6 @@ abstract class OrcPartitionDiscoveryTest extends OrcTest { spark.read.orc(base.getCanonicalPath).createOrReplaceTempView("t") withTempTable("t") { - checkAnswer( - sql("SELECT * FROM t"), - for { - i <- 1 to 10 - pi <- Seq(1, 2) - ps <- Seq("foo", "bar") - } yield Row(i, i.toString, pi, ps)) checkAnswer( sql("SELECT intField, pi FROM t"), @@ -84,19 +77,6 @@ abstract class OrcPartitionDiscoveryTest extends OrcTest { _ <- Seq("foo", "bar") } yield Row(i, pi)) - checkAnswer( - sql("SELECT * FROM t WHERE pi = 1"), - for { - i <- 1 to 10 - ps <- Seq("foo", "bar") - } yield Row(i, i.toString, 1, ps)) - - checkAnswer( - sql("SELECT * FROM t WHERE ps = 'foo'"), - for { - i <- 1 to 10 - pi <- Seq(1, 2) - } yield Row(i, i.toString, pi, "foo")) } } } From a59f54e4e0c9546810e95b87dbcee53bdedf23f9 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Tue, 3 Apr 2018 20:45:35 +0800 Subject: [PATCH 07/25] add trait FileSourceReader --- .../datasources/v2/FileSourceReader.scala | 110 ++++++++++++++++++ .../datasources/v2/orc/OrcDataSourceV2.scala | 108 +++-------------- 2 files changed, 127 insertions(+), 91 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala new file mode 100644 index 000000000000..89c17934eaa3 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala @@ -0,0 +1,110 @@ +/* + * 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.v2 + +import java.util.{List => JList} + +import scala.collection.JavaConverters._ + +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Expression, UnsafeRow} +import org.apache.spark.sql.execution.PartitionedFileUtil +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.sources.v2.DataSourceOptions +import org.apache.spark.sql.sources.v2.reader._ +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.vectorized.ColumnarBatch + +trait FileSourceReader extends DataSourceReader + with SupportsScanUnsafeRow + with SupportsPushDownRequiredColumns { + def options: DataSourceOptions + def userSpecifiedSchema: Option[StructType] + def dataSchema: StructType + def readFunction: PartitionedFile => Iterator[InternalRow] + protected val sparkSession = SparkSession.getActiveSession + .getOrElse(SparkSession.getDefaultSession.get) + protected val hadoopConf = + sparkSession.sessionState.newHadoopConfWithOptions(options.asMap().asScala.toMap) + protected val sqlConf = sparkSession.sessionState.conf + protected val isCaseSensitive = sqlConf.caseSensitiveAnalysis + protected val ignoreCorruptFiles = sqlConf.ignoreCorruptFiles + protected val ignoreMissingFiles = sqlConf.ignoreMissingFiles + protected val fileIndex = { + val filePath = options.get("path") + if (!filePath.isPresent) { + throw new AnalysisException("ORC data source requires a" + + " path (e.g. data backed by a local or distributed file system).") + } + val rootPathsSpecified = + DataSource.checkAndGlobPathIfNecessary(hadoopConf, filePath.get, checkFilesExist = true) + new InMemoryFileIndex(sparkSession, rootPathsSpecified, options.asMap().asScala.toMap, None) + } + + protected val partitionSchema = PartitioningUtils.combineInferredAndUserSpecifiedPartitionSchema( + fileIndex, userSpecifiedSchema, isCaseSensitive) + protected val (fullSchema, _) = + PartitioningUtils.mergeDataAndPartitionSchema(dataSchema, partitionSchema, isCaseSensitive) + protected var requiredSchema = fullSchema + protected var partitionFilters: Array[Expression] = Array.empty + + protected def partitions: Seq[FilePartition] = { + val selectedPartitions = fileIndex.listFiles(partitionFilters, Seq.empty) + val maxSplitBytes = PartitionedFileUtil.maxSplitBytes(sparkSession, selectedPartitions) + val splitFiles = selectedPartitions.flatMap { partition => + partition.files.flatMap { file => + PartitionedFileUtil.splitFiles( + sparkSession = sparkSession, + file = file, + filePath = file.getPath, + isSplitable = true, + maxSplitBytes = maxSplitBytes, + partitionValues = partition.values + ) + }.toArray.sortBy(_.length)(implicitly[Ordering[Long]].reverse) + } + FilePartitionUtil.getFilePartitions(sparkSession, splitFiles, maxSplitBytes) + } + + override def readSchema(): StructType = { + requiredSchema + } + + override def pruneColumns(requiredSchema: StructType): Unit = { + this.requiredSchema = requiredSchema + } + + override def createUnsafeRowReaderFactories: JList[DataReaderFactory[UnsafeRow]] = { + partitions.map { filePartition => + new FileReaderFactory[UnsafeRow](filePartition, readFunction, ignoreCorruptFiles, + ignoreMissingFiles) + .asInstanceOf[DataReaderFactory[UnsafeRow]] + }.asJava + } +} + +trait ColumnarBatchFileSourceReader extends FileSourceReader with SupportsScanColumnarBatch { + override def createBatchDataReaderFactories(): JList[DataReaderFactory[ColumnarBatch]] = { + partitions.map { filePartition => + new FileReaderFactory[ColumnarBatch](filePartition, readFunction, ignoreCorruptFiles, + ignoreMissingFiles) + .asInstanceOf[DataReaderFactory[ColumnarBatch]] + }.asJava + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala index 7e715f439ebf..4dc7c4f0dd2e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala @@ -17,9 +17,7 @@ package org.apache.spark.sql.execution.datasources.v2.orc import java.net.URI -import java.util.{List => JList, Locale} - -import scala.collection.JavaConverters._ +import java.util.Locale import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -33,17 +31,15 @@ import org.apache.orc.mapreduce.OrcInputFormat import org.apache.spark.TaskContext import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Expression, JoinedRow, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.{Expression, JoinedRow} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection -import org.apache.spark.sql.execution.PartitionedFileUtil import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.orc.{OrcColumnarBatchReader, OrcDeserializer, OrcFilters, OrcUtils} -import org.apache.spark.sql.execution.datasources.v2.FileReaderFactory +import org.apache.spark.sql.execution.datasources.v2.ColumnarBatchFileSourceReader import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema} import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.types.{AtomicType, StructType} -import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration class OrcDataSourceV2 extends DataSourceV2 with ReadSupport with ReadSupportWithSchema { @@ -56,75 +52,21 @@ class OrcDataSourceV2 extends DataSourceV2 with ReadSupport with ReadSupportWith } } -class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: Option[StructType]) - extends DataSourceReader - with SupportsScanColumnarBatch - with SupportsScanUnsafeRow - with SupportsPushDownCatalystFilters - with SupportsPushDownRequiredColumns { - - private val sparkSession = SparkSession.getActiveSession - .getOrElse(SparkSession.getDefaultSession.get) - private val hadoopConf = - sparkSession.sessionState.newHadoopConfWithOptions(options.asMap().asScala.toMap) - private val sqlConf = sparkSession.sessionState.conf - private val isCaseSensitive = sqlConf.caseSensitiveAnalysis - private val ignoreCorruptFiles = sqlConf.ignoreCorruptFiles - private val ignoreMissingFiles = sqlConf.ignoreMissingFiles - private val fileIndex = { - val filePath = options.get("path") - if (!filePath.isPresent) { - throw new AnalysisException("ORC data source requires a" + - " path (e.g. data backed by a local or distributed file system).") - } - val rootPathsSpecified = - DataSource.checkAndGlobPathIfNecessary(hadoopConf, filePath.get, checkFilesExist = true) - new InMemoryFileIndex(sparkSession, rootPathsSpecified, options.asMap().asScala.toMap, None) - } - - private val partitionSchema = PartitioningUtils.combineInferredAndUserSpecifiedPartitionSchema( - fileIndex, userSpecifiedSchema, isCaseSensitive) +case class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: Option[StructType]) + extends ColumnarBatchFileSourceReader + with SupportsPushDownCatalystFilters { - private val dataSchema = userSpecifiedSchema.getOrElse { + lazy val dataSchema: StructType = userSpecifiedSchema.getOrElse { val files = fileIndex.allFiles() OrcUtils.readSchema(sparkSession, files).getOrElse { throw new AnalysisException( s"Unable to infer schema for Orc. It must be specified manually.") } } - private val (fullSchema, _) = - PartitioningUtils.mergeDataAndPartitionSchema(dataSchema, partitionSchema, isCaseSensitive) - private var pushedFiltersArray: Array[Expression] = Array.empty - private var partitionKeyFilters: Array[Expression] = Array.empty - private var requiredSchema = fullSchema - - private def partitions: Seq[FilePartition] = { - val selectedPartitions = fileIndex.listFiles(partitionKeyFilters, Seq.empty) - val maxSplitBytes = PartitionedFileUtil.maxSplitBytes(sparkSession, selectedPartitions) - val splitFiles = selectedPartitions.flatMap { partition => - partition.files.flatMap { file => - PartitionedFileUtil.splitFiles( - sparkSession = sparkSession, - file = file, - filePath = file.getPath, - isSplitable = true, - maxSplitBytes = maxSplitBytes, - partitionValues = partition.values - ) - }.toArray.sortBy(_.length)(implicitly[Ordering[Long]].reverse) - } - FilePartitionUtil.getFilePartitions(sparkSession, splitFiles, maxSplitBytes) - } - - override def readSchema(): StructType = { - requiredSchema - } - override def pruneColumns(requiredSchema: StructType): Unit = { - this.requiredSchema = requiredSchema - } + private var pushedFiltersArray: Array[Expression] = Array.empty - private def readFunction: PartitionedFile => Iterator[InternalRow] = { + def readFunction: PartitionedFile => Iterator[InternalRow] = { val capacity = sqlConf.orcVectorizedReaderBatchSize val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled val copyToSpark = sqlConf.getConf(SQLConf.ORC_COPY_BATCH_TO_SPARK) @@ -205,35 +147,12 @@ class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: Optio } } - override def createBatchDataReaderFactories(): JList[DataReaderFactory[ColumnarBatch]] = { - partitions.map { filePartition => - new FileReaderFactory[ColumnarBatch](filePartition, readFunction, ignoreCorruptFiles, - ignoreMissingFiles) - .asInstanceOf[DataReaderFactory[ColumnarBatch]] - }.asJava - } - - override def createUnsafeRowReaderFactories: JList[DataReaderFactory[UnsafeRow]] = { - partitions.map { filePartition => - new FileReaderFactory[UnsafeRow](filePartition, readFunction, ignoreCorruptFiles, - ignoreMissingFiles) - .asInstanceOf[DataReaderFactory[UnsafeRow]] - }.asJava - } - - override def enableBatchRead(): Boolean = { - val schema = readSchema() - sqlConf.orcVectorizedReaderEnabled && sqlConf.wholeStageEnabled && - schema.length <= sqlConf.wholeStageMaxNumFields && - schema.forall(_.dataType.isInstanceOf[AtomicType]) - } - override def pushCatalystFilters(filters: Array[Expression]): Array[Expression] = { val partitionColumnNames = partitionSchema.toAttributes.map(_.name).toSet val (partitionKeyFilters, otherFilters) = filters.partition { _.references.map(_.name).toSet.subsetOf(partitionColumnNames) } - this.partitionKeyFilters = partitionKeyFilters + this.partitionFilters = partitionKeyFilters pushedFiltersArray = partitionKeyFilters if (sqlConf.orcFilterPushDown) { val dataFilters = otherFilters.map { f => @@ -254,6 +173,13 @@ class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: Optio override def pushedCatalystFilters(): Array[Expression] = { pushedFiltersArray } + + override def enableBatchRead(): Boolean = { + val schema = readSchema() + sqlConf.orcVectorizedReaderEnabled && sqlConf.wholeStageEnabled && + schema.length <= sqlConf.wholeStageMaxNumFields && + schema.forall(_.dataType.isInstanceOf[AtomicType]) + } } object OrcDataSourceV2 { From 0fbb09165d04dd7829f7018da6cb02d53e65b0a1 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Tue, 3 Apr 2018 22:18:38 +0800 Subject: [PATCH 08/25] Revise --- .../datasources/v2/FileSourceReader.scala | 30 ++++++++++++++----- .../datasources/v2/orc/OrcDataSourceV2.scala | 4 ++- 2 files changed, 26 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala index 89c17934eaa3..f2994acead27 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala @@ -21,6 +21,8 @@ import java.util.{List => JList} import scala.collection.JavaConverters._ +import org.apache.hadoop.fs.Path + import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Expression, UnsafeRow} @@ -36,8 +38,21 @@ trait FileSourceReader extends DataSourceReader with SupportsPushDownRequiredColumns { def options: DataSourceOptions def userSpecifiedSchema: Option[StructType] + + /** + * Returns schema of input data + */ def dataSchema: StructType + def readFunction: PartitionedFile => Iterator[InternalRow] + + /** + * Returns whether a file with `path` could be split or not. + */ + def isSplitable(path: Path): Boolean = { + false + } + protected val sparkSession = SparkSession.getActiveSession .getOrElse(SparkSession.getDefaultSession.get) protected val hadoopConf = @@ -49,7 +64,7 @@ trait FileSourceReader extends DataSourceReader protected val fileIndex = { val filePath = options.get("path") if (!filePath.isPresent) { - throw new AnalysisException("ORC data source requires a" + + throw new AnalysisException("Reading data source requires a" + " path (e.g. data backed by a local or distributed file system).") } val rootPathsSpecified = @@ -69,11 +84,12 @@ trait FileSourceReader extends DataSourceReader val maxSplitBytes = PartitionedFileUtil.maxSplitBytes(sparkSession, selectedPartitions) val splitFiles = selectedPartitions.flatMap { partition => partition.files.flatMap { file => + val filePath = file.getPath PartitionedFileUtil.splitFiles( sparkSession = sparkSession, file = file, - filePath = file.getPath, - isSplitable = true, + filePath = filePath, + isSplitable = isSplitable(filePath), maxSplitBytes = maxSplitBytes, partitionValues = partition.values ) @@ -92,8 +108,8 @@ trait FileSourceReader extends DataSourceReader override def createUnsafeRowReaderFactories: JList[DataReaderFactory[UnsafeRow]] = { partitions.map { filePartition => - new FileReaderFactory[UnsafeRow](filePartition, readFunction, ignoreCorruptFiles, - ignoreMissingFiles) + new FileReaderFactory[UnsafeRow](filePartition, readFunction, + ignoreCorruptFiles, ignoreMissingFiles) .asInstanceOf[DataReaderFactory[UnsafeRow]] }.asJava } @@ -102,8 +118,8 @@ trait FileSourceReader extends DataSourceReader trait ColumnarBatchFileSourceReader extends FileSourceReader with SupportsScanColumnarBatch { override def createBatchDataReaderFactories(): JList[DataReaderFactory[ColumnarBatch]] = { partitions.map { filePartition => - new FileReaderFactory[ColumnarBatch](filePartition, readFunction, ignoreCorruptFiles, - ignoreMissingFiles) + new FileReaderFactory[ColumnarBatch](filePartition, readFunction, + ignoreCorruptFiles, ignoreMissingFiles) .asInstanceOf[DataReaderFactory[ColumnarBatch]] }.asJava } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala index 4dc7c4f0dd2e..e6836c29f930 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala @@ -66,7 +66,7 @@ case class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: private var pushedFiltersArray: Array[Expression] = Array.empty - def readFunction: PartitionedFile => Iterator[InternalRow] = { + override def readFunction: PartitionedFile => Iterator[InternalRow] = { val capacity = sqlConf.orcVectorizedReaderBatchSize val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled val copyToSpark = sqlConf.getConf(SQLConf.ORC_COPY_BATCH_TO_SPARK) @@ -180,6 +180,8 @@ case class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: schema.length <= sqlConf.wholeStageMaxNumFields && schema.forall(_.dataType.isInstanceOf[AtomicType]) } + + override def isSplitable(path: Path): Boolean = true } object OrcDataSourceV2 { From 9936333a27ca2367f8db1df80d19e51b3eb94870 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Wed, 4 Apr 2018 06:47:07 +0800 Subject: [PATCH 09/25] revise --- .../apache/spark/sql/execution/datasources/orc/OrcUtils.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala index 0dfc6e301502..460194ba61c8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.datasources.orc -import java.util.Locale - import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration From 5dba7e26e70e6933ac38784d45b0010dd88963e6 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Wed, 4 Apr 2018 20:35:16 +0800 Subject: [PATCH 10/25] revise fileIndex --- .../datasources/v2/FileSourceReader.scala | 20 +++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala index f2994acead27..5b98ae9b4047 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala @@ -61,19 +61,27 @@ trait FileSourceReader extends DataSourceReader protected val isCaseSensitive = sqlConf.caseSensitiveAnalysis protected val ignoreCorruptFiles = sqlConf.ignoreCorruptFiles protected val ignoreMissingFiles = sqlConf.ignoreMissingFiles - protected val fileIndex = { + private val rootPathsSpecified = { val filePath = options.get("path") if (!filePath.isPresent) { throw new AnalysisException("Reading data source requires a" + " path (e.g. data backed by a local or distributed file system).") } - val rootPathsSpecified = - DataSource.checkAndGlobPathIfNecessary(hadoopConf, filePath.get, checkFilesExist = true) - new InMemoryFileIndex(sparkSession, rootPathsSpecified, options.asMap().asScala.toMap, None) + DataSource.checkAndGlobPathIfNecessary(hadoopConf, filePath.get, checkFilesExist = true) + } + private val fileStatusCache = FileStatusCache.getOrCreate(sparkSession) + protected val partitionSchema = { + val tempFileIndex = { + new InMemoryFileIndex(sparkSession, + rootPathsSpecified, options.asMap().asScala.toMap, None, fileStatusCache) + } + PartitioningUtils.combineInferredAndUserSpecifiedPartitionSchema( + tempFileIndex, userSpecifiedSchema, isCaseSensitive) } - protected val partitionSchema = PartitioningUtils.combineInferredAndUserSpecifiedPartitionSchema( - fileIndex, userSpecifiedSchema, isCaseSensitive) + protected val fileIndex = + new InMemoryFileIndex(sparkSession, + rootPathsSpecified, options.asMap().asScala.toMap, Some(partitionSchema), fileStatusCache) protected val (fullSchema, _) = PartitioningUtils.mergeDataAndPartitionSchema(dataSchema, partitionSchema, isCaseSensitive) protected var requiredSchema = fullSchema From a6dba6a26d1d5e30e5a9ba1a197b89d7c561a8e7 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Thu, 5 Apr 2018 00:02:08 +0800 Subject: [PATCH 11/25] use inferSchema --- .../datasources/v2/FileSourceReader.scala | 23 +++++++++++++------ .../datasources/v2/orc/OrcDataSourceV2.scala | 12 ++++------ 2 files changed, 20 insertions(+), 15 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala index 5b98ae9b4047..deb7ce0f2404 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala @@ -21,7 +21,7 @@ import java.util.{List => JList} import scala.collection.JavaConverters._ -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.InternalRow @@ -40,9 +40,11 @@ trait FileSourceReader extends DataSourceReader def userSpecifiedSchema: Option[StructType] /** - * Returns schema of input data + * When possible, this method should return the schema of the given `files`. When the format + * does not support inference, or no valid files are given should return None. In these cases + * Spark will require that user specify the schema manually. */ - def dataSchema: StructType + def inferSchema(files: Seq[FileStatus]): Option[StructType] def readFunction: PartitionedFile => Iterator[InternalRow] @@ -72,16 +74,23 @@ trait FileSourceReader extends DataSourceReader private val fileStatusCache = FileStatusCache.getOrCreate(sparkSession) protected val partitionSchema = { val tempFileIndex = { - new InMemoryFileIndex(sparkSession, - rootPathsSpecified, options.asMap().asScala.toMap, None, fileStatusCache) + new InMemoryFileIndex(sparkSession, rootPathsSpecified, + options.asMap().asScala.toMap, None, fileStatusCache) } PartitioningUtils.combineInferredAndUserSpecifiedPartitionSchema( tempFileIndex, userSpecifiedSchema, isCaseSensitive) } protected val fileIndex = - new InMemoryFileIndex(sparkSession, - rootPathsSpecified, options.asMap().asScala.toMap, Some(partitionSchema), fileStatusCache) + new InMemoryFileIndex(sparkSession, rootPathsSpecified, + options.asMap().asScala.toMap, Some(partitionSchema), fileStatusCache) + + protected lazy val dataSchema = userSpecifiedSchema.orElse { + inferSchema(fileIndex.allFiles()) + }.getOrElse { + throw new AnalysisException( + s"Unable to infer schema for $rootPathsSpecified. It must be specified manually.") + } protected val (fullSchema, _) = PartitioningUtils.mergeDataAndPartitionSchema(dataSchema, partitionSchema, isCaseSensitive) protected var requiredSchema = fullSchema diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala index e6836c29f930..2a6edde32af9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala @@ -20,7 +20,7 @@ import java.net.URI import java.util.Locale import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType} import org.apache.hadoop.mapreduce.lib.input.FileSplit import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl @@ -29,7 +29,7 @@ import org.apache.orc.mapred.OrcStruct import org.apache.orc.mapreduce.OrcInputFormat import org.apache.spark.TaskContext -import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Expression, JoinedRow} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection @@ -56,12 +56,8 @@ case class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: extends ColumnarBatchFileSourceReader with SupportsPushDownCatalystFilters { - lazy val dataSchema: StructType = userSpecifiedSchema.getOrElse { - val files = fileIndex.allFiles() - OrcUtils.readSchema(sparkSession, files).getOrElse { - throw new AnalysisException( - s"Unable to infer schema for Orc. It must be specified manually.") - } + override def inferSchema(files: Seq[FileStatus]): Option[StructType] = { + OrcUtils.readSchema(sparkSession, files) } private var pushedFiltersArray: Array[Expression] = Array.empty From 4955714bc519c1bb4623f55e15e5264d1d199b7a Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Fri, 6 Apr 2018 16:56:19 +0800 Subject: [PATCH 12/25] address comments --- .../orc/OrcColumnarBatchReader.java | 19 +++++++++++----- .../orc/OrcPartitionDiscoverySuite.scala | 22 ++++++++++++++++++- 2 files changed, 34 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java index aa5e31460926..6982ebb80cc9 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java @@ -58,7 +58,7 @@ public class OrcColumnarBatchReader extends RecordReader { /** * The column IDs of the physical ORC file schema which are required by this reader. - * -1 means this required column doesn't exist in the ORC file. + * -1 means this required column is partition column, or it doesn't exist in the ORC file. */ private int[] requestedColIds; @@ -164,11 +164,17 @@ public void initBatch( InternalRow partitionValues) { batch = orcSchema.createRowBatch(capacity); assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`. - this.requiredFields = requiredFields; - this.requestedColIds = requestedColIds; - this.requestedPartitionColIds = requestedPartitionColIds; assert(requiredFields.length == requestedColIds.length); assert(requiredFields.length == requestedPartitionColIds.length); + // If a required column is also partition column, use partition value and don't read from file. + for (int i = 0; i < requiredFields.length; i++) { + if (requestedPartitionColIds[i] != -1) { + requestedColIds[i] = -1; + } + } + this.requestedPartitionColIds = requestedPartitionColIds; + this.requiredFields = requiredFields; + this.requestedColIds = requestedColIds; StructType resultSchema = new StructType(requiredFields); if (copyToSpark) { @@ -234,7 +240,8 @@ private boolean nextBatch() throws IOException { if (!copyToSpark) { for (int i = 0; i < requiredFields.length; i++) { - if (requestedColIds[i] != -1 && requestedPartitionColIds[i] == -1) { + // It is possible that.. + if (requestedColIds[i] != -1) { ((OrcColumnVector) orcVectorWrappers[i]).setBatchSize(batchSize); } } @@ -249,7 +256,7 @@ private boolean nextBatch() throws IOException { StructField field = requiredFields[i]; WritableColumnVector toColumn = columnVectors[i]; - if (requestedColIds[i] >= 0) { + if (requestedColIds[i] != -1) { ColumnVector fromColumn = batch.cols[requestedColIds[i]]; if (fromColumn.isRepeating) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala index bd0d1ee7503a..a955647281fc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala @@ -68,8 +68,15 @@ abstract class OrcPartitionDiscoveryTest extends OrcTest { spark.read.orc(base.getCanonicalPath).createOrReplaceTempView("t") withTempTable("t") { - checkAnswer( + sql("SELECT * FROM t"), + for { + i <- 1 to 10 + pi <- Seq(1, 2) + ps <- Seq("foo", "bar") + } yield Row(i, i.toString, pi, ps)) + + checkAnswer( sql("SELECT intField, pi FROM t"), for { i <- 1 to 10 @@ -77,6 +84,19 @@ abstract class OrcPartitionDiscoveryTest extends OrcTest { _ <- Seq("foo", "bar") } yield Row(i, pi)) + checkAnswer( + sql("SELECT * FROM t WHERE pi = 1"), + for { + i <- 1 to 10 + ps <- Seq("foo", "bar") + } yield Row(i, i.toString, 1, ps)) + + checkAnswer( + sql("SELECT * FROM t WHERE ps = 'foo'"), + for { + i <- 1 to 10 + pi <- Seq(1, 2) + } yield Row(i, i.toString, pi, "foo")) } } } From 7bc99518a94223343c73d68329bcc432ce6228e1 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Mon, 9 Apr 2018 20:56:29 +0800 Subject: [PATCH 13/25] better hack for lookupDataSource --- .../org/apache/spark/sql/DataFrameReader.scala | 13 +++---------- .../sql/execution/datasources/DataSource.scala | 15 +++++++++++++-- .../datasources/v2/orc/OrcDataSourceV2.scala | 15 --------------- 3 files changed, 16 insertions(+), 27 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 78867c8ac2d6..9341918a9323 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -37,7 +37,6 @@ import org.apache.spark.sql.execution.datasources.jdbc._ import org.apache.spark.sql.execution.datasources.json.TextInputJsonDataSource import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils -import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema} import org.apache.spark.sql.types.{StringType, StructType} import org.apache.spark.unsafe.types.UTF8String @@ -192,15 +191,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { "read files of Hive data source directly.") } - // SPARK-23817 Since datasource V2 didn't support reading multiple files yet, - // ORC V2 is only used when loading single file path. - val allPaths = CaseInsensitiveMap(extraOptions.toMap).get("path") ++ paths - val orcV2 = OrcDataSourceV2.satisfy(sparkSession, source, allPaths.toSeq) - if (orcV2.isDefined) { - option("path", allPaths.head) - source = orcV2.get - } - val cls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf) + val allPaths = (CaseInsensitiveMap(extraOptions.toMap).get("path") ++ paths).toSeq + option("path", allPaths.mkString(",")) + val cls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf, allPaths) if (classOf[DataSourceV2].isAssignableFrom(cls)) { val ds = cls.newInstance().asInstanceOf[DataSourceV2] if (ds.isInstanceOf[ReadSupport] || ds.isInstanceOf[ReadSupportWithSchema]) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 72c170a41314..0401107c39e4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -39,6 +39,7 @@ 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.datasources.v2.orc.OrcDataSourceV2 import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources.{RateStreamProvider, TextSocketSourceProvider} import org.apache.spark.sql.internal.SQLConf @@ -605,11 +606,21 @@ object DataSource extends Logging { "org.apache.spark.Logging") /** Given a provider name, look up the data source class definition. */ - def lookupDataSource(provider: String, conf: SQLConf): Class[_] = { + def lookupDataSource( + provider: String, + conf: SQLConf, + paths: Seq[String] = Seq.empty): Class[_] = { + val disabledV2Readers = conf.disabledV2DataSourceReader.split(",") val provider1 = backwardCompatibilityMap.getOrElse(provider, provider) match { case name if name.equalsIgnoreCase("orc") && conf.getConf(SQLConf.ORC_IMPLEMENTATION) == "native" => - classOf[OrcFileFormat].getCanonicalName + // SPARK-23817 Since datasource V2 didn't support reading multiple files yet, + // ORC V2 is only used when loading single file path. + if (paths.length == 1 && !disabledV2Readers.contains("orc")) { + classOf[OrcDataSourceV2].getCanonicalName + } else { + classOf[OrcFileFormat].getCanonicalName + } case name if name.equalsIgnoreCase("orc") && conf.getConf(SQLConf.ORC_IMPLEMENTATION) == "hive" => "org.apache.spark.sql.hive.orc.OrcFileFormat" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala index 2a6edde32af9..4384ec07800b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution.datasources.v2.orc import java.net.URI -import java.util.Locale import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} @@ -29,7 +28,6 @@ import org.apache.orc.mapred.OrcStruct import org.apache.orc.mapreduce.OrcInputFormat import org.apache.spark.TaskContext -import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Expression, JoinedRow} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection @@ -179,16 +177,3 @@ case class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: override def isSplitable(path: Path): Boolean = true } - -object OrcDataSourceV2 { - def satisfy(sparkSession: SparkSession, source: String, paths: Seq[String]): Option[String] = { - val disabledV2Readers = sparkSession.sqlContext.conf.disabledV2DataSourceReader.split(",") - val isNative = sparkSession.sqlContext.conf.getConf(SQLConf.ORC_IMPLEMENTATION) == "native" - if (source.toLowerCase(Locale.ROOT) == "orc" && isNative && - !disabledV2Readers.contains(source) && paths.length == 1) { - Some("org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2") - } else { - None - } - } -} From 31fe23517c63fb840466d2952632490a6991c799 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Mon, 16 Apr 2018 16:36:48 +0800 Subject: [PATCH 14/25] fix compilation --- .../apache/spark/sql/DataFrameReader.scala | 2 +- .../execution/datasources/DataSource.scala | 47 ++++++++++++------- .../datasources/v2/FileSourceReader.scala | 22 ++++----- 3 files changed, 40 insertions(+), 31 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 9341918a9323..61a90d12616b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -192,7 +192,6 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { } val allPaths = (CaseInsensitiveMap(extraOptions.toMap).get("path") ++ paths).toSeq - option("path", allPaths.mkString(",")) val cls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf, allPaths) if (classOf[DataSourceV2].isAssignableFrom(cls)) { val ds = cls.newInstance().asInstanceOf[DataSourceV2] @@ -203,6 +202,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { val objectMapper = new ObjectMapper() DataSourceOptions.PATHS_KEY -> objectMapper.writeValueAsString(paths.toArray) } + Dataset.ofRows(sparkSession, DataSourceV2Relation.create( ds, extraOptions.toMap ++ sessionOptions + pathsOption, userSpecifiedSchema = userSpecifiedSchema)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 0401107c39e4..0cceb155661f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -23,6 +23,7 @@ import scala.collection.JavaConverters._ import scala.language.{existentials, implicitConversions} import scala.util.{Failure, Success, Try} +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.deploy.SparkHadoopUtil @@ -538,23 +539,8 @@ case class DataSource( checkFilesExist: Boolean): Seq[Path] = { val allPaths = caseInsensitiveOptions.get("path") ++ paths val hadoopConf = sparkSession.sessionState.newHadoopConf() - allPaths.flatMap { path => - val hdfsPath = new Path(path) - val fs = hdfsPath.getFileSystem(hadoopConf) - val qualified = hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory) - val globPath = SparkHadoopUtil.get.globPathIfNecessary(fs, qualified) - - if (checkEmptyGlobPath && globPath.isEmpty) { - throw new AnalysisException(s"Path does not exist: $qualified") - } - - // Sufficient to check head of the globPath seq for non-glob scenario - // Don't need to check once again if files exist in streaming mode - if (checkFilesExist && !fs.exists(globPath.head)) { - throw new AnalysisException(s"Path does not exist: ${globPath.head}") - } - globPath - }.toSeq + DataSource.checkAndGlobPathIfNecessary(allPaths.toSeq, hadoopConf, + checkEmptyGlobPath, checkFilesExist) } } @@ -701,6 +687,33 @@ object DataSource extends Logging { } } + /** + * Checks and returns files in all the paths. + */ + private[sql] def checkAndGlobPathIfNecessary( + paths: Seq[String], + hadoopConf: Configuration, + checkEmptyGlobPath: Boolean, + checkFilesExist: Boolean): Seq[Path] = { + paths.flatMap { path => + val hdfsPath = new Path(path) + val fs = hdfsPath.getFileSystem(hadoopConf) + val qualified = hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory) + val globPath = SparkHadoopUtil.get.globPathIfNecessary(fs, qualified) + + if (checkEmptyGlobPath && globPath.isEmpty) { + throw new AnalysisException(s"Path does not exist: $qualified") + } + + // Sufficient to check head of the globPath seq for non-glob scenario + // Don't need to check once again if files exist in streaming mode + if (checkFilesExist && !fs.exists(globPath.head)) { + throw new AnalysisException(s"Path does not exist: ${globPath.head}") + } + globPath + } + } + /** * When creating a data source table, the `path` option has a special meaning: the table location. * This method extracts the `path` option and treat it as table location to build a diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala index deb7ce0f2404..d0215fd87208 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala @@ -63,27 +63,23 @@ trait FileSourceReader extends DataSourceReader protected val isCaseSensitive = sqlConf.caseSensitiveAnalysis protected val ignoreCorruptFiles = sqlConf.ignoreCorruptFiles protected val ignoreMissingFiles = sqlConf.ignoreMissingFiles - private val rootPathsSpecified = { + private lazy val rootPathsSpecified = { val filePath = options.get("path") if (!filePath.isPresent) { throw new AnalysisException("Reading data source requires a" + " path (e.g. data backed by a local or distributed file system).") } - DataSource.checkAndGlobPathIfNecessary(hadoopConf, filePath.get, checkFilesExist = true) - } - private val fileStatusCache = FileStatusCache.getOrCreate(sparkSession) - protected val partitionSchema = { - val tempFileIndex = { - new InMemoryFileIndex(sparkSession, rootPathsSpecified, - options.asMap().asScala.toMap, None, fileStatusCache) - } - PartitioningUtils.combineInferredAndUserSpecifiedPartitionSchema( - tempFileIndex, userSpecifiedSchema, isCaseSensitive) + DataSource.checkAndGlobPathIfNecessary(Seq(filePath.get), hadoopConf, + checkEmptyGlobPath = false, checkFilesExist = false) } - protected val fileIndex = + protected lazy val fileIndex = { + val fileStatusCache = FileStatusCache.getOrCreate(sparkSession) new InMemoryFileIndex(sparkSession, rootPathsSpecified, - options.asMap().asScala.toMap, Some(partitionSchema), fileStatusCache) + options.asMap().asScala.toMap, userSpecifiedSchema, fileStatusCache) + } + + protected lazy val partitionSchema = fileIndex.partitionSchema protected lazy val dataSchema = userSpecifiedSchema.orElse { inferSchema(fileIndex.allFiles()) From 8894bbd2bdd2914ec1f7dce75b12b5edc8637dd2 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Mon, 16 Apr 2018 21:02:58 +0800 Subject: [PATCH 15/25] revise --- .../datasources/DataSourceStrategy.scala | 3 ++- .../datasources/PartitioningUtils.scala | 22 ------------------- 2 files changed, 2 insertions(+), 23 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index fe18b977e072..6c083b7d0c05 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -37,6 +37,7 @@ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{RowDataSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 import org.apache.spark.sql.internal.SQLConf @@ -226,7 +227,7 @@ class FallBackToOrcV1(sparkSession: SparkSession) extends Rule[LogicalPlan] { sparkSession = sparkSession, paths = Seq.empty, userSpecifiedSchema = v2Relation.userSpecifiedSchema, - className = "org.apache.spark.sql.execution.datasources.orc", + className = classOf[OrcFileFormat].getCanonicalName, options = v2Relation.options).resolveRelation() LogicalRelation(v1) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index f4377fc18ad9..862ee64de9b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -541,28 +541,6 @@ object PartitioningUtils { StructType(fields) } - /** - * In the read path, only managed tables by Hive provide the partition columns properly when - * initializing this class. All other file based data sources will try to infer the partitioning, - * and then cast the inferred types to user specified dataTypes if the partition columns exist - * inside `userSpecifiedSchema`, otherwise we can hit data corruption bugs like SPARK-18510, or - * inconsistent data types as reported in SPARK-21463. - * @param fileIndex A FileIndex that will perform partition inference - * @return The PartitionSchema resolved from inference and cast according to `userSpecifiedSchema` - */ - def combineInferredAndUserSpecifiedPartitionSchema( - fileIndex: FileIndex, - userSpecifiedSchema: Option[StructType] = None, - caseSensitive: Boolean): StructType = { - val equality = columnNameEquality(caseSensitive) - val resolved = fileIndex.partitionSchema.map { partitionField => - // SPARK-18510: try to get schema from userSpecifiedSchema, otherwise fallback to inferred - userSpecifiedSchema.flatMap(_.find(f => equality(f.name, partitionField.name))).getOrElse( - partitionField) - } - StructType(resolved) - } - private def getColName(f: StructField, caseSensitive: Boolean): String = { if (caseSensitive) { f.name From 3f0a67c7934f940cfcd577a5ab842d4197ce12cf Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Mon, 16 Apr 2018 22:08:24 +0800 Subject: [PATCH 16/25] revise --- .../execution/datasources/v2/FileSourceReader.scala | 10 +++++++++- .../execution/datasources/v2/orc/OrcDataSourceV2.scala | 6 ------ 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala index d0215fd87208..2d0073005f36 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala @@ -35,7 +35,8 @@ import org.apache.spark.sql.vectorized.ColumnarBatch trait FileSourceReader extends DataSourceReader with SupportsScanUnsafeRow - with SupportsPushDownRequiredColumns { + with SupportsPushDownRequiredColumns + with SupportsPushDownCatalystFilters { def options: DataSourceOptions def userSpecifiedSchema: Option[StructType] @@ -91,6 +92,7 @@ trait FileSourceReader extends DataSourceReader PartitioningUtils.mergeDataAndPartitionSchema(dataSchema, partitionSchema, isCaseSensitive) protected var requiredSchema = fullSchema protected var partitionFilters: Array[Expression] = Array.empty + protected var pushedFiltersArray: Array[Expression] = Array.empty protected def partitions: Seq[FilePartition] = { val selectedPartitions = fileIndex.listFiles(partitionFilters, Seq.empty) @@ -119,6 +121,12 @@ trait FileSourceReader extends DataSourceReader this.requiredSchema = requiredSchema } + override def pushCatalystFilters(filters: Array[Expression]): Array[Expression] = Array.empty + + override def pushedCatalystFilters(): Array[Expression] = { + pushedFiltersArray + } + override def createUnsafeRowReaderFactories: JList[DataReaderFactory[UnsafeRow]] = { partitions.map { filePartition => new FileReaderFactory[UnsafeRow](filePartition, readFunction, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala index 4384ec07800b..730b0305cc3c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala @@ -58,8 +58,6 @@ case class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: OrcUtils.readSchema(sparkSession, files) } - private var pushedFiltersArray: Array[Expression] = Array.empty - override def readFunction: PartitionedFile => Iterator[InternalRow] = { val capacity = sqlConf.orcVectorizedReaderBatchSize val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled @@ -164,10 +162,6 @@ case class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: otherFilters } - override def pushedCatalystFilters(): Array[Expression] = { - pushedFiltersArray - } - override def enableBatchRead(): Boolean = { val schema = readSchema() sqlConf.orcVectorizedReaderEnabled && sqlConf.wholeStageEnabled && From c90f1f7574b86a98f7f535621d3e415458a6c81e Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Mon, 16 Apr 2018 22:18:04 +0800 Subject: [PATCH 17/25] revise --- .../sql/execution/datasources/v2/orc/OrcDataSourceV2.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala index 730b0305cc3c..2cc0f9b34668 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala @@ -51,8 +51,7 @@ class OrcDataSourceV2 extends DataSourceV2 with ReadSupport with ReadSupportWith } case class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: Option[StructType]) - extends ColumnarBatchFileSourceReader - with SupportsPushDownCatalystFilters { + extends ColumnarBatchFileSourceReader { override def inferSchema(files: Seq[FileStatus]): Option[StructType] = { OrcUtils.readSchema(sparkSession, files) From ba84051390d5a8863708ba1c03f2b1127e3a16d0 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Thu, 19 Apr 2018 16:04:55 +0800 Subject: [PATCH 18/25] update to latest master --- .../spark/sql/execution/datasources/DataSource.scala | 8 +++----- .../sql/execution/datasources/DataSourceStrategy.scala | 2 +- .../execution/datasources/v2/DataSourceV2Relation.scala | 2 +- .../sql/execution/datasources/v2/FileSourceReader.scala | 6 +++--- 4 files changed, 8 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 0cceb155661f..a7a828c64e64 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -600,12 +600,10 @@ object DataSource extends Logging { val provider1 = backwardCompatibilityMap.getOrElse(provider, provider) match { case name if name.equalsIgnoreCase("orc") && conf.getConf(SQLConf.ORC_IMPLEMENTATION) == "native" => - // SPARK-23817 Since datasource V2 didn't support reading multiple files yet, - // ORC V2 is only used when loading single file path. - if (paths.length == 1 && !disabledV2Readers.contains("orc")) { - classOf[OrcDataSourceV2].getCanonicalName - } else { + if (disabledV2Readers.contains("orc") || paths.length == 0) { classOf[OrcFileFormat].getCanonicalName + } else { + classOf[OrcDataSourceV2].getCanonicalName } case name if name.equalsIgnoreCase("orc") && conf.getConf(SQLConf.ORC_IMPLEMENTATION) == "hive" => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 6c083b7d0c05..16ba4cb546e5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -225,7 +225,7 @@ class FallBackToOrcV1(sparkSession: SparkSession) extends Rule[LogicalPlan] { private def convertToOrcV1(v2Relation: DataSourceV2Relation): LogicalPlan = { val v1 = DataSource.apply( sparkSession = sparkSession, - paths = Seq.empty, + paths = v2Relation.v2Options.paths(), userSpecifiedSchema = v2Relation.userSpecifiedSchema, className = classOf[OrcFileFormat].getCanonicalName, options = v2Relation.options).resolveRelation() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala index 90fb5a14c9fc..83ef7e534517 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -52,7 +52,7 @@ case class DataSourceV2Relation( AttributeReference(f.name, f.dataType, f.nullable, f.metadata)())) } - private lazy val v2Options: DataSourceOptions = makeV2Options(options) + private[sql] lazy val v2Options: DataSourceOptions = makeV2Options(options) // postScanFilters: filters that need to be evaluated after the scan. // pushedFilters: filters that will be pushed down and evaluated in the underlying data sources. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala index 2d0073005f36..f87fb656cb00 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala @@ -65,12 +65,12 @@ trait FileSourceReader extends DataSourceReader protected val ignoreCorruptFiles = sqlConf.ignoreCorruptFiles protected val ignoreMissingFiles = sqlConf.ignoreMissingFiles private lazy val rootPathsSpecified = { - val filePath = options.get("path") - if (!filePath.isPresent) { + val filePaths = options.paths() + if (filePaths.isEmpty) { throw new AnalysisException("Reading data source requires a" + " path (e.g. data backed by a local or distributed file system).") } - DataSource.checkAndGlobPathIfNecessary(Seq(filePath.get), hadoopConf, + DataSource.checkAndGlobPathIfNecessary(filePaths, hadoopConf, checkEmptyGlobPath = false, checkFilesExist = false) } From c91db6d6b1435da85f3f84bdf5e36f55968a9ff1 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Fri, 20 Apr 2018 14:44:47 +0800 Subject: [PATCH 19/25] add FileDataSourceV2 and better fall back --- .../apache/spark/sql/DataFrameReader.scala | 21 +++++++++----- .../apache/spark/sql/DataFrameWriter.scala | 10 ++++--- .../execution/datasources/DataSource.scala | 18 ++++++------ .../datasources/DataSourceStrategy.scala | 28 +++++++++---------- .../datasources/v2/FileDataSourceV2.scala | 24 ++++++++++++++++ .../datasources/v2/orc/OrcDataSourceV2.scala | 10 +++++-- .../internal/BaseSessionStateBuilder.scala | 2 +- 7 files changed, 76 insertions(+), 37 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 61a90d12616b..f02871f204b9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -35,8 +35,7 @@ import org.apache.spark.sql.execution.datasources.{DataSource, FailureSafeParser import org.apache.spark.sql.execution.datasources.csv._ import org.apache.spark.sql.execution.datasources.jdbc._ import org.apache.spark.sql.execution.datasources.json.TextInputJsonDataSource -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2Utils, FileDataSourceV2} import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema} import org.apache.spark.sql.types.{StringType, StructType} import org.apache.spark.unsafe.types.UTF8String @@ -194,8 +193,12 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { val allPaths = (CaseInsensitiveMap(extraOptions.toMap).get("path") ++ paths).toSeq val cls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf, allPaths) if (classOf[DataSourceV2].isAssignableFrom(cls)) { + val disabledV2Readers = sparkSession.sessionState.conf.disabledV2DataSourceReader.split(",") val ds = cls.newInstance().asInstanceOf[DataSourceV2] - if (ds.isInstanceOf[ReadSupport] || ds.isInstanceOf[ReadSupportWithSchema]) { + val fallBackToV1 = ds.isInstanceOf[FileDataSourceV2] && + disabledV2Readers.contains(ds.asInstanceOf[FileDataSourceV2].shortName) + val supportsRead = ds.isInstanceOf[ReadSupport] || ds.isInstanceOf[ReadSupportWithSchema] + if (supportsRead && !fallBackToV1) { val sessionOptions = DataSourceV2Utils.extractSessionConfigs( ds = ds, conf = sparkSession.sessionState.conf) val pathsOption = { @@ -207,21 +210,25 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { ds, extraOptions.toMap ++ sessionOptions + pathsOption, userSpecifiedSchema = userSpecifiedSchema)) } else { - loadV1Source(paths: _*) + if (fallBackToV1) { + loadV1Source(ds.asInstanceOf[FileDataSourceV2].fallBackFileFormat, paths: _*) + } else { + loadV1Source(None, paths: _*) + } } } else { - loadV1Source(paths: _*) + loadV1Source(None, paths: _*) } } - private def loadV1Source(paths: String*) = { + private def loadV1Source(cls: Option[Class[_]], paths: String*) = { // Code path for data source v1. sparkSession.baseRelationToDataFrame( DataSource.apply( sparkSession, paths = paths, userSpecifiedSchema = userSpecifiedSchema, - className = source, + className = cls.map(_.getCanonicalName).getOrElse(source), options = extraOptions.toMap).resolveRelation()) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 90bea2d676e2..deed13a3f6cc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -30,8 +30,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{AnalysisBarrier, InsertIntoT import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, LogicalRelation} -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils -import org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2 +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Utils, FileDataSourceV2, FileSourceReader, WriteToDataSourceV2} import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.sources.v2._ import org.apache.spark.sql.types.StructType @@ -258,6 +257,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } } + case f: FileDataSourceV2 => + saveToV1Source(f.fallBackFileFormat) + // Streaming also uses the data source V2 API. So it may be that the data source implements // v2, but has no v2 implementation for batch writes. In that case, we fall back to saving // as though it's a V1 source. @@ -268,12 +270,12 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } } - private def saveToV1Source(): Unit = { + private def saveToV1Source(cls: Option[Class[_]] = None): Unit = { // Code path for data source v1. runCommand(df.sparkSession, "save") { DataSource( sparkSession = df.sparkSession, - className = source, + className = cls.map(_.getCanonicalName).getOrElse(source), partitionColumns = partitioningColumns.getOrElse(Nil), options = extraOptions.toMap).planForWriting(mode, AnalysisBarrier(df.logicalPlan)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index a7a828c64e64..af0a4431330f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.datasources import java.util.{Locale, ServiceConfigurationError, ServiceLoader} +import javax.activation.FileDataSource import scala.collection.JavaConverters._ import scala.language.{existentials, implicitConversions} @@ -40,6 +41,7 @@ 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.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources.{RateStreamProvider, TextSocketSourceProvider} @@ -91,8 +93,13 @@ case class DataSource( case class SourceInfo(name: String, schema: StructType, partitionColumns: Seq[String]) - lazy val providingClass: Class[_] = - DataSource.lookupDataSource(className, sparkSession.sessionState.conf) + lazy val providingClass: Class[_] = { + val cls = DataSource.lookupDataSource(className, sparkSession.sessionState.conf) + cls.newInstance() match { + case f: FileDataSourceV2 => f.fallBackFileFormat.getOrElse(cls) + case _ => cls + } + } lazy val sourceInfo: SourceInfo = sourceSchema() private val caseInsensitiveOptions = CaseInsensitiveMap(options) private val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis @@ -596,15 +603,10 @@ object DataSource extends Logging { provider: String, conf: SQLConf, paths: Seq[String] = Seq.empty): Class[_] = { - val disabledV2Readers = conf.disabledV2DataSourceReader.split(",") val provider1 = backwardCompatibilityMap.getOrElse(provider, provider) match { case name if name.equalsIgnoreCase("orc") && conf.getConf(SQLConf.ORC_IMPLEMENTATION) == "native" => - if (disabledV2Readers.contains("orc") || paths.length == 0) { - classOf[OrcFileFormat].getCanonicalName - } else { - classOf[OrcDataSourceV2].getCanonicalName - } + classOf[OrcDataSourceV2].getCanonicalName case name if name.equalsIgnoreCase("orc") && conf.getConf(SQLConf.ORC_IMPLEMENTATION) == "hive" => "org.apache.spark.sql.hive.orc.OrcFileFormat" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 16ba4cb546e5..c6fbbe64f3f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.datasources import java.util.Locale import java.util.concurrent.Callable +import javax.activation.FileDataSource import org.apache.hadoop.fs.Path @@ -38,7 +39,7 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{RowDataSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileDataSourceV2} import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ @@ -217,23 +218,22 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast } /** - * Replaces [[OrcDataSourceV2]] with [[DataSource]] if parent node is [[InsertIntoTable]]. - * This is because [[OrcDataSourceV2]] doesn't support writing data yet. + * Replaces [[FileDataSourceV2]] with [[DataSource]] if parent node is [[InsertIntoTable]]. * @param sparkSession */ -class FallBackToOrcV1(sparkSession: SparkSession) extends Rule[LogicalPlan] { - private def convertToOrcV1(v2Relation: DataSourceV2Relation): LogicalPlan = { - val v1 = DataSource.apply( - sparkSession = sparkSession, - paths = v2Relation.v2Options.paths(), - userSpecifiedSchema = v2Relation.userSpecifiedSchema, - className = classOf[OrcFileFormat].getCanonicalName, - options = v2Relation.options).resolveRelation() - LogicalRelation(v1) - } +class FallBackFileDataSourceToV1(sparkSession: SparkSession) extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan transform { case i @InsertIntoTable(d: DataSourceV2Relation, _, _, _, _) - if d.source.isInstanceOf[OrcDataSourceV2] => i.copy(table = convertToOrcV1(d)) + if d.source.isInstanceOf[FileDataSourceV2] => + val cls = d.source.asInstanceOf[FileDataSourceV2].fallBackFileFormat + assert(cls.isDefined, "File data source V2 doesn't support catalog yet.") + val v1 = DataSource.apply( + sparkSession = sparkSession, + paths = d.v2Options.paths(), + userSpecifiedSchema = d.userSpecifiedSchema, + className = cls.get.getCanonicalName, + options = d.options).resolveRelation() + i.copy(table = LogicalRelation(v1)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala new file mode 100644 index 000000000000..63582b5d7c0a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala @@ -0,0 +1,24 @@ +/* + * 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.v2 + +import org.apache.spark.sql.sources.v2.{DataSourceV2, ReadSupport, ReadSupportWithSchema} + +abstract class FileDataSourceV2 extends DataSourceV2 { + def fallBackFileFormat: Option[Class[_]] = None + def shortName(): String +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala index 2cc0f9b34668..57997a2d4419 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala @@ -32,15 +32,15 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Expression, JoinedRow} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.datasources.orc.{OrcColumnarBatchReader, OrcDeserializer, OrcFilters, OrcUtils} -import org.apache.spark.sql.execution.datasources.v2.ColumnarBatchFileSourceReader +import org.apache.spark.sql.execution.datasources.orc._ +import org.apache.spark.sql.execution.datasources.v2.{ColumnarBatchFileSourceReader, FileDataSourceV2} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema} import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.types.{AtomicType, StructType} import org.apache.spark.util.SerializableConfiguration -class OrcDataSourceV2 extends DataSourceV2 with ReadSupport with ReadSupportWithSchema { +class OrcDataSourceV2 extends FileDataSourceV2 with ReadSupport with ReadSupportWithSchema { override def createReader(options: DataSourceOptions): DataSourceReader = { new OrcDataSourceReader(options, None) } @@ -48,6 +48,10 @@ class OrcDataSourceV2 extends DataSourceV2 with ReadSupport with ReadSupportWith override def createReader(schema: StructType, options: DataSourceOptions): DataSourceReader = { new OrcDataSourceReader(options, Some(schema)) } + + override def fallBackFileFormat: Option[Class[_]] = Some(classOf[OrcFileFormat]) + + override def shortName(): String = "orc" } case class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: Option[StructType]) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index 10f713c586c6..81beb9b1c304 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -158,7 +158,7 @@ abstract class BaseSessionStateBuilder( override val extendedResolutionRules: Seq[Rule[LogicalPlan]] = new FindDataSourceTable(session) +: new ResolveSQLOnFile(session) +: - new FallBackToOrcV1(session) +: + new FallBackFileDataSourceToV1(session) +: customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = From 6dc1f10662955414539e7e2abdbfd5df94a63bb3 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Mon, 23 Apr 2018 16:26:42 +0800 Subject: [PATCH 20/25] fix test failure --- .../scala/org/apache/spark/sql/execution/command/tables.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 44749190c79e..0da1c2630dfb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -40,6 +40,7 @@ import org.apache.spark.sql.execution.datasources.{DataSource, PartitioningUtils import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.json.JsonFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.sql.util.SchemaUtils @@ -236,7 +237,7 @@ case class AlterTableAddColumnsCommand( // TextFileFormat only default to one column "value" // Hive type is already considered as hive serde table, so the logic will not // come in here. - case _: JsonFileFormat | _: CSVFileFormat | _: ParquetFileFormat => + case _: JsonFileFormat | _: CSVFileFormat | _: ParquetFileFormat | _: OrcDataSourceV2 => case s if s.getClass.getCanonicalName.endsWith("OrcFileFormat") => case s => throw new AnalysisException( From eee5c94af7f0338575f06aa6678dd3b63516570a Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Mon, 23 Apr 2018 19:24:22 +0800 Subject: [PATCH 21/25] fix test failure --- .../org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index d93215fefb81..3402ed240f8b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -908,7 +908,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv } assert(e.getMessage.contains( "The format of the existing table default.appendOrcToParquet is `ParquetFileFormat`. " + - "It doesn't match the specified format `OrcFileFormat`")) + "It doesn't match the specified format")) } withTable("appendParquetToJson") { From edbe034510917f02a5a0be405bb4af9f6768850c Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Wed, 25 Apr 2018 17:36:41 +0800 Subject: [PATCH 22/25] add FileDataSourceV2 and fall back mechanism --- .../apache/spark/sql/internal/SQLConf.scala | 27 ++- .../apache/spark/sql/DataFrameReader.scala | 28 +-- .../apache/spark/sql/DataFrameWriter.scala | 57 ++++--- .../execution/datasources/DataSource.scala | 5 +- .../datasources/DataSourceStrategy.scala | 15 +- .../datasources/v2/FileDataSourceV2.scala | 23 ++- .../datasources/v2/FileReaderFactory.scala | 83 +++++++++ .../datasources/v2/orc/OrcDataSourceV2.scala | 2 +- .../orc/OrcPartitionDiscoverySuite.scala | 2 +- .../datasources/orc/OrcQuerySuite.scala | 2 +- .../v2/FileDataSourceV2FallBackSuite.scala | 160 ++++++++++++++++++ .../sql/hive/HiveSessionStateBuilder.scala | 1 + 12 files changed, 342 insertions(+), 63 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 1d7badb46800..d354b2320177 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1205,6 +1205,20 @@ object SQLConf { .timeConf(TimeUnit.MILLISECONDS) .createWithDefault(100) + val DISABLED_V2_FILE_DATA_SOURCE_READERS = buildConf("spark.sql.disabledV2FileDataSourceReaders") + .internal() + .doc("A comma-separated list of file data source short names for which DataSourceReader" + + " is disabled. Reads from these sources will fall back to the V1 sources") + .stringConf + .createWithDefault("") + + val DISABLED_V2_FILE_DATA_SOURCE_WRITERS = buildConf("spark.sql.disabledV2FileDataSourceWriters") + .internal() + .doc("A comma-separated list of file data source short names for which DataSourceWriter" + + " is disabled. Writes to these sources will fall back to the V1 FileFormat") + .stringConf + .createWithDefault("") + val DISABLED_V2_STREAMING_WRITERS = buildConf("spark.sql.streaming.disabledV2Writers") .internal() .doc("A comma-separated list of fully qualified data source register class names for which" + @@ -1229,13 +1243,6 @@ object SQLConf { .booleanConf .createWithDefault(true) - val DISABLED_V2_DATA_SOURCE_READERS = buildConf("spark.sql.disabledV2DataSourceReaders") - .internal() - .doc("A comma-separated list of data source short names for which " + - "DataSourceReader is disabled. Reads from these sources will fall back to the V1 sources") - .stringConf - .createWithDefault("") - object PartitionOverwriteMode extends Enumeration { val STATIC, DYNAMIC = Value } @@ -1613,13 +1620,15 @@ class SQLConf extends Serializable with Logging { def continuousStreamingExecutorPollIntervalMs: Long = getConf(CONTINUOUS_STREAMING_EXECUTOR_POLL_INTERVAL_MS) + def disabledV2FileDataSourceReader: String = getConf(DISABLED_V2_FILE_DATA_SOURCE_READERS) + + def disabledV2FileDataSourceWriter: String = getConf(DISABLED_V2_FILE_DATA_SOURCE_WRITERS) + def disabledV2StreamingWriters: String = getConf(DISABLED_V2_STREAMING_WRITERS) def disabledV2StreamingMicroBatchReaders: String = getConf(DISABLED_V2_STREAMING_MICROBATCH_READERS) - def disabledV2DataSourceReader: String = getConf(DISABLED_V2_DATA_SOURCE_READERS) - def concatBinaryAsString: Boolean = getConf(CONCAT_BINARY_AS_STRING) def eltOutputAsString: Boolean = getConf(ELT_OUTPUT_AS_STRING) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index f02871f204b9..55421a2ed9db 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -193,12 +193,17 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { val allPaths = (CaseInsensitiveMap(extraOptions.toMap).get("path") ++ paths).toSeq val cls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf, allPaths) if (classOf[DataSourceV2].isAssignableFrom(cls)) { - val disabledV2Readers = sparkSession.sessionState.conf.disabledV2DataSourceReader.split(",") val ds = cls.newInstance().asInstanceOf[DataSourceV2] - val fallBackToV1 = ds.isInstanceOf[FileDataSourceV2] && - disabledV2Readers.contains(ds.asInstanceOf[FileDataSourceV2].shortName) + + val (needToFallBackFileDataSourceV2, fallBackFileFormat) = ds match { + case f: FileDataSourceV2 => + val disabledV2Readers = + sparkSession.sessionState.conf.disabledV2FileDataSourceReader.split(",") + (disabledV2Readers.contains(f.shortName), f.fallBackFileFormat.getCanonicalName) + case _ => (false, source) + } val supportsRead = ds.isInstanceOf[ReadSupport] || ds.isInstanceOf[ReadSupportWithSchema] - if (supportsRead && !fallBackToV1) { + if (supportsRead && !needToFallBackFileDataSourceV2) { val sessionOptions = DataSourceV2Utils.extractSessionConfigs( ds = ds, conf = sparkSession.sessionState.conf) val pathsOption = { @@ -210,25 +215,24 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { ds, extraOptions.toMap ++ sessionOptions + pathsOption, userSpecifiedSchema = userSpecifiedSchema)) } else { - if (fallBackToV1) { - loadV1Source(ds.asInstanceOf[FileDataSourceV2].fallBackFileFormat, paths: _*) - } else { - loadV1Source(None, paths: _*) - } + // In the following cases, we fall back to loading with V1: + // 1. The data source implements v2, but has no v2 implementation for read path. + // 2. The v2 reader of the data source is configured as disabled. + loadV1Source(fallBackFileFormat, paths: _*) } } else { - loadV1Source(None, paths: _*) + loadV1Source(source, paths: _*) } } - private def loadV1Source(cls: Option[Class[_]], paths: String*) = { + private def loadV1Source(className: String, paths: String*) = { // Code path for data source v1. sparkSession.baseRelationToDataFrame( DataSource.apply( sparkSession, paths = paths, userSpecifiedSchema = userSpecifiedSchema, - className = cls.map(_.getCanonicalName).getOrElse(source), + className = className, options = extraOptions.toMap).resolveRelation()) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index deed13a3f6cc..488a14b01396 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{AnalysisBarrier, InsertIntoT import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, LogicalRelation} -import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Utils, FileDataSourceV2, FileSourceReader, WriteToDataSourceV2} +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Utils, FileDataSourceV2, WriteToDataSourceV2} import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.sources.v2._ import org.apache.spark.sql.types.StructType @@ -240,42 +240,47 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val cls = DataSource.lookupDataSource(source, df.sparkSession.sessionState.conf) if (classOf[DataSourceV2].isAssignableFrom(cls)) { val ds = cls.newInstance() - ds match { - case ws: WriteSupport => - val options = new DataSourceOptions((extraOptions ++ - DataSourceV2Utils.extractSessionConfigs( - ds = ds.asInstanceOf[DataSourceV2], - conf = df.sparkSession.sessionState.conf)).asJava) - // Using a timestamp and a random UUID to distinguish different writing jobs. This is good - // enough as there won't be tons of writing jobs created at the same second. - val jobId = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US) - .format(new Date()) + "-" + UUID.randomUUID() - val writer = ws.createWriter(jobId, df.logicalPlan.schema, mode, options) - if (writer.isPresent) { - runCommand(df.sparkSession, "save") { - WriteToDataSourceV2(writer.get(), df.logicalPlan) - } - } - + val (needToFallBackFileDataSourceV2, fallBackFileFormat) = ds match { case f: FileDataSourceV2 => - saveToV1Source(f.fallBackFileFormat) + val disabledV2Readers = + df.sparkSession.sessionState.conf.disabledV2FileDataSourceWriter.split(",") + (disabledV2Readers.contains(f.shortName), f.fallBackFileFormat.getCanonicalName) + case _ => (false, source) + } - // Streaming also uses the data source V2 API. So it may be that the data source implements - // v2, but has no v2 implementation for batch writes. In that case, we fall back to saving - // as though it's a V1 source. - case _ => saveToV1Source() + if (ds.isInstanceOf[WriteSupport] && !needToFallBackFileDataSourceV2) { + val options = new DataSourceOptions((extraOptions ++ + DataSourceV2Utils.extractSessionConfigs( + ds = ds.asInstanceOf[DataSourceV2], + conf = df.sparkSession.sessionState.conf)).asJava) + // Using a timestamp and a random UUID to distinguish different writing jobs. This is good + // enough as there won't be tons of writing jobs created at the same second. + val jobId = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US) + .format(new Date()) + "-" + UUID.randomUUID() + val writer = ds.asInstanceOf[WriteSupport] + .createWriter(jobId, df.logicalPlan.schema, mode, options) + if (writer.isPresent) { + runCommand(df.sparkSession, "save") { + WriteToDataSourceV2(writer.get(), df.logicalPlan) + } + } + } else { + // In the following cases, we fall back to saving with V1: + // 1. The data source implements v2, but has no v2 implementation for write path. + // 2. The v2 writer of the data source is configured as disabled. + saveToV1Source(fallBackFileFormat) } } else { - saveToV1Source() + saveToV1Source(source) } } - private def saveToV1Source(cls: Option[Class[_]] = None): Unit = { + private def saveToV1Source(className: String): Unit = { // Code path for data source v1. runCommand(df.sparkSession, "save") { DataSource( sparkSession = df.sparkSession, - className = cls.map(_.getCanonicalName).getOrElse(source), + className = className, partitionColumns = partitioningColumns.getOrElse(Nil), options = extraOptions.toMap).planForWriting(mode, AnalysisBarrier(df.logicalPlan)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index af0a4431330f..61e1a8da27b5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -95,8 +95,11 @@ case class DataSource( lazy val providingClass: Class[_] = { val cls = DataSource.lookupDataSource(className, sparkSession.sessionState.conf) + // Here `providingClass` is supposed to be V1 file format. Currently [[FileDataSourceV2]] + // doesn't support catalog, so creating tables with V2 file format still uses this code path. + // As a temporary hack to avoid failure, [[FileDataSourceV2]] is falled back to [[FileFormat]]. cls.newInstance() match { - case f: FileDataSourceV2 => f.fallBackFileFormat.getOrElse(cls) + case f: FileDataSourceV2 => f.fallBackFileFormat case _ => cls } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index c6fbbe64f3f4..748d94d33362 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -38,9 +38,7 @@ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{RowDataSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.command._ -import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileDataSourceV2} -import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ @@ -218,20 +216,21 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast } /** - * Replaces [[FileDataSourceV2]] with [[DataSource]] if parent node is [[InsertIntoTable]]. - * @param sparkSession + * Replace the V2 data source of table in [[InsertIntoTable]] to V1 [[FileFormat]]. + * E.g, with temporary view `t` using [[FileDataSourceV2]], inserting into view `t` fails + * since there is no correspoding physical plan. + * This is a temporary hack for making current data source V2 work. */ class FallBackFileDataSourceToV1(sparkSession: SparkSession) extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan transform { case i @InsertIntoTable(d: DataSourceV2Relation, _, _, _, _) if d.source.isInstanceOf[FileDataSourceV2] => - val cls = d.source.asInstanceOf[FileDataSourceV2].fallBackFileFormat - assert(cls.isDefined, "File data source V2 doesn't support catalog yet.") + val v1FileFormat = d.source.asInstanceOf[FileDataSourceV2].fallBackFileFormat val v1 = DataSource.apply( sparkSession = sparkSession, - paths = d.v2Options.paths(), + paths = Seq.empty, userSpecifiedSchema = d.userSpecifiedSchema, - className = cls.get.getCanonicalName, + className = v1FileFormat.getCanonicalName, options = d.options).resolveRelation() i.copy(table = LogicalRelation(v1)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala index 63582b5d7c0a..82218d5eeec2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala @@ -16,9 +16,24 @@ */ package org.apache.spark.sql.execution.datasources.v2 -import org.apache.spark.sql.sources.v2.{DataSourceV2, ReadSupport, ReadSupportWithSchema} +import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.sources.DataSourceRegister +import org.apache.spark.sql.sources.v2.DataSourceV2 +import org.apache.spark.sql.sources.v2.ReadSupport +import org.apache.spark.sql.sources.v2.WriteSupport -abstract class FileDataSourceV2 extends DataSourceV2 { - def fallBackFileFormat: Option[Class[_]] = None - def shortName(): String +/** + * A base interface for data source v2 implementations of the built-in file-based data sources. + */ +trait FileDataSourceV2 extends DataSourceV2 with DataSourceRegister { + /** + * Returns a V1 [[FileFormat]] class of the same file data source. + * This is a solution for the following cases: + * 1. File datasource V2 might be implemented partially during migration. + * E.g. if [[ReadSupport]] is implemented while [[WriteSupport]] is not, + * write path should fall back to V1 implementation. + * 2. File datasource V2 implementations cause regression. + * 3. Catalog support is required, which is still under development for data source V2. + */ + def fallBackFileFormat: Class[_ <: FileFormat] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileReaderFactory.scala index 62430eefd920..4d20a9263e36 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileReaderFactory.scala @@ -18,9 +18,12 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.TaskContext +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.rdd.InputFileBlockHolder import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.{FilePartition, FilePartitionUtil, PartitionedFile} import org.apache.spark.sql.sources.v2.reader.{DataReader, DataReaderFactory} +import org.apache.spark.sql.vectorized.ColumnarBatch case class FileReaderFactory[T]( file: FilePartition, @@ -48,3 +51,83 @@ case class InternalRowDataReader[T](iter: Iterator[InternalRow]) override def close(): Unit = {} } + +abstract class FileDataReader[T] extends DataReader[T] { + def file: PartitionedFile +} + +case class FileReaderFoobar[T]( + context: TaskContext, + readers: Iterator[FileDataReader[T]], + ignoreMissingFiles: Boolean, + ignoreCorruptFiles: Boolean) + extends DataReader[T] { + private val inputMetrics = context.taskMetrics().inputMetrics + private val existingBytesRead = inputMetrics.bytesRead + + // Find a function that will return the FileSystem bytes read by this thread. Do this before + // apply readFunction, because it might read some bytes. + private val getBytesReadCallback = + SparkHadoopUtil.get.getFSBytesReadOnThreadCallback() + + // We get our input bytes from thread-local Hadoop FileSystem statistics. + // If we do a coalesce, however, we are likely to compute multiple partitions in the same + // task and in the same thread, in which case we need to avoid override values written by + // previous partitions (SPARK-13071). + private def updateBytesRead(): Unit = { + inputMetrics.setBytesRead(existingBytesRead + getBytesReadCallback()) + } + + // If we can't get the bytes read from the FS stats, fall back to the file size, + // which may be inaccurate. + private def updateBytesReadWithFileSize(): Unit = { + if (currentFile != null) { + inputMetrics.incBytesRead(currentFile.file.length) + } + } + + private[this] var currentFile: FileDataReader[T] = null + + override def next(): Boolean = { + // Kill the task in case it has been marked as killed. This logic is from + // InterruptibleIterator, but we inline it here instead of wrapping the iterator in order + // to avoid performance overhead. + context.killTaskIfInterrupted() + + if (currentFile != null && currentFile.next()) { + return true + } + while (readers.hasNext) { + currentFile = readers.next() + if (currentFile.next()) { + return true + } + } + + return false + } + + override def get(): T = { + val nextElement = currentFile.get() + // TODO: we should have a better separation of row based and batch based scan, so that we + // don't need to run this `if` for every record. + if (nextElement.isInstanceOf[ColumnarBatch]) { + inputMetrics.incRecordsRead(nextElement.asInstanceOf[ColumnarBatch].numRows()) + } else { + inputMetrics.incRecordsRead(1) + } + if (inputMetrics.recordsRead % SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS == 0) { + updateBytesRead() + } + nextElement + } + + override def close(): Unit = { + updateBytesRead() + updateBytesReadWithFileSize() + InputFileBlockHolder.unset() + if (currentFile != null) { + currentFile.close() + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala index 57997a2d4419..b68960249f63 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala @@ -49,7 +49,7 @@ class OrcDataSourceV2 extends FileDataSourceV2 with ReadSupport with ReadSupport new OrcDataSourceReader(options, Some(schema)) } - override def fallBackFileFormat: Option[Class[_]] = Some(classOf[OrcFileFormat]) + override def fallBackFileFormat: Class[_ <: FileFormat] = classOf[OrcFileFormat] override def shortName(): String = "orc" } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala index a955647281fc..15e6500b894d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala @@ -232,5 +232,5 @@ class OrcPartitionDiscoverySuite extends OrcPartitionDiscoveryTest with SharedSQ class OrcV1PartitionDiscoverySuite extends OrcPartitionDiscoveryTest with SharedSQLContext { override protected def sparkConf: SparkConf = - super.sparkConf.set(SQLConf.DISABLED_V2_DATA_SOURCE_READERS, "orc") + super.sparkConf.set(SQLConf.DISABLED_V2_FILE_DATA_SOURCE_READERS, "orc") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala index 3efb977b9c8f..3a7a24c6f055 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala @@ -659,5 +659,5 @@ class OrcQuerySuite extends OrcQueryTest with SharedSQLContext { class OrcV1QuerySuite extends OrcQuerySuite { override protected def sparkConf: SparkConf = - super.sparkConf.set(SQLConf.DISABLED_V2_DATA_SOURCE_READERS, "orc") + super.sparkConf.set(SQLConf.DISABLED_V2_FILE_DATA_SOURCE_READERS, "orc") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala new file mode 100644 index 000000000000..de7db3004f45 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala @@ -0,0 +1,160 @@ +/* + * 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.sources.v2 + +import java.util.Optional + +import org.apache.spark.sql.{AnalysisException, QueryTest, SaveMode} +import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, ParquetTest} +import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.reader.DataSourceReader +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.StructType + +class DummyReadOnlyFileDataSourceV2 extends FileDataSourceV2 with ReadSupport { + override def createReader(options: DataSourceOptions): DataSourceReader = { + throw new AnalysisException("Dummy file reader") + } + + override def fallBackFileFormat: Class[_ <: FileFormat] = classOf[ParquetFileFormat] + + override def shortName(): String = "parquet" +} + +class DummyWriteOnlyFileDataSourceV2 extends FileDataSourceV2 with WriteSupport { + override def createWriter( + jobId: String, + schema: StructType, + mode: SaveMode, + options: DataSourceOptions): Optional[DataSourceWriter] = { + throw new AnalysisException("Dummy file writer") + } + + override def fallBackFileFormat: Class[_ <: FileFormat] = classOf[ParquetFileFormat] + + override def shortName(): String = "parquet" +} + +class SimpleFileDataSourceV2 extends SimpleDataSourceV2 with FileDataSourceV2 { + override def fallBackFileFormat: Class[_ <: FileFormat] = classOf[ParquetFileFormat] + + override def shortName(): String = "parquet" +} + +class FileDataSourceV2FallBackSuite extends QueryTest with ParquetTest with SharedSQLContext { + import testImplicits._ + + private val dummyParquetReaderV2 = classOf[DummyReadOnlyFileDataSourceV2].getName + private val dummyParquetWriterV2 = classOf[DummyWriteOnlyFileDataSourceV2].getName + private val simpleFileDataSourceV2 = classOf[SimpleFileDataSourceV2].getName + + test("Fall back to v1 when writing to file with read only FileDataSourceV2") { + val df = spark.range(10).toDF() + withTempPath { file => + val path = file.getCanonicalPath + // Writing file should fall back to v1 and succeed. + df.write.format(dummyParquetReaderV2).save(path) + + // Validate write result with [[ParquetFileFormat]]. + checkAnswer(spark.read.parquet(path), df) + + // Dummy File reader should fail as expected. + val exception = intercept[AnalysisException] { + spark.read.format(dummyParquetReaderV2).load(path) + } + assert(exception.message.equals("Dummy file reader")) + } + } + + test("Fall back to v1 when reading file with write only FileDataSourceV2") { + val df = spark.range(10).toDF() + withTempPath { file => + val path = file.getCanonicalPath + + // Dummy File writer should fail as expected. + val exception = intercept[AnalysisException] { + df.write.format(dummyParquetWriterV2).save(path) + } + assert(exception.message.equals("Dummy file writer")) + + df.write.parquet(path) + // Reading file should fall back to v1 and succeed. + checkAnswer(spark.read.format(dummyParquetWriterV2).load(path), df) + } + } + + test("Fall back read path to v1 with configuration DISABLED_V2_FILE_DATA_SOURCE_READERS") { + val df = spark.range(10).toDF() + withTempPath { file => + val path = file.getCanonicalPath + df.write.parquet(path) + withSQLConf(SQLConf.DISABLED_V2_FILE_DATA_SOURCE_READERS.key -> "foo,parquet,bar") { + // Reading file should fall back to v1 and succeed. + checkAnswer(spark.read.format(dummyParquetReaderV2).load(path), df) + } + + withSQLConf(SQLConf.DISABLED_V2_FILE_DATA_SOURCE_READERS.key -> "foo,bar") { + // Dummy File reader should fail as DISABLED_V2_FILE_DATA_SOURCE_READERS doesn't include it. + val exception = intercept[AnalysisException] { + spark.read.format(dummyParquetReaderV2).load(path) + } + assert(exception.message.equals("Dummy file reader")) + } + } + } + + test("Fall back write path to v1 with configuration DISABLED_V2_FILE_DATA_SOURCE_READERS") { + val df = spark.range(10).toDF() + withTempPath { file => + val path = file.getCanonicalPath + + withSQLConf(SQLConf.DISABLED_V2_FILE_DATA_SOURCE_WRITERS.key -> "foo,bar") { + // Dummy File writer should fail as expected. + val exception = intercept[AnalysisException] { + df.write.format(dummyParquetWriterV2).save(path) + } + assert(exception.message.equals("Dummy file writer")) + } + + withSQLConf(SQLConf.DISABLED_V2_FILE_DATA_SOURCE_WRITERS.key -> "foo,parquet,bar") { + // Writing file should fall back to v1 and succeed. + df.write.format(dummyParquetWriterV2).save(path) + } + + checkAnswer(spark.read.format(dummyParquetWriterV2).load(path), df) + } + } + + test("InsertIntoTable: Fall back to V1") { + val df1 = (100 until 105).map(i => (i, -i)).toDF("i", "j") + val df2 = (5 until 10).map(i => (i, -i)).toDF("i", "j") + withTempPath { file => + val path = file.getCanonicalPath + withTempView("tmp", "tbl") { + df1.createOrReplaceTempView("tmp") + df2.write.parquet(path) + // Create temporary view with FileDataSourceV2 + spark.read.format(simpleFileDataSourceV2).load(path).createOrReplaceTempView("tbl") + sql("INSERT INTO TABLE tbl SELECT * FROM tmp") + checkAnswer(spark.read.parquet(path), df1.union(df2)) + } + } + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index 2882672f327c..c26dca239de3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -71,6 +71,7 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session new ResolveHiveSerdeTable(session) +: new FindDataSourceTable(session) +: new ResolveSQLOnFile(session) +: + new FallBackFileDataSourceToV1(session) +: customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = From aa80f6f8c981bfaf2ff99fb3477173607c3d6165 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Wed, 25 Apr 2018 21:57:26 +0800 Subject: [PATCH 23/25] refactor FileReaderFactory and FileSourceReader --- .../datasources/DataSourceStrategy.scala | 4 +- .../datasources/v2/EmptyDataReader.scala | 34 ++++ .../datasources/v2/FileReaderFactory.scala | 100 +++++----- .../datasources/v2/FileSourceReader.scala | 21 ++- .../datasources/v2/RecordDataReader.scala | 40 ++++ .../datasources/v2/orc/OrcDataSourceV2.scala | 178 ++++++++++-------- 6 files changed, 241 insertions(+), 136 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyDataReader.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RecordDataReader.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 748d94d33362..e4fdd12316a0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -228,10 +228,10 @@ class FallBackFileDataSourceToV1(sparkSession: SparkSession) extends Rule[Logica val v1FileFormat = d.source.asInstanceOf[FileDataSourceV2].fallBackFileFormat val v1 = DataSource.apply( sparkSession = sparkSession, - paths = Seq.empty, + paths = d.v2Options.paths(), userSpecifiedSchema = d.userSpecifiedSchema, className = v1FileFormat.getCanonicalName, - options = d.options).resolveRelation() + options = d.options - "path").resolveRelation() i.copy(table = LogicalRelation(v1)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyDataReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyDataReader.scala new file mode 100644 index 000000000000..59e0da89cc7d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyDataReader.scala @@ -0,0 +1,34 @@ +/* + * 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.v2 + +import java.io.IOException + +import org.apache.spark.sql.sources.v2.reader.DataReader + +/** + * A [[DataReader]] with empty output. + */ +class EmptyDataReader[T] extends DataReader[T] { + override def next(): Boolean = false + + override def get(): T = + throw new IOException("No records should be returned from EmptyDataReader") + + override def close(): Unit = {} +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileReaderFactory.scala index 4d20a9263e36..267cb35aec07 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileReaderFactory.scala @@ -27,15 +27,14 @@ import org.apache.spark.sql.vectorized.ColumnarBatch case class FileReaderFactory[T]( file: FilePartition, - readFunction: (PartitionedFile) => Iterator[InternalRow], + readFunction: (PartitionedFile) => DataReader[T], ignoreCorruptFiles: Boolean = false, ignoreMissingFiles: Boolean = false) extends DataReaderFactory[T] { override def createDataReader(): DataReader[T] = { val taskContext = TaskContext.get() - val iter = FilePartitionUtil.compute(file, taskContext, readFunction, - ignoreCorruptFiles, ignoreMissingFiles) - InternalRowDataReader[T](iter) + val iter = file.files.iterator.map(f => PartitionedFileDataReader(f, readFunction(f))) + FileDataReader(taskContext, iter) } override def preferredLocations(): Array[String] = { @@ -43,50 +42,61 @@ case class FileReaderFactory[T]( } } -case class InternalRowDataReader[T](iter: Iterator[InternalRow]) - extends DataReader[T] { - override def next(): Boolean = iter.hasNext +case class PartitionedFileDataReader[T]( + file: PartitionedFile, + reader: DataReader[T]) extends DataReader[T] { + override def next(): Boolean = reader.next() - override def get(): T = iter.next().asInstanceOf[T] + override def get(): T = reader.get() - override def close(): Unit = {} + override def close(): Unit = reader.close() } -abstract class FileDataReader[T] extends DataReader[T] { - def file: PartitionedFile -} - -case class FileReaderFoobar[T]( +case class FileDataReader[T]( context: TaskContext, - readers: Iterator[FileDataReader[T]], - ignoreMissingFiles: Boolean, - ignoreCorruptFiles: Boolean) - extends DataReader[T] { - private val inputMetrics = context.taskMetrics().inputMetrics - private val existingBytesRead = inputMetrics.bytesRead - - // Find a function that will return the FileSystem bytes read by this thread. Do this before - // apply readFunction, because it might read some bytes. - private val getBytesReadCallback = - SparkHadoopUtil.get.getFSBytesReadOnThreadCallback() - - // We get our input bytes from thread-local Hadoop FileSystem statistics. - // If we do a coalesce, however, we are likely to compute multiple partitions in the same - // task and in the same thread, in which case we need to avoid override values written by - // previous partitions (SPARK-13071). - private def updateBytesRead(): Unit = { - inputMetrics.setBytesRead(existingBytesRead + getBytesReadCallback()) + readers: Iterator[PartitionedFileDataReader[T]]) extends DataReader[T] { + private val inputMetrics = context.taskMetrics().inputMetrics + private val existingBytesRead = inputMetrics.bytesRead + + // Find a function that will return the FileSystem bytes read by this thread. Do this before + // apply readFunction, because it might read some bytes. + private val getBytesReadCallback = + SparkHadoopUtil.get.getFSBytesReadOnThreadCallback() + + // We get our input bytes from thread-local Hadoop FileSystem statistics. + // If we do a coalesce, however, we are likely to compute multiple partitions in the same + // task and in the same thread, in which case we need to avoid override values written by + // previous partitions (SPARK-13071). + private def updateBytesRead(): Unit = { + inputMetrics.setBytesRead(existingBytesRead + getBytesReadCallback()) + } + + // If we can't get the bytes read from the FS stats, fall back to the file size, + // which may be inaccurate. + private def updateBytesReadWithFileSize(): Unit = { + if (currentFile != null) { + inputMetrics.incBytesRead(currentFile.file.length) } + } + + private[this] var currentFile: PartitionedFileDataReader[T] = null - // If we can't get the bytes read from the FS stats, fall back to the file size, - // which may be inaccurate. - private def updateBytesReadWithFileSize(): Unit = { - if (currentFile != null) { - inputMetrics.incBytesRead(currentFile.file.length) + private def hasNext(): Boolean = { + if (currentFile == null) { + if (readers.hasNext) { + currentFile = readers.next() + } else { + return false } } - - private[this] var currentFile: FileDataReader[T] = null + if (currentFile.next()) { + return true + } else { + close() + currentFile = null + } + hasNext() + } override def next(): Boolean = { // Kill the task in case it has been marked as killed. This logic is from @@ -94,17 +104,7 @@ case class FileReaderFoobar[T]( // to avoid performance overhead. context.killTaskIfInterrupted() - if (currentFile != null && currentFile.next()) { - return true - } - while (readers.hasNext) { - currentFile = readers.next() - if (currentFile.next()) { - return true - } - } - - return false + hasNext() } override def get(): T = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala index f87fb656cb00..73e82499b334 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala @@ -33,13 +33,11 @@ import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch -trait FileSourceReader extends DataSourceReader +abstract class FileSourceReader(options: DataSourceOptions, userSpecifiedSchema: Option[StructType]) + extends DataSourceReader with SupportsScanUnsafeRow with SupportsPushDownRequiredColumns with SupportsPushDownCatalystFilters { - def options: DataSourceOptions - def userSpecifiedSchema: Option[StructType] - /** * When possible, this method should return the schema of the given `files`. When the format * does not support inference, or no valid files are given should return None. In these cases @@ -47,8 +45,6 @@ trait FileSourceReader extends DataSourceReader */ def inferSchema(files: Seq[FileStatus]): Option[StructType] - def readFunction: PartitionedFile => Iterator[InternalRow] - /** * Returns whether a file with `path` could be split or not. */ @@ -56,6 +52,8 @@ trait FileSourceReader extends DataSourceReader false } + def unsafeRowDataReader: PartitionedFile => DataReader[UnsafeRow] + protected val sparkSession = SparkSession.getActiveSession .getOrElse(SparkSession.getDefaultSession.get) protected val hadoopConf = @@ -129,17 +127,22 @@ trait FileSourceReader extends DataSourceReader override def createUnsafeRowReaderFactories: JList[DataReaderFactory[UnsafeRow]] = { partitions.map { filePartition => - new FileReaderFactory[UnsafeRow](filePartition, readFunction, + new FileReaderFactory[UnsafeRow](filePartition, unsafeRowDataReader, ignoreCorruptFiles, ignoreMissingFiles) .asInstanceOf[DataReaderFactory[UnsafeRow]] }.asJava } } -trait ColumnarBatchFileSourceReader extends FileSourceReader with SupportsScanColumnarBatch { +abstract class ColumnarBatchFileSourceReader( + options: DataSourceOptions, + userSpecifiedSchema: Option[StructType]) + extends FileSourceReader(options: DataSourceOptions, userSpecifiedSchema: Option[StructType]) + with SupportsScanColumnarBatch { + def columnarBatchDataReader: PartitionedFile => DataReader[ColumnarBatch] override def createBatchDataReaderFactories(): JList[DataReaderFactory[ColumnarBatch]] = { partitions.map { filePartition => - new FileReaderFactory[ColumnarBatch](filePartition, readFunction, + new FileReaderFactory[ColumnarBatch](filePartition, columnarBatchDataReader, ignoreCorruptFiles, ignoreMissingFiles) .asInstanceOf[DataReaderFactory[ColumnarBatch]] }.asJava diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RecordDataReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RecordDataReader.scala new file mode 100644 index 000000000000..c4a9585b3a32 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RecordDataReader.scala @@ -0,0 +1,40 @@ +/* + * 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.v2 + +import org.apache.hadoop.mapreduce.RecordReader + +import org.apache.spark.sql.sources.v2.reader.DataReader + +class RecordDataReader[T](private[this] var rowReader: RecordReader[_, T]) extends DataReader[T] { + override def next(): Boolean = rowReader.nextKeyValue() + + override def get(): T = rowReader.getCurrentValue + + override def close(): Unit = rowReader.close() +} + +class RecordDataReadeWithProject[X, T]( + private[this] var rowReader: RecordReader[_, X], + project: X => T) extends DataReader[T] { + override def next(): Boolean = rowReader.nextKeyValue() + + override def get(): T = project(rowReader.getCurrentValue) + + override def close(): Unit = rowReader.close() +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala index b68960249f63..8188ff3be880 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala @@ -28,16 +28,16 @@ import org.apache.orc.mapred.OrcStruct import org.apache.orc.mapreduce.OrcInputFormat import org.apache.spark.TaskContext -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Expression, JoinedRow} +import org.apache.spark.sql.catalyst.expressions.{Expression, JoinedRow, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.orc._ -import org.apache.spark.sql.execution.datasources.v2.{ColumnarBatchFileSourceReader, FileDataSourceV2} +import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, ReadSupportWithSchema} +import org.apache.spark.sql.sources.v2.{DataSourceOptions, ReadSupport, ReadSupportWithSchema} import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.types.{AtomicType, StructType} +import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration class OrcDataSourceV2 extends FileDataSourceV2 with ReadSupport with ReadSupportWithSchema { @@ -55,13 +55,46 @@ class OrcDataSourceV2 extends FileDataSourceV2 with ReadSupport with ReadSupport } case class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: Option[StructType]) - extends ColumnarBatchFileSourceReader { + extends ColumnarBatchFileSourceReader(options: DataSourceOptions, + userSpecifiedSchema: Option[StructType]) { override def inferSchema(files: Seq[FileStatus]): Option[StructType] = { OrcUtils.readSchema(sparkSession, files) } - override def readFunction: PartitionedFile => Iterator[InternalRow] = { + override def pushCatalystFilters(filters: Array[Expression]): Array[Expression] = { + val partitionColumnNames = partitionSchema.toAttributes.map(_.name).toSet + val (partitionKeyFilters, otherFilters) = filters.partition { + _.references.map(_.name).toSet.subsetOf(partitionColumnNames) + } + this.partitionFilters = partitionKeyFilters + pushedFiltersArray = partitionKeyFilters + if (sqlConf.orcFilterPushDown) { + val dataFilters = otherFilters.map { f => + (DataSourceStrategy.translateFilter(f), f) + }.collect { case (optionalFilter, catalystFilter) if optionalFilter.isDefined => + (optionalFilter.get, catalystFilter) + }.toMap + val pushedDataFilters = + OrcFilters.convertibleFilters(fullSchema, dataFilters.keys.toSeq).map(dataFilters).toArray + pushedFiltersArray ++= pushedDataFilters + OrcFilters.createFilter(fullSchema, dataFilters.keys.toSeq).foreach { f => + OrcInputFormat.setSearchArgument(hadoopConf, f, fullSchema.fieldNames) + } + } + otherFilters + } + + override def enableBatchRead(): Boolean = { + val schema = readSchema() + sqlConf.orcVectorizedReaderEnabled && sqlConf.wholeStageEnabled && + schema.length <= sqlConf.wholeStageMaxNumFields && + schema.forall(_.dataType.isInstanceOf[AtomicType]) + } + + override def isSplitable(path: Path): Boolean = true + + override def columnarBatchDataReader: (PartitionedFile) => DataReader[ColumnarBatch] = { val capacity = sqlConf.orcVectorizedReaderBatchSize val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled val copyToSpark = sqlConf.getConf(SQLConf.ORC_COPY_BATCH_TO_SPARK) @@ -71,7 +104,6 @@ case class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: val partitionSchema = this.partitionSchema val broadcastedConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) - val enableVectorizedReader = enableBatchRead() (file: PartitionedFile) => { val conf = broadcastedConf.value.value @@ -85,7 +117,7 @@ case class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: isCaseSensitive, dataSchema, readSchema, reader, conf) if (requestedColIdsOrEmptyFile.isEmpty) { - Iterator.empty + new EmptyDataReader } else { val requestedColIds = requestedColIdsOrEmptyFile.get assert(requestedColIds.length == readSchema.length, @@ -99,78 +131,74 @@ case class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: val taskAttemptContext = new TaskAttemptContextImpl(taskConf, attemptId) val taskContext = Option(TaskContext.get()) - if (enableVectorizedReader) { - val batchReader = new OrcColumnarBatchReader( - enableOffHeapColumnVector && taskContext.isDefined, copyToSpark, capacity) - // SPARK-23399 Register a task completion listener first to call `close()` in all cases. - // There is a possibility that `initialize` and `initBatch` hit some errors (like OOM) - // after opening a file. - val iter = new RecordReaderIterator(batchReader) - Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => iter.close())) - batchReader.initialize(fileSplit, taskAttemptContext) - val partitionColIds = PartitioningUtils.requestedPartitionColumnIds( - partitionSchema, readSchema, isCaseSensitive) - - batchReader.initBatch( - reader.getSchema, - readSchema.fields, - requestedColIds, - partitionColIds, - file.partitionValues) - iter.asInstanceOf[Iterator[InternalRow]] - } else { - val requiredDataSchema = - PartitioningUtils.subtractSchema(readSchema, partitionSchema, isCaseSensitive) - val orcRecordReader = new OrcInputFormat[OrcStruct] - .createRecordReader(fileSplit, taskAttemptContext) - val iter = new RecordReaderIterator[OrcStruct](orcRecordReader) - taskContext.foreach(_.addTaskCompletionListener(_ => iter.close())) - - val fullSchema = requiredDataSchema.toAttributes ++ partitionSchema.toAttributes - val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) - val deserializer = new OrcDeserializer(dataSchema, requiredDataSchema, requestedColIds) - - if (partitionSchema.length == 0) { - iter.map(value => unsafeProjection(deserializer.deserialize(value))) - } else { - val joinedRow = new JoinedRow() - iter.map(value => - unsafeProjection(joinedRow(deserializer.deserialize(value), file.partitionValues))) - } - } + val batchReader = new OrcColumnarBatchReader( + enableOffHeapColumnVector && taskContext.isDefined, copyToSpark, capacity) + batchReader.initialize(fileSplit, taskAttemptContext) + val partitionColIds = PartitioningUtils.requestedPartitionColumnIds( + partitionSchema, readSchema, isCaseSensitive) + + batchReader.initBatch( + reader.getSchema, + readSchema.fields, + requestedColIds, + partitionColIds, + file.partitionValues) + new RecordDataReader(batchReader) } } } - override def pushCatalystFilters(filters: Array[Expression]): Array[Expression] = { - val partitionColumnNames = partitionSchema.toAttributes.map(_.name).toSet - val (partitionKeyFilters, otherFilters) = filters.partition { - _.references.map(_.name).toSet.subsetOf(partitionColumnNames) - } - this.partitionFilters = partitionKeyFilters - pushedFiltersArray = partitionKeyFilters - if (sqlConf.orcFilterPushDown) { - val dataFilters = otherFilters.map { f => - (DataSourceStrategy.translateFilter(f), f) - }.collect { case (optionalFilter, catalystFilter) if optionalFilter.isDefined => - (optionalFilter.get, catalystFilter) - }.toMap - val pushedDataFilters = - OrcFilters.convertibleFilters(fullSchema, dataFilters.keys.toSeq).map(dataFilters).toArray - pushedFiltersArray ++= pushedDataFilters - OrcFilters.createFilter(fullSchema, dataFilters.keys.toSeq).foreach { f => - OrcInputFormat.setSearchArgument(hadoopConf, f, fullSchema.fieldNames) + override def unsafeRowDataReader: (PartitionedFile) => DataReader[UnsafeRow] = { + val isCaseSensitive = this.isCaseSensitive + val dataSchema = this.dataSchema + val readSchema = this.readSchema() + val partitionSchema = this.partitionSchema + val broadcastedConf = + sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) + (file: PartitionedFile) => { + val conf = broadcastedConf.value.value + + val filePath = new Path(new URI(file.filePath)) + + val fs = filePath.getFileSystem(conf) + val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) + val reader = OrcFile.createReader(filePath, readerOptions) + + val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( + isCaseSensitive, dataSchema, readSchema, reader, conf) + + if (requestedColIdsOrEmptyFile.isEmpty) { + new EmptyDataReader[UnsafeRow] + } else { + val requestedColIds = requestedColIdsOrEmptyFile.get + assert(requestedColIds.length == readSchema.length, + "[BUG] requested column IDs do not match required schema") + val taskConf = new Configuration(conf) + taskConf.set(OrcConf.INCLUDE_COLUMNS.getAttribute, + requestedColIds.filter(_ != -1).sorted.mkString(",")) + + val fileSplit = new FileSplit(filePath, file.start, file.length, Array.empty) + val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) + val taskAttemptContext = new TaskAttemptContextImpl(taskConf, attemptId) + + val requiredDataSchema = + PartitioningUtils.subtractSchema(readSchema, partitionSchema, isCaseSensitive) + val orcRecordReader = new OrcInputFormat[OrcStruct] + .createRecordReader(fileSplit, taskAttemptContext) + + val fullSchema = requiredDataSchema.toAttributes ++ partitionSchema.toAttributes + val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) + val deserializer = new OrcDeserializer(dataSchema, requiredDataSchema, requestedColIds) + + val projection = if (partitionSchema.length == 0) { + (value: OrcStruct) => unsafeProjection(deserializer.deserialize(value)) + } else { + val joinedRow = new JoinedRow() + (value: OrcStruct) => + unsafeProjection(joinedRow(deserializer.deserialize(value), file.partitionValues)) + } + new RecordDataReadeWithProject(orcRecordReader, projection) } } - otherFilters - } - - override def enableBatchRead(): Boolean = { - val schema = readSchema() - sqlConf.orcVectorizedReaderEnabled && sqlConf.wholeStageEnabled && - schema.length <= sqlConf.wholeStageMaxNumFields && - schema.forall(_.dataType.isInstanceOf[AtomicType]) } - - override def isSplitable(path: Path): Boolean = true } From cca43212f880a86ff8731c1863a2c2c6fc05adf2 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Thu, 26 Apr 2018 00:42:36 +0800 Subject: [PATCH 24/25] handle ignoreCorruptFiles and ignoreMissingFiles --- .../datasources/v2/FileReaderFactory.scala | 30 ++++++++++++++++--- .../datasources/v2/FileSourceReader.scala | 10 +++++++ 2 files changed, 36 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileReaderFactory.scala index 267cb35aec07..fae416e48774 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileReaderFactory.scala @@ -17,10 +17,12 @@ package org.apache.spark.sql.execution.datasources.v2 +import java.io.{FileNotFoundException, IOException} + import org.apache.spark.TaskContext import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging import org.apache.spark.rdd.InputFileBlockHolder -import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.{FilePartition, FilePartitionUtil, PartitionedFile} import org.apache.spark.sql.sources.v2.reader.{DataReader, DataReaderFactory} import org.apache.spark.sql.vectorized.ColumnarBatch @@ -34,7 +36,7 @@ case class FileReaderFactory[T]( override def createDataReader(): DataReader[T] = { val taskContext = TaskContext.get() val iter = file.files.iterator.map(f => PartitionedFileDataReader(f, readFunction(f))) - FileDataReader(taskContext, iter) + FileDataReader(taskContext, iter, ignoreCorruptFiles, ignoreMissingFiles) } override def preferredLocations(): Array[String] = { @@ -54,7 +56,9 @@ case class PartitionedFileDataReader[T]( case class FileDataReader[T]( context: TaskContext, - readers: Iterator[PartitionedFileDataReader[T]]) extends DataReader[T] { + readers: Iterator[PartitionedFileDataReader[T]], + ignoreCorruptFiles: Boolean, + ignoreMissingFiles: Boolean) extends DataReader[T] with Logging { private val inputMetrics = context.taskMetrics().inputMetrics private val existingBytesRead = inputMetrics.bytesRead @@ -84,7 +88,25 @@ case class FileDataReader[T]( private def hasNext(): Boolean = { if (currentFile == null) { if (readers.hasNext) { - currentFile = readers.next() + if (ignoreMissingFiles || ignoreCorruptFiles) { + try { + currentFile = readers.next() + } catch { + case e: FileNotFoundException if ignoreMissingFiles => + logWarning(s"Skipped missing file: $currentFile", e) + currentFile = null + return false + // Throw FileNotFoundException even if `ignoreCorruptFiles` is true + case e: FileNotFoundException if !ignoreMissingFiles => throw e + case e @ (_: RuntimeException | _: IOException) if ignoreCorruptFiles => + logWarning( + s"Skipped the rest of the content in the corrupted file: $currentFile", e) + currentFile = null + return false + } + } else { + currentFile = readers.next() + } } else { return false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala index 73e82499b334..b861f89f2d9f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala @@ -52,6 +52,10 @@ abstract class FileSourceReader(options: DataSourceOptions, userSpecifiedSchema: false } + /** + * Returns a function that can be used to read a single file in as an [[DataReader]] of + * [[UnsafeRow]]. + */ def unsafeRowDataReader: PartitionedFile => DataReader[UnsafeRow] protected val sparkSession = SparkSession.getActiveSession @@ -59,6 +63,7 @@ abstract class FileSourceReader(options: DataSourceOptions, userSpecifiedSchema: protected val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(options.asMap().asScala.toMap) protected val sqlConf = sparkSession.sessionState.conf + protected val isCaseSensitive = sqlConf.caseSensitiveAnalysis protected val ignoreCorruptFiles = sqlConf.ignoreCorruptFiles protected val ignoreMissingFiles = sqlConf.ignoreMissingFiles @@ -139,7 +144,12 @@ abstract class ColumnarBatchFileSourceReader( userSpecifiedSchema: Option[StructType]) extends FileSourceReader(options: DataSourceOptions, userSpecifiedSchema: Option[StructType]) with SupportsScanColumnarBatch { + /** + * Returns a function that can be used to read a single file in as an [[DataReader]] of + * [[ColumnarBatch]]. + */ def columnarBatchDataReader: PartitionedFile => DataReader[ColumnarBatch] + override def createBatchDataReaderFactories(): JList[DataReaderFactory[ColumnarBatch]] = { partitions.map { filePartition => new FileReaderFactory[ColumnarBatch](filePartition, columnarBatchDataReader, From 67b1748c8b939a6b484bfc868fd311e381d7f8e0 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Mon, 14 May 2018 18:30:09 +0800 Subject: [PATCH 25/25] update to latest master --- ....scala => EmptyInputPartitionReader.scala} | 6 ++--- ...Factory.scala => FileInputPartition.scala} | 26 +++++++++---------- .../datasources/v2/FileSourceReader.scala | 21 +++++++-------- ...ader.scala => PartitionRecordReader.scala} | 9 ++++--- .../datasources/v2/orc/OrcDataSourceV2.scala | 13 +++++----- 5 files changed, 38 insertions(+), 37 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/{EmptyDataReader.scala => EmptyInputPartitionReader.scala} (85%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/{FileReaderFactory.scala => FileInputPartition.scala} (86%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/{RecordDataReader.scala => PartitionRecordReader.scala} (81%) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyDataReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyInputPartitionReader.scala similarity index 85% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyDataReader.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyInputPartitionReader.scala index 59e0da89cc7d..bf570a4ba07a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyDataReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyInputPartitionReader.scala @@ -19,12 +19,12 @@ package org.apache.spark.sql.execution.datasources.v2 import java.io.IOException -import org.apache.spark.sql.sources.v2.reader.DataReader +import org.apache.spark.sql.sources.v2.reader.InputPartitionReader /** - * A [[DataReader]] with empty output. + * A [[InputPartitionReader]] with empty output. */ -class EmptyDataReader[T] extends DataReader[T] { +class EmptyInputPartitionReader[T] extends InputPartitionReader[T] { override def next(): Boolean = false override def get(): T = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileInputPartition.scala similarity index 86% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileReaderFactory.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileInputPartition.scala index fae416e48774..9399c98e2710 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileInputPartition.scala @@ -24,19 +24,19 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.rdd.InputFileBlockHolder import org.apache.spark.sql.execution.datasources.{FilePartition, FilePartitionUtil, PartitionedFile} -import org.apache.spark.sql.sources.v2.reader.{DataReader, DataReaderFactory} +import org.apache.spark.sql.sources.v2.reader.{InputPartition, InputPartitionReader} import org.apache.spark.sql.vectorized.ColumnarBatch -case class FileReaderFactory[T]( +case class FileInputPartition[T]( file: FilePartition, - readFunction: (PartitionedFile) => DataReader[T], + readFunction: (PartitionedFile) => InputPartitionReader[T], ignoreCorruptFiles: Boolean = false, ignoreMissingFiles: Boolean = false) - extends DataReaderFactory[T] { - override def createDataReader(): DataReader[T] = { + extends InputPartition[T] { + override def createPartitionReader(): InputPartitionReader[T] = { val taskContext = TaskContext.get() - val iter = file.files.iterator.map(f => PartitionedFileDataReader(f, readFunction(f))) - FileDataReader(taskContext, iter, ignoreCorruptFiles, ignoreMissingFiles) + val iter = file.files.iterator.map(f => PartitionedFileReader(f, readFunction(f))) + FileInputPartitionReader(taskContext, iter, ignoreCorruptFiles, ignoreMissingFiles) } override def preferredLocations(): Array[String] = { @@ -44,9 +44,9 @@ case class FileReaderFactory[T]( } } -case class PartitionedFileDataReader[T]( +case class PartitionedFileReader[T]( file: PartitionedFile, - reader: DataReader[T]) extends DataReader[T] { + reader: InputPartitionReader[T]) extends InputPartitionReader[T] { override def next(): Boolean = reader.next() override def get(): T = reader.get() @@ -54,11 +54,11 @@ case class PartitionedFileDataReader[T]( override def close(): Unit = reader.close() } -case class FileDataReader[T]( +case class FileInputPartitionReader[T]( context: TaskContext, - readers: Iterator[PartitionedFileDataReader[T]], + readers: Iterator[PartitionedFileReader[T]], ignoreCorruptFiles: Boolean, - ignoreMissingFiles: Boolean) extends DataReader[T] with Logging { + ignoreMissingFiles: Boolean) extends InputPartitionReader[T] with Logging { private val inputMetrics = context.taskMetrics().inputMetrics private val existingBytesRead = inputMetrics.bytesRead @@ -83,7 +83,7 @@ case class FileDataReader[T]( } } - private[this] var currentFile: PartitionedFileDataReader[T] = null + private[this] var currentFile: PartitionedFileReader[T] = null private def hasNext(): Boolean = { if (currentFile == null) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala index b861f89f2d9f..c4ab843a22b8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileSourceReader.scala @@ -24,7 +24,6 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.sql.{AnalysisException, SparkSession} -import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Expression, UnsafeRow} import org.apache.spark.sql.execution.PartitionedFileUtil import org.apache.spark.sql.execution.datasources._ @@ -53,10 +52,10 @@ abstract class FileSourceReader(options: DataSourceOptions, userSpecifiedSchema: } /** - * Returns a function that can be used to read a single file in as an [[DataReader]] of + * Returns a function that can be used to read a single file in as an [[InputPartitionReader]] of * [[UnsafeRow]]. */ - def unsafeRowDataReader: PartitionedFile => DataReader[UnsafeRow] + def unsafeInputPartitionReader: PartitionedFile => InputPartitionReader[UnsafeRow] protected val sparkSession = SparkSession.getActiveSession .getOrElse(SparkSession.getDefaultSession.get) @@ -130,11 +129,11 @@ abstract class FileSourceReader(options: DataSourceOptions, userSpecifiedSchema: pushedFiltersArray } - override def createUnsafeRowReaderFactories: JList[DataReaderFactory[UnsafeRow]] = { + override def planUnsafeInputPartitions: JList[InputPartition[UnsafeRow]] = { partitions.map { filePartition => - new FileReaderFactory[UnsafeRow](filePartition, unsafeRowDataReader, + new FileInputPartition[UnsafeRow](filePartition, unsafeInputPartitionReader, ignoreCorruptFiles, ignoreMissingFiles) - .asInstanceOf[DataReaderFactory[UnsafeRow]] + .asInstanceOf[InputPartition[UnsafeRow]] }.asJava } } @@ -145,16 +144,16 @@ abstract class ColumnarBatchFileSourceReader( extends FileSourceReader(options: DataSourceOptions, userSpecifiedSchema: Option[StructType]) with SupportsScanColumnarBatch { /** - * Returns a function that can be used to read a single file in as an [[DataReader]] of + * Returns a function that can be used to read a single file in as an [[InputPartitionReader]] of * [[ColumnarBatch]]. */ - def columnarBatchDataReader: PartitionedFile => DataReader[ColumnarBatch] + def columnarBatchInputPartitionReader: PartitionedFile => InputPartitionReader[ColumnarBatch] - override def createBatchDataReaderFactories(): JList[DataReaderFactory[ColumnarBatch]] = { + override def planBatchInputPartitions(): JList[InputPartition[ColumnarBatch]] = { partitions.map { filePartition => - new FileReaderFactory[ColumnarBatch](filePartition, columnarBatchDataReader, + new FileInputPartition[ColumnarBatch](filePartition, columnarBatchInputPartitionReader, ignoreCorruptFiles, ignoreMissingFiles) - .asInstanceOf[DataReaderFactory[ColumnarBatch]] + .asInstanceOf[InputPartition[ColumnarBatch]] }.asJava } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RecordDataReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionRecordReader.scala similarity index 81% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RecordDataReader.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionRecordReader.scala index c4a9585b3a32..a0404a21f845 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RecordDataReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionRecordReader.scala @@ -19,9 +19,10 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.hadoop.mapreduce.RecordReader -import org.apache.spark.sql.sources.v2.reader.DataReader +import org.apache.spark.sql.sources.v2.reader.InputPartitionReader -class RecordDataReader[T](private[this] var rowReader: RecordReader[_, T]) extends DataReader[T] { +class PartitionRecordReader[T]( + private[this] var rowReader: RecordReader[_, T]) extends InputPartitionReader[T] { override def next(): Boolean = rowReader.nextKeyValue() override def get(): T = rowReader.getCurrentValue @@ -29,9 +30,9 @@ class RecordDataReader[T](private[this] var rowReader: RecordReader[_, T]) exten override def close(): Unit = rowReader.close() } -class RecordDataReadeWithProject[X, T]( +class PartitionRecordDReaderWithProject[X, T]( private[this] var rowReader: RecordReader[_, X], - project: X => T) extends DataReader[T] { + project: X => T) extends InputPartitionReader[T] { override def next(): Boolean = rowReader.nextKeyValue() override def get(): T = project(rowReader.getCurrentValue) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala index 8188ff3be880..08e1e4d30e3a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala @@ -94,7 +94,8 @@ case class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: override def isSplitable(path: Path): Boolean = true - override def columnarBatchDataReader: (PartitionedFile) => DataReader[ColumnarBatch] = { + override def columnarBatchInputPartitionReader: + (PartitionedFile) => InputPartitionReader[ColumnarBatch] = { val capacity = sqlConf.orcVectorizedReaderBatchSize val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled val copyToSpark = sqlConf.getConf(SQLConf.ORC_COPY_BATCH_TO_SPARK) @@ -117,7 +118,7 @@ case class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: isCaseSensitive, dataSchema, readSchema, reader, conf) if (requestedColIdsOrEmptyFile.isEmpty) { - new EmptyDataReader + new EmptyInputPartitionReader } else { val requestedColIds = requestedColIdsOrEmptyFile.get assert(requestedColIds.length == readSchema.length, @@ -143,12 +144,12 @@ case class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: requestedColIds, partitionColIds, file.partitionValues) - new RecordDataReader(batchReader) + new PartitionRecordReader(batchReader) } } } - override def unsafeRowDataReader: (PartitionedFile) => DataReader[UnsafeRow] = { + override def unsafeInputPartitionReader: (PartitionedFile) => InputPartitionReader[UnsafeRow] = { val isCaseSensitive = this.isCaseSensitive val dataSchema = this.dataSchema val readSchema = this.readSchema() @@ -168,7 +169,7 @@ case class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: isCaseSensitive, dataSchema, readSchema, reader, conf) if (requestedColIdsOrEmptyFile.isEmpty) { - new EmptyDataReader[UnsafeRow] + new EmptyInputPartitionReader[UnsafeRow] } else { val requestedColIds = requestedColIdsOrEmptyFile.get assert(requestedColIds.length == readSchema.length, @@ -197,7 +198,7 @@ case class OrcDataSourceReader(options: DataSourceOptions, userSpecifiedSchema: (value: OrcStruct) => unsafeProjection(joinedRow(deserializer.deserialize(value), file.partitionValues)) } - new RecordDataReadeWithProject(orcRecordReader, projection) + new PartitionRecordDReaderWithProject(orcRecordReader, projection) } } }