From 5d5f59e83c7c3b6bcb102c3b4793359c9b9b453d Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Fri, 4 Jan 2019 01:22:41 +0800 Subject: [PATCH 01/31] orc data source v2 --- .../apache/spark/sql/internal/SQLConf.scala | 9 + .../apache/spark/sql/DataFrameReader.scala | 12 +- .../sql/execution/DataSourceScanExec.scala | 68 ++------ .../sql/execution/PartitionedFileUtil.scala | 107 ++++++++++++ .../spark/sql/execution/command/tables.scala | 3 +- .../datasources/CatalogFileIndex.scala | 3 +- .../execution/datasources/DataSource.scala | 97 +++++++---- .../datasources/DataSourceStrategy.scala | 21 +++ .../datasources/FilePartitionUtil.scala | 60 +++++++ .../execution/datasources/FileScanRDD.scala | 38 +++-- .../datasources/HadoopFsRelation.scala | 22 +-- .../datasources/InMemoryFileIndex.scala | 43 ++++- .../PartitioningAwareFileIndex.scala | 10 +- .../datasources/PartitioningUtils.scala | 73 +++++++- .../PruneFileSourcePartitions.scala | 37 ++++ .../datasources/orc/OrcFilters.scala | 15 +- .../datasources/v2/EmptyPartitionReader.scala | 34 ++++ .../datasources/v2/FileDataSourceV2.scala | 57 +++++++ .../datasources/v2/FilePartitionReader.scala | 75 ++++++++ .../v2/FilePartitionReaderFactory.scala | 44 +++++ .../execution/datasources/v2/FileScan.scala | 67 ++++++++ .../datasources/v2/FileScanBuilder.scala | 32 ++++ .../execution/datasources/v2/FileTable.scala | 54 ++++++ .../v2/PartitionRecordReader.scala | 41 +++++ .../datasources/v2/orc/OrcDataSourceV2.scala | 37 ++++ .../v2/orc/OrcPartitionReaderFactory.scala | 151 +++++++++++++++++ .../datasources/v2/orc/OrcScan.scala | 44 +++++ .../datasources/v2/orc/OrcScanBuilder.scala | 47 +++++ .../datasources/v2/orc/OrcTable.scala | 38 +++++ .../streaming/MetadataLogFileIndex.scala | 1 + .../internal/BaseSessionStateBuilder.scala | 1 + .../datasources/orc/OrcFilterSuite.scala | 160 +++++++++++++----- .../orc/OrcPartitionDiscoverySuite.scala | 9 +- .../datasources/orc/OrcQuerySuite.scala | 9 +- .../v2/FileDataSourceV2FallBackSuite.scala | 78 +++++++++ .../apache/spark/sql/test/SQLTestUtils.scala | 25 ++- .../sql/hive/HiveSessionStateBuilder.scala | 1 + .../sql/hive/MetastoreDataSourcesSuite.scala | 2 +- 38 files changed, 1440 insertions(+), 185 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/EmptyPartitionReader.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionRecordReader.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/OrcPartitionReaderFactory.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala 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 c1b885a72ad3e..bca8d67f6042a 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 @@ -1419,6 +1419,13 @@ 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_STREAMING_WRITERS = buildConf("spark.sql.streaming.disabledV2Writers") .internal() .doc("A comma-separated list of fully qualified data source register class names for which" + @@ -2002,6 +2009,8 @@ 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 disabledV2StreamingWriters: String = getConf(DISABLED_V2_STREAMING_WRITERS) def disabledV2StreamingMicroBatchReaders: 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 af369a5bca464..d69fff6fe9650 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,8 +37,7 @@ import org.apache.spark.sql.execution.datasources.DataSource 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, FileTable} import org.apache.spark.sql.sources.v2._ import org.apache.spark.sql.types.StructType import org.apache.spark.unsafe.types.UTF8String @@ -193,7 +192,14 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { "read files of Hive data source directly.") } - val cls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf) + val disabledV2Readers = + sparkSession.sessionState.conf.disabledV2FileDataSourceReader.split(",") + val lookupCls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf) + val cls = lookupCls.newInstance() match { + case f: FileDataSourceV2 if disabledV2Readers.contains(f.shortName()) => f.fallBackFileFormat + case _ => lookupCls + } + if (classOf[TableProvider].isAssignableFrom(cls)) { val provider = cls.getConstructor().newInstance().asInstanceOf[TableProvider] val sessionOptions = DataSourceV2Utils.extractSessionConfigs( 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 8b84eda361038..985daa5eeebba 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 @@ -373,8 +373,7 @@ case class FileSourceScanExec( val filesGroupedToBuckets = selectedPartitions.flatMap { p => p.files.filter(_.getLen > 0).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 @@ -410,64 +409,31 @@ case class FileSourceScanExec( readFile: (PartitionedFile) => Iterator[InternalRow], selectedPartitions: Seq[PartitionDirectory], fsRelation: HadoopFsRelation): RDD[InternalRow] = { - val defaultMaxSplitBytes = - fsRelation.sparkSession.sessionState.conf.filesMaxPartitionBytes 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)) + val maxSplitBytes = + PartitionedFileUtil.maxSplitBytes(fsRelation.sparkSession, selectedPartitions) 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.filter(_.getLen > 0).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.getFilePartitions(relation.sparkSession, splitFiles, maxSplitBytes) new FileScanRDD(fsRelation.sparkSession, readFile, partitions) } 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 0000000000000..06585271c1650 --- /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/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index e2cd40906f401..314c52388f0ea 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 @@ -237,7 +238,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( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CatalogFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CatalogFileIndex.scala index a66a07673e25f..93196b970bb19 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CatalogFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CatalogFileIndex.scala @@ -82,7 +82,7 @@ class CatalogFileIndex( val partitionSpec = PartitionSpec(partitionSchema, partitions) val timeNs = System.nanoTime() - startTime new PrunedInMemoryFileIndex( - sparkSession, new Path(baseLocation.get), fileStatusCache, partitionSpec, Option(timeNs)) + sparkSession, fileStatusCache, partitionSpec, Option(timeNs)) } else { new InMemoryFileIndex( sparkSession, rootPaths, table.storage.properties, userSpecifiedSchema = None) @@ -111,7 +111,6 @@ class CatalogFileIndex( */ private class PrunedInMemoryFileIndex( sparkSession: SparkSession, - tableBasePath: Path, fileStatusCache: FileStatusCache, override val partitionSpec: PartitionSpec, override val metadataOpsTimeNs: Option[Long]) 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 5dad784e45af5..809abaf771fef 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,11 +18,13 @@ 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} import scala.util.{Failure, Success, Try} +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.deploy.SparkHadoopUtil @@ -40,6 +42,8 @@ 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} import org.apache.spark.sql.internal.SQLConf @@ -90,10 +94,19 @@ 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) + // 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 + case _ => cls + } + } 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 => @@ -429,7 +442,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 => @@ -543,40 +555,9 @@ case class DataSource( checkFilesExist: Boolean): Seq[Path] = { val allPaths = caseInsensitiveOptions.get("path") ++ paths val hadoopConf = sparkSession.sessionState.newHadoopConf() - val allGlobPath = 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 - if (checkFilesExist) { - val (filteredOut, filteredIn) = allGlobPath.partition { path => - InMemoryFileIndex.shouldFilterOut(path.getName) - } - if (filteredOut.nonEmpty) { - if (filteredIn.isEmpty) { - logWarning( - s"All paths were ignored:\n ${filteredOut.mkString("\n ")}") - } else { - logDebug( - s"Some paths were ignored:\n ${filteredOut.mkString("\n ")}") - } - } - } - - allGlobPath + DataSource.checkAndGlobPathIfNecessary(allPaths.toSeq, hadoopConf, + checkEmptyGlobPath, checkFilesExist) } } @@ -632,7 +613,7 @@ object DataSource extends Logging { val provider1 = backwardCompatibilityMap.getOrElse(provider, provider) match { case name if name.equalsIgnoreCase("orc") && conf.getConf(SQLConf.ORC_IMPLEMENTATION) == "native" => - classOf[OrcFileFormat].getCanonicalName + classOf[OrcDataSourceV2].getCanonicalName case name if name.equalsIgnoreCase("orc") && conf.getConf(SQLConf.ORC_IMPLEMENTATION) == "hive" => "org.apache.spark.sql.hive.orc.OrcFileFormat" @@ -721,6 +702,48 @@ 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] = { + val allGlobPath = 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 + } + + if (checkFilesExist) { + val (filteredOut, filteredIn) = allGlobPath.partition { path => + InMemoryFileIndex.shouldFilterOut(path.getName) + } + if (filteredIn.isEmpty) { + logWarning( + s"All paths were ignored:\n ${filteredOut.mkString("\n ")}") + } else { + logDebug( + s"Some paths were ignored:\n ${filteredOut.mkString("\n ")}") + } + } + + allGlobPath + } + /** * 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/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index b5cf8c9515bfb..5fc2d86f3a9f5 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,9 @@ package org.apache.spark.sql.execution.datasources import java.util.Locale import java.util.concurrent.Callable +import javax.activation.FileDataSource + +import scala.collection.JavaConverters._ import org.apache.hadoop.fs.Path @@ -37,8 +40,10 @@ import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoTab 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, FileDataSourceV2, FileTable} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ +import org.apache.spark.sql.sources.v2.DataSourceOptions import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -213,6 +218,22 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast } } +/** + * 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 resolveOperators { + case i @InsertIntoTable(d @ + DataSourceV2Relation(source: FileDataSourceV2, table: FileTable, _, _, _), _, _, _, _) => + val v1FileFormat = source.fallBackFileFormat.getConstructor().newInstance() + val relation = HadoopFsRelation(table.getFileIndex, table.getFileIndex.partitionSchema, + table.schema(), None, v1FileFormat, d.options)(sparkSession) + i.copy(table = LogicalRelation(relation)) + } +} /** * 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 0000000000000..c29dc0a32c95a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartitionUtil.scala @@ -0,0 +1,60 @@ +/* + * 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 scala.collection.mutable.ArrayBuffer + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession + +object FilePartitionUtil extends Logging { + + def getFilePartitions( + 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 + } +} 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 ffea33c08ef94..6c3b0b5e04771 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 @@ -29,6 +29,7 @@ import org.apache.spark.rdd.{InputFileBlockHolder, 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.sources.v2.reader.InputPartition import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.NextIterator @@ -57,7 +58,25 @@ case class PartitionedFile( * A collection of file blocks that should be read as a single task * (possibly from multiple partitioned directories). */ -case class FilePartition(index: Int, files: Seq[PartitionedFile]) extends RDDPartition +case class FilePartition(index: Int, files: Seq[PartitionedFile]) + extends RDDPartition with InputPartition { + override def preferredLocations(): Array[String] = { + // 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 + }.toArray + } +} /** * An RDD that scans a list of file partitions. @@ -216,21 +235,6 @@ class FileScanRDD( 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 - } + split.asInstanceOf[FilePartition].preferredLocations() } } 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 b2f73b7f8d1fc..d278802e6c9f2 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/InMemoryFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala index fe418e610da8f..a6b7cee65c508 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala @@ -28,7 +28,8 @@ import org.apache.hadoop.mapred.{FileInputFormat, JobConf} import org.apache.spark.SparkContext import org.apache.spark.internal.Logging import org.apache.spark.metrics.source.HiveCatalogMetrics -import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, InterpretedPredicate} import org.apache.spark.sql.execution.streaming.FileStreamSink import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration @@ -132,6 +133,46 @@ class InMemoryFileIndex( } output } + + /** + * Returns a [[InMemoryFileIndex]] for this table restricted to the subset of partitions + * specified by the given partition-pruning filters. + * + * @param filters partition-pruning filters + */ + def filterPartitions(filters: Seq[Expression]): InMemoryFileIndex = { + if (partitionSchema.nonEmpty) { + val startTime = System.nanoTime() + val partitionColumnNames = partitionSchema.fields.map(_.name).toSet + + val nonPartitionPruningPredicates = filters.filterNot { + _.references.map(_.name).toSet.subsetOf(partitionColumnNames) + } + if (nonPartitionPruningPredicates.nonEmpty) { + throw new AnalysisException("Expected only partition pruning predicates: " + + nonPartitionPruningPredicates) + } + + val boundPredicate = + InterpretedPredicate.create(filters.reduce(And).transform { + case att: AttributeReference => + val index = partitionSchema.indexWhere(_.name == att.name) + BoundReference(index, partitionSchema(index).dataType, nullable = true) + }) + + val spec = partitionSpec() + val prunedPartitions = spec.partitions.filter { p => + boundPredicate.eval(p.values) + } + val prunedPartitionSpec = spec.copy(partitions = prunedPartitions) + + val timeNs = System.nanoTime() - startTime + new PrunedInMemoryFileIndex( + sparkSession, fileStatusCache, prunedPartitionSpec, Option(timeNs)) + } else { + this + } + } } object InMemoryFileIndex extends Logging { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala index b2e4155e6f49e..39e23550e8feb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.datasources +import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.hadoop.conf.Configuration @@ -27,6 +28,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{expressions, InternalRow} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.sources.v2.DataSourceOptions import org.apache.spark.sql.types.{StringType, StructType} /** @@ -49,7 +51,13 @@ abstract class PartitioningAwareFileIndex( override def partitionSchema: StructType = partitionSpec().partitionColumns - protected val hadoopConf: Configuration = + def dataSourceOptions(): DataSourceOptions = new DataSourceOptions(parameters.asJava) + + def getSparkSession: SparkSession = this.sparkSession + + def getUserSpecifiedSchema: Option[StructType] = this.userSpecifiedSchema + + val hadoopConf: Configuration = sparkSession.sessionState.newHadoopConfWithOptions(parameters) protected def leafFiles: mutable.LinkedHashMap[Path, FileStatus] 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 ee770426e61f9..4e2fbcb26c6ca 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 @@ -30,7 +31,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{Resolver, TypeCoercion} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec -import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Cast, Literal} import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateFormatter, DateTimeUtils, TimestampFormatter} import org.apache.spark.sql.types._ import org.apache.spark.sql.util.SchemaUtils @@ -65,9 +66,7 @@ object PartitioningUtils { require(columnNames.size == literals.size) } - import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.DEFAULT_PARTITION_NAME - import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName - import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.unescapePathName + import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.{escapePathName, unescapePathName, DEFAULT_PARTITION_NAME} /** * Given a group of qualified paths, tries to parse them and returns a partition specification. @@ -558,6 +557,72 @@ object PartitioningUtils { }).asNullable } + def partitionColumns( + output: Seq[AttributeReference], + partitionSchema: StructType, + caseSensitive: Boolean): Seq[AttributeReference] = { + val equality = columnNameEquality(caseSensitive) + partitionSchema.fields.map { col => + output.find(a => equality(a.name, col.name)).getOrElse { + throw new AnalysisException(s"Partition column `$col` not found in $output") + } + } + } + + def mergeDataAndPartitionSchema( + dataSchema: StructType, + partitionSchema: StructType, + caseSensitive: Boolean): (StructType, Map[String, StructField]) = { + 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) + } + + 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/PruneFileSourcePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala index 329b9539f52e3..61d60ae738f21 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala @@ -22,6 +22,9 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileTable} +import org.apache.spark.sql.sources.v2.SupportsBatchRead +import org.apache.spark.sql.types.StructType private[sql] object PruneFileSourcePartitions extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { @@ -74,5 +77,39 @@ private[sql] object PruneFileSourcePartitions extends Rule[LogicalPlan] { } else { op } + case op @ PhysicalOperation(projects, filters, + relation @DataSourceV2Relation(_, table: FileTable, output: Seq[AttributeReference], _, _)) + if filters.nonEmpty && table.getFileIndex.isInstanceOf[InMemoryFileIndex] && + ! table.getFileIndex.isInstanceOf[PrunedInMemoryFileIndex] + && table.getFileIndex.partitionSchema.nonEmpty => + // The attribute name of predicate could be different than the one in schema in case of + // case insensitive, we should change them to match the one in schema, so we donot need to + // worry about case sensitivity anymore. + val normalizedFilters = filters.filterNot(SubqueryExpression.hasSubquery).map { e => + e transform { + case a: AttributeReference => + a.withName(output.find(_.semanticEquals(a)).get.name) + } + } + val fileIndex = table.getFileIndex.asInstanceOf[InMemoryFileIndex] + val caseSensitive = fileIndex.getSparkSession.sessionState.conf.caseSensitiveAnalysis + val partitionColumns = + PartitioningUtils.partitionColumns(output, fileIndex.partitionSchema, caseSensitive) + val partitionSet = AttributeSet(partitionColumns) + val partitionKeyFilters = ExpressionSet(normalizedFilters + .filter(_.references.subsetOf(partitionSet))) + + if (partitionKeyFilters.nonEmpty) { + val prunedFileIndex = fileIndex.filterPartitions(partitionKeyFilters.toSeq) + val prunedTable = table.withNewFileIndex(prunedFileIndex) + val prunedRelation = relation.copy(table = prunedTable) + // Todo: Change table stats based on the sizeInBytes of pruned files + // Keep partition-pruning predicates so that they are visible in physical planning + val filterExpression = filters.reduceLeft(And) + val filter = Filter(filterExpression, prunedRelation) + Project(projects, filter) + } else { + op + } } } 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 0a64981b421c6..210eef4e4a7cc 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 @@ -85,19 +85,24 @@ private[sql] 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, newBuilder) - } yield filter + val convertible = convertibleFilters(schema, filters) for { // Combines all convertible filters using `And` to produce a single conjunction - conjunction <- buildTree(convertibleFilters) + conjunction <- buildTree(convertible) // Then tries to build a single ORC `SearchArgument` for the conjunction predicate builder <- buildSearchArgument(dataTypeMap, conjunction, 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, 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/v2/EmptyPartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyPartitionReader.scala new file mode 100644 index 0000000000000..b177d15e1fe32 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyPartitionReader.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.PartitionReader + +/** + * A [[PartitionReader]] with empty output. + */ +class EmptyPartitionReader[T] extends PartitionReader[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/FileDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala new file mode 100644 index 0000000000000..80e815e048197 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala @@ -0,0 +1,57 @@ +/* + * 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 scala.collection.JavaConverters._ + +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.sources.DataSourceRegister +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, SupportsBatchRead, TableProvider} +import org.apache.spark.sql.types.StructType + +/** + * A base interface for data source v2 implementations of the built-in file-based data sources. + */ +trait FileDataSourceV2 extends TableProvider 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 [[SupportsBatchRead]] is implemented while [[SupportsBatchWrite]] 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] + + lazy val sparkSession = + SparkSession.getActiveSession.getOrElse(SparkSession.getDefaultSession.get) + + def getFileIndex( + options: DataSourceOptions, + userSpecifiedSchema: Option[StructType]): PartitioningAwareFileIndex = { + val filePaths = options.paths() + val hadoopConf = + sparkSession.sessionState.newHadoopConfWithOptions(options.asMap().asScala.toMap) + val rootPathsSpecified = DataSource.checkAndGlobPathIfNecessary(filePaths, hadoopConf, + checkEmptyGlobPath = false, checkFilesExist = false) + val fileStatusCache = FileStatusCache.getOrCreate(sparkSession) + new InMemoryFileIndex(sparkSession, rootPathsSpecified, + options.asMap().asScala.toMap, userSpecifiedSchema, fileStatusCache) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala new file mode 100644 index 0000000000000..f588722040150 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala @@ -0,0 +1,75 @@ +/* + * 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.{FileNotFoundException, IOException} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.reader.PartitionReader + +class FilePartitionReader[T]( + readers: Iterator[PartitionReader[T]]) extends PartitionReader[T] with Logging { + private var currentFile: PartitionReader[T] = null + + private val sqlConf = SQLConf.get + private val ignoreMissingFiles = sqlConf.ignoreMissingFiles + private val ignoreCorruptFiles = sqlConf.ignoreCorruptFiles + + override def next(): Boolean = { + if (currentFile == null) { + if (readers.hasNext) { + 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 + } + } + if (currentFile.next()) { + return true + } else { + close() + currentFile = null + } + next() + } + + override def get(): T = currentFile.get() + + override def close(): Unit = { + if (currentFile != null) { + currentFile.close() + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala new file mode 100644 index 0000000000000..16d224ffc4070 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala @@ -0,0 +1,44 @@ +/* + * 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.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile} +import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader, PartitionReaderFactory} +import org.apache.spark.sql.vectorized.ColumnarBatch + +abstract class FilePartitionReaderFactory extends PartitionReaderFactory { + override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { + assert(partition.isInstanceOf[FilePartition]) + val filePartition = partition.asInstanceOf[FilePartition] + val iter = filePartition.files.toIterator.map(buildReader) + new FilePartitionReader[InternalRow](iter) + } + + override def createColumnarReader(partition: InputPartition): PartitionReader[ColumnarBatch] = { + assert(partition.isInstanceOf[FilePartition]) + val filePartition = partition.asInstanceOf[FilePartition] + val iter = filePartition.files.toIterator.map(buildColumnarReader) + new FilePartitionReader[ColumnarBatch](iter) + } + + def buildReader(partitionedFile: PartitionedFile): PartitionReader[InternalRow] + + def buildColumnarReader(partitionedFile: PartitionedFile): PartitionReader[ColumnarBatch] = { + throw new UnsupportedOperationException("Cannot create columnar reader.") + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala new file mode 100644 index 0000000000000..3d812738bfa98 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala @@ -0,0 +1,67 @@ +/* + * 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 scala.collection.JavaConverters._ + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.{AnalysisException, SparkSession} +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.{Batch, InputPartition, Scan} +import org.apache.spark.sql.types.StructType + +abstract class FileScan( + fileIndex: PartitioningAwareFileIndex, + schema: StructType, + readSchema: StructType) extends Scan with Batch { + /** + * Returns whether a file with `path` could be split or not. + */ + def isSplitable(path: Path): Boolean = { + false + } + + protected def partitions: Seq[FilePartition] = { + val sparkSession = fileIndex.getSparkSession + val selectedPartitions = fileIndex.listFiles(Seq.empty, Seq.empty) + 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 = filePath, + isSplitable = isSplitable(filePath), + maxSplitBytes = maxSplitBytes, + partitionValues = partition.values + ) + }.toArray.sortBy(_.length)(implicitly[Ordering[Long]].reverse) + } + FilePartitionUtil.getFilePartitions(sparkSession, splitFiles, maxSplitBytes) + } + + override def planInputPartitions(): Array[InputPartition] = { + partitions.toArray + } + + override def toBatch: Batch = this +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala new file mode 100644 index 0000000000000..d7f16bc2cc7fb --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala @@ -0,0 +1,32 @@ +/* + * 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.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.sources.v2.reader.{ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns} +import org.apache.spark.sql.types.StructType + +abstract class FileScanBuilder(fileIndex: PartitioningAwareFileIndex, schema: StructType) + extends ScanBuilder + with SupportsPushDownRequiredColumns + with SupportsPushDownFilters { + protected var readSchema = schema + + override def pruneColumns(requiredSchema: StructType): Unit = { + this.readSchema = requiredSchema + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala new file mode 100644 index 0000000000000..699c41268ffba --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala @@ -0,0 +1,54 @@ +/* + * 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.fs.FileStatus + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.sources.v2.{SupportsBatchRead, Table} +import org.apache.spark.sql.types.StructType + +abstract class FileTable( + fileIndex: PartitioningAwareFileIndex, + userSpecifiedSchema: Option[StructType]) extends Table with SupportsBatchRead { + def getFileIndex: PartitioningAwareFileIndex = this.fileIndex + + override def name(): String = fileIndex.dataSourceOptions().paths().mkString(";") + + lazy val dataSchema: StructType = userSpecifiedSchema.orElse { + inferSchema(fileIndex.allFiles()) + }.getOrElse { + throw new AnalysisException( + s"Unable to infer schema for $name. It must be specified manually.") + }.asNullable + + override def schema(): StructType = { + val caseSensitive = fileIndex.getSparkSession.sessionState.conf.caseSensitiveAnalysis + PartitioningUtils.mergeDataAndPartitionSchema(dataSchema, + fileIndex.partitionSchema, caseSensitive)._1 + } + + /** + * 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 inferSchema(files: Seq[FileStatus]): Option[StructType] + + def withNewFileIndex(newFileIndex: PartitioningAwareFileIndex): FileTable +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionRecordReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionRecordReader.scala new file mode 100644 index 0000000000000..f716fa679bffd --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionRecordReader.scala @@ -0,0 +1,41 @@ +/* + * 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.PartitionReader + +class PartitionRecordReader[T]( + private[this] var rowReader: RecordReader[_, T]) extends PartitionReader[T] { + override def next(): Boolean = rowReader.nextKeyValue() + + override def get(): T = rowReader.getCurrentValue + + override def close(): Unit = rowReader.close() +} + +class PartitionRecordDReaderWithProject[X, T]( + private[this] var rowReader: RecordReader[_, X], + project: X => T) extends PartitionReader[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 new file mode 100644 index 0000000000000..7b823fb1f3519 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala @@ -0,0 +1,37 @@ +/* + * 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 org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.orc._ +import org.apache.spark.sql.execution.datasources.v2._ +import org.apache.spark.sql.sources.v2.{DataSourceOptions, Table} +import org.apache.spark.sql.types.StructType + +class OrcDataSourceV2 extends FileDataSourceV2 { + + override def fallBackFileFormat: Class[_ <: FileFormat] = classOf[OrcFileFormat] + + override def shortName(): String = "orc" + + override def getTable(options: DataSourceOptions): Table = { + OrcTable(getFileIndex(options, None), None) + } + + override def getTable(options: DataSourceOptions, schema: StructType): Table = + OrcTable(getFileIndex(options, Some(schema)), Some(schema)) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala new file mode 100644 index 0000000000000..33f26065baf07 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala @@ -0,0 +1,151 @@ +/* + * 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 +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.execution.datasources.{PartitionedFile, PartitioningUtils} +import org.apache.spark.sql.execution.datasources.orc.{OrcColumnarBatchReader, OrcDeserializer, OrcUtils} +import org.apache.spark.sql.execution.datasources.v2.{EmptyPartitionReader, FilePartitionReaderFactory, PartitionRecordDReaderWithProject, PartitionRecordReader} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader} +import org.apache.spark.sql.types.{AtomicType, StructType} +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.SerializableConfiguration + +case class OrcPartitionReaderFactory( + broadcastedConf: Broadcast[SerializableConfiguration], + schema: StructType, + dataSchema: StructType, + readSchema: StructType, + partitionSchema: StructType) extends FilePartitionReaderFactory { + private val sqlConf = SQLConf.get + private val isCaseSensitive = sqlConf.caseSensitiveAnalysis + val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled + val copyToSpark = sqlConf.getConf(SQLConf.ORC_COPY_BATCH_TO_SPARK) + val capacity = sqlConf.orcVectorizedReaderBatchSize + + override def supportColumnarReads(partition: InputPartition): Boolean = { + sqlConf.orcVectorizedReaderEnabled && sqlConf.wholeStageEnabled && + schema.length <= sqlConf.wholeStageMaxNumFields && + schema.forall(_.dataType.isInstanceOf[AtomicType]) + } + + override def buildReader(file: PartitionedFile): PartitionReader[InternalRow] = { + 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 EmptyPartitionReader[InternalRow] + } 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 PartitionRecordDReaderWithProject(orcRecordReader, projection) + } + } + + override def buildColumnarReader(file: PartitionedFile): PartitionReader[ColumnarBatch] = { + 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, schema, readSchema, reader, conf) + + if (requestedColIdsOrEmptyFile.isEmpty) { + new EmptyPartitionReader + } 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()) + + 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 PartitionRecordReader(batchReader) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala new file mode 100644 index 0000000000000..144f5c6da57a2 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala @@ -0,0 +1,44 @@ +/* + * 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 org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.execution.datasources.v2.FileScan +import org.apache.spark.sql.sources.v2.DataSourceOptions +import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.SerializableConfiguration + +case class OrcScan( + fileIndex: PartitioningAwareFileIndex, + schema: StructType, + dataSchema: StructType, + readSchema: StructType) + extends FileScan(fileIndex, schema, readSchema) { + override def isSplitable(path: Path): Boolean = true + + override def createReaderFactory(): PartitionReaderFactory = { + val broadcastedConf = fileIndex.getSparkSession.sparkContext.broadcast( + new SerializableConfiguration(fileIndex.hadoopConf)) + OrcPartitionReaderFactory(broadcastedConf, schema, + dataSchema, readSchema, fileIndex.partitionSchema) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala new file mode 100644 index 0000000000000..6c967c71b8dca --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala @@ -0,0 +1,47 @@ +/* + * 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 org.apache.orc.mapreduce.OrcInputFormat + +import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.execution.datasources.orc.OrcFilters +import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.sources.v2.reader.Scan +import org.apache.spark.sql.types.StructType + +case class OrcScanBuilder( + fileIndex: PartitioningAwareFileIndex, + schema: StructType, + dataSchema: StructType) extends FileScanBuilder(fileIndex, schema) { + override def build(): Scan = OrcScan(fileIndex, schema, dataSchema, readSchema) + + private var _pushedFilters: Array[Filter] = Array.empty + + override def pushFilters(filters: Array[Filter]): Array[Filter] = { + if (fileIndex.getSparkSession.sessionState.conf.orcFilterPushDown) { + OrcFilters.createFilter(schema, filters).foreach { f => + OrcInputFormat.setSearchArgument(fileIndex.hadoopConf, f, schema.fieldNames) + } + _pushedFilters = OrcFilters.convertibleFilters(schema, filters).toArray + } + filters + } + + override def pushedFilters(): Array[Filter] = _pushedFilters +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala new file mode 100644 index 0000000000000..8ee5045501b8f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala @@ -0,0 +1,38 @@ +/* + * 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 org.apache.hadoop.fs.FileStatus + +import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.execution.datasources.orc.OrcUtils +import org.apache.spark.sql.execution.datasources.v2.FileTable +import org.apache.spark.sql.sources.v2.DataSourceOptions +import org.apache.spark.sql.types.StructType + +case class OrcTable( + fileIndex: PartitioningAwareFileIndex, + userSpecifiedSchema: Option[StructType]) extends FileTable(fileIndex, userSpecifiedSchema) { + override def newScanBuilder(options: DataSourceOptions): OrcScanBuilder = + new OrcScanBuilder(fileIndex, schema, dataSchema) + + override def inferSchema(files: Seq[FileStatus]): Option[StructType] = + OrcUtils.readSchema(fileIndex.getSparkSession, files) + + override def withNewFileIndex(newFileIndex: PartitioningAwareFileIndex): FileTable = + copy(fileIndex = newFileIndex) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileIndex.scala index 5cacdd070b735..89f1006fa51bf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileIndex.scala @@ -22,6 +22,7 @@ import scala.collection.mutable import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.types.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 319c2649592fb..569a04921d4ca 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 @@ -159,6 +159,7 @@ abstract class BaseSessionStateBuilder( override val extendedResolutionRules: Seq[Rule[LogicalPlan]] = new FindDataSourceTable(session) +: new ResolveSQLOnFile(session) +: + new FallBackFileDataSourceToV1(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 ee12f30892436..5966dd8532965 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 @@ -24,11 +24,16 @@ import scala.collection.JavaConverters._ import org.apache.orc.storage.ql.io.sarg.{PredicateLeaf, SearchArgument} -import org.apache.spark.sql.{Column, DataFrame} +import org.apache.spark.SparkConf +import org.apache.spark.sql.{AnalysisException, Column, DataFrame} 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.OrcTable +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.DataSourceOptions import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -41,7 +46,7 @@ import org.apache.spark.sql.types._ */ class OrcFilterSuite extends OrcTest with SharedSQLContext { - private def checkFilterPredicate( + protected def checkFilterPredicate( df: DataFrame, predicate: Predicate, checker: (SearchArgument) => Unit): Unit = { @@ -50,24 +55,24 @@ class OrcFilterSuite extends OrcTest with SharedSQLContext { .select(output.map(e => Column(e)): _*) .where(Column(predicate)) - var maybeRelation: Option[HadoopFsRelation] = None - val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => - maybeRelation = Some(orcRelation) - 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 maybeFilter = OrcFilters.createFilter(query.schema, selectedFilters) - assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $selectedFilters") - checker(maybeFilter.get) + query.queryExecution.optimizedPlan match { + case PhysicalOperation(_, filters, + DataSourceV2Relation(_, orcTable: OrcTable, _, options, _)) => + assert(filters.nonEmpty, "No filter is analyzed from the given query") + val scanBuilder = orcTable.newScanBuilder(new DataSourceOptions(options.asJava)) + scanBuilder.pushFilters(filters.flatMap(DataSourceStrategy.translateFilter).toArray) + val pushedFilters = scanBuilder.pushedFilters() + assert(pushedFilters.nonEmpty, "No filter is pushed down") + val maybeFilter = OrcFilters.createFilter(query.schema, pushedFilters) + assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $pushedFilters") + checker(maybeFilter.get) + + case _ => + throw new AnalysisException("Can not match OrcTable in the query.") + } } - private def checkFilterPredicate + protected def checkFilterPredicate (predicate: Predicate, filterOperator: PredicateLeaf.Operator) (implicit df: DataFrame): Unit = { def checkComparisonOperator(filter: SearchArgument) = { @@ -77,7 +82,7 @@ class OrcFilterSuite extends OrcTest with SharedSQLContext { checkFilterPredicate(df, predicate, checkComparisonOperator) } - private def checkFilterPredicate + protected def checkFilterPredicate (predicate: Predicate, stringExpr: String) (implicit df: DataFrame): Unit = { def checkLogicalOperator(filter: SearchArgument) = { @@ -86,28 +91,32 @@ class OrcFilterSuite extends OrcTest with SharedSQLContext { checkFilterPredicate(df, predicate, checkLogicalOperator) } - private def checkNoFilterPredicate - (predicate: Predicate) + protected def checkNoFilterPredicate + (predicate: Predicate, noneSupported: Boolean = false) (implicit df: DataFrame): Unit = { val output = predicate.collect { case a: Attribute => a }.distinct val query = df .select(output.map(e => Column(e)): _*) .where(Column(predicate)) - var maybeRelation: Option[HadoopFsRelation] = None - val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => - maybeRelation = Some(orcRelation) - 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 maybeFilter = OrcFilters.createFilter(query.schema, selectedFilters) - assert(maybeFilter.isEmpty, s"Could generate filter predicate for $selectedFilters") + query.queryExecution.optimizedPlan match { + case PhysicalOperation(_, filters, + DataSourceV2Relation(_, orcTable: OrcTable, _, options, _)) => + assert(filters.nonEmpty, "No filter is analyzed from the given query") + val scanBuilder = orcTable.newScanBuilder(new DataSourceOptions(options.asJava)) + scanBuilder.pushFilters(filters.flatMap(DataSourceStrategy.translateFilter).toArray) + val pushedFilters = scanBuilder.pushedFilters() + if (noneSupported) { + assert(pushedFilters.isEmpty, "Unsupported filters should not show in pushed filters") + } else { + assert(pushedFilters.nonEmpty, "No filter is pushed down") + val maybeFilter = OrcFilters.createFilter(query.schema, pushedFilters) + assert(maybeFilter.isEmpty, s"Couldn't generate filter predicate for $pushedFilters") + } + + case _ => + throw new AnalysisException("Can not match OrcTable in the query.") + } } test("filter pushdown - integer") { @@ -346,15 +355,15 @@ class OrcFilterSuite extends OrcTest with SharedSQLContext { } // ArrayType withOrcDataFrame((1 to 4).map(i => Tuple1(Array(i)))) { implicit df => - checkNoFilterPredicate('_1.isNull) + checkNoFilterPredicate('_1.isNull, noneSupported = true) } // BinaryType withOrcDataFrame((1 to 4).map(i => Tuple1(i.b))) { implicit df => - checkNoFilterPredicate('_1 <=> 1.b) + checkNoFilterPredicate('_1 <=> 1.b, noneSupported = true) } // MapType withOrcDataFrame((1 to 4).map(i => Tuple1(Map(i -> i)))) { implicit df => - checkNoFilterPredicate('_1.isNotNull) + checkNoFilterPredicate('_1.isNotNull, noneSupported = true) } } @@ -419,3 +428,78 @@ class OrcFilterSuite extends OrcTest with SharedSQLContext { } } } + +class OrcV1FilterSuite extends OrcFilterSuite { + + override protected def sparkConf: SparkConf = + super.sparkConf.set(SQLConf.DISABLED_V2_FILE_DATA_SOURCE_READERS, "orc") + + override def checkFilterPredicate( + df: DataFrame, + predicate: Predicate, + checker: (SearchArgument) => Unit): Unit = { + val output = predicate.collect { case a: Attribute => a }.distinct + val query = df + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) + + var maybeRelation: Option[HadoopFsRelation] = None + val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { + case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => + maybeRelation = Some(orcRelation) + 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 maybeFilter = OrcFilters.createFilter(query.schema, selectedFilters) + assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $selectedFilters") + checker(maybeFilter.get) + } + + override def checkFilterPredicate + (predicate: Predicate, filterOperator: PredicateLeaf.Operator) + (implicit df: DataFrame): Unit = { + def checkComparisonOperator(filter: SearchArgument) = { + val operator = filter.getLeaves.asScala + assert(operator.map(_.getOperator).contains(filterOperator)) + } + checkFilterPredicate(df, predicate, checkComparisonOperator) + } + + override def checkFilterPredicate + (predicate: Predicate, stringExpr: String) + (implicit df: DataFrame): Unit = { + def checkLogicalOperator(filter: SearchArgument) = { + assert(filter.toString == stringExpr) + } + checkFilterPredicate(df, predicate, checkLogicalOperator) + } + + override def checkNoFilterPredicate + (predicate: Predicate, noneSupported: Boolean = false) + (implicit df: DataFrame): Unit = { + val output = predicate.collect { case a: Attribute => a }.distinct + val query = df + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) + + var maybeRelation: Option[HadoopFsRelation] = None + val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { + case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => + maybeRelation = Some(orcRelation) + 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 maybeFilter = OrcFilters.createFilter(query.schema, selectedFilters) + assert(maybeFilter.isEmpty, s"Could generate filter predicate for $selectedFilters") + } +} 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 d1911ea7f32a9..15e6500b894da 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. @@ -74,7 +76,7 @@ abstract class OrcPartitionDiscoveryTest extends OrcTest { ps <- Seq("foo", "bar") } yield Row(i, i.toString, pi, ps)) - checkAnswer( + checkAnswer( sql("SELECT intField, pi FROM t"), for { i <- 1 to 10 @@ -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.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 918dbcdfa1cc5..a2f254326885c 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} @@ -574,7 +574,7 @@ abstract class OrcQueryTest extends OrcTest { val m1 = intercept[AnalysisException] { testAllCorruptFiles() }.getMessage - assert(m1.contains("Unable to infer schema for ORC")) + assert(m1.contains("Unable to infer schema")) testAllCorruptFilesWithoutSchemaInfer() } @@ -681,3 +681,8 @@ class OrcQuerySuite extends OrcQueryTest with SharedSQLContext { } } } + +class OrcV1QuerySuite extends OrcQuerySuite { + override protected def sparkConf: SparkConf = + 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 0000000000000..d5e9c811de3dc --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala @@ -0,0 +1,78 @@ +/* + * 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 org.apache.spark.sql.{AnalysisException, QueryTest} +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.test.SharedSQLContext + +class DummyReadOnlyFileDataSourceV2 extends FileDataSourceV2 { + + override def fallBackFileFormat: Class[_ <: FileFormat] = classOf[ParquetFileFormat] + + override def shortName(): String = "parquet" + + override def getTable(options: DataSourceOptions): Table = { + throw new AnalysisException("Dummy file reader") + } +} + +class FileDataSourceV2FallBackSuite extends QueryTest with ParquetTest with SharedSQLContext { + + private val dummyParquetReaderV2 = classOf[DummyReadOnlyFileDataSourceV2].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 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")) + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index 85963ec4ca699..f914b9d64d731 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -24,6 +24,7 @@ import java.util.{Locale, UUID} import scala.concurrent.duration._ import scala.language.implicitConversions +import scala.util.Try import scala.util.control.NonFatal import org.apache.hadoop.fs.Path @@ -39,7 +40,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.PlanTestBase import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.execution.FilterExec +import org.apache.spark.sql.execution.{FilterExec, SQLExecution} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.UninterruptibleThread import org.apache.spark.util.Utils @@ -227,7 +228,27 @@ private[sql] trait SQLTestUtilsBase protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { SparkSession.setActiveSession(spark) - super.withSQLConf(pairs: _*)(f) + val sc = spark.sparkContext + // Set all the specified SQL configs to local properties, so that they can be available at + // the executor side. + // Note: `allSparkConfigs` may `overlap` with pairs, so use `.toMap` to reduce duplicated keys. + val allSparkConfigs = (spark.sessionState.conf.getAllConfs ++ pairs.toSeq).filter { + case (key, _) => key.startsWith("spark") + }.toMap + val originalLocalProps = allSparkConfigs.map { + case (key, value) => + val originalValue = sc.getLocalProperty(key) + sc.setLocalProperty(key, value) + (key, originalValue) + } + + try { + super.withSQLConf(pairs: _*)(f) + } finally { + for ((key, value) <- originalLocalProps) { + sc.setLocalProperty(key, value) + } + } } /** 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 4f3914740ec20..caa71e7cce133 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]] = 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 d93215fefb810..3402ed240f8b9 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 9ac801e3b9d4d36e2f74bb038ffe2e1f38e82794 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Sun, 6 Jan 2019 21:10:51 +0800 Subject: [PATCH 02/31] better data source fallback --- .../apache/spark/sql/internal/SQLConf.scala | 8 ++++---- .../apache/spark/sql/DataFrameReader.scala | 16 +++++++++------ .../apache/spark/sql/DataFrameWriter.scala | 14 ++++++++----- .../execution/datasources/DataSource.scala | 14 +++---------- .../datasources/DataSourceStrategy.scala | 20 +++++++++++++++++++ .../datasources/orc/OrcFilterSuite.scala | 2 +- .../orc/OrcPartitionDiscoverySuite.scala | 2 +- .../datasources/orc/OrcQuerySuite.scala | 2 +- .../v2/FileDataSourceV2FallBackSuite.scala | 4 ++-- 9 files changed, 51 insertions(+), 31 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 bca8d67f6042a..55fa520689659 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 @@ -1419,10 +1419,10 @@ object SQLConf { .timeConf(TimeUnit.MILLISECONDS) .createWithDefault(100) - val DISABLED_V2_FILE_DATA_SOURCE_READERS = buildConf("spark.sql.disabledV2FileDataSourceReaders") + val DISABLED_V2_FILE_READS = buildConf("spark.sql.files.disabledV2Reads") .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") + .doc("A comma-separated list of FileDataSourceV2 short names for which read paths " + + "are disabled. Reads from these sources will fall back to the V1 sources.") .stringConf .createWithDefault("") @@ -2009,7 +2009,7 @@ 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 disabledV2FileReads: String = getConf(DISABLED_V2_FILE_READS) def disabledV2StreamingWriters: String = getConf(DISABLED_V2_STREAMING_WRITERS) 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 d69fff6fe9650..e6aa94cd8ebcc 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,8 +192,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { "read files of Hive data source directly.") } - val disabledV2Readers = - sparkSession.sessionState.conf.disabledV2FileDataSourceReader.split(",") + val disabledV2Readers = sparkSession.sessionState.conf.disabledV2FileReads.split(",") val lookupCls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf) val cls = lookupCls.newInstance() match { case f: FileDataSourceV2 if disabledV2Readers.contains(f.shortName()) => f.fallBackFileFormat @@ -217,21 +216,26 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { table match { case _: SupportsBatchRead => Dataset.ofRows(sparkSession, DataSourceV2Relation.create(table, finalOptions)) - case _ => loadV1Source(paths: _*) + + // If the FileDataSourceV2 doesn't implement read path, fall back to FileFormat. + case fileSource: FileDataSourceV2 => + loadV1Source(fileSource.fallBackFileFormat.getCanonicalName, paths: _*) + + case _ => loadV1Source(cls.getCanonicalName, paths: _*) } } else { - loadV1Source(paths: _*) + loadV1Source(cls.getCanonicalName, paths: _*) } } - private def loadV1Source(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 = 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 228dcb94b9acc..c593734918a64 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 @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{AppendData, InsertIntoTable, 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.{DataSourceV2Relation, DataSourceV2Utils, WriteToDataSourceV2} +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2Utils, FileDataSourceV2, WriteToDataSourceV2} import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.sources.v2._ import org.apache.spark.sql.sources.v2.writer.SupportsSaveMode @@ -277,22 +277,26 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } } + // If the FileDataSourceV2 doesn't implement write path, fall back to FileFormat. + case fileSource: FileDataSourceV2 => + saveToV1Source(fileSource.fallBackFileFormat.getCanonicalName) + // 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() + case _ => saveToV1Source(cls.getCanonicalName) } } else { - saveToV1Source() + saveToV1Source(cls.getCanonicalName) } } - private def saveToV1Source(): Unit = { + private def saveToV1Source(className: String): Unit = { // Code path for data source v1. runCommand(df.sparkSession, "save") { DataSource( sparkSession = df.sparkSession, - className = source, + className = className, partitionColumns = partitioningColumns.getOrElse(Nil), options = extraOptions.toMap).planForWriting(mode, 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 809abaf771fef..6dfeb7876922e 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 @@ -94,19 +94,10 @@ case class DataSource( case class SourceInfo(name: String, schema: StructType, partitionColumns: Seq[String]) - 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 - case _ => cls - } - } + lazy val providingClass: Class[_] = + 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 => @@ -442,6 +433,7 @@ 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 5fc2d86f3a9f5..ff26393191f79 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 @@ -232,6 +232,26 @@ class FallBackFileDataSourceToV1(sparkSession: SparkSession) extends Rule[Logica val relation = HadoopFsRelation(table.getFileIndex, table.getFileIndex.partitionSchema, table.schema(), None, v1FileFormat, d.options)(sparkSession) i.copy(table = LogicalRelation(relation)) + + case c @CreateTable(tableDesc, _, _) if DDLUtils.isDatasourceTable(tableDesc) => + val cls = DataSource.lookupDataSource(tableDesc.provider.get, 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 => + val newTableDesc = tableDesc.copy(provider = Some(f.fallBackFileFormat.getCanonicalName)) + c.copy(tableDesc = newTableDesc) + case _ => c + } + + case c: CreateTempViewUsing => + val cls = DataSource.lookupDataSource(c.provider, sparkSession.sessionState.conf) + cls.newInstance() match { + case f: FileDataSourceV2 => + c.copy(provider = f.fallBackFileFormat.getCanonicalName) + case _ => c + } } } 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 5966dd8532965..067e24095aca1 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 @@ -432,7 +432,7 @@ class OrcFilterSuite extends OrcTest with SharedSQLContext { class OrcV1FilterSuite extends OrcFilterSuite { override protected def sparkConf: SparkConf = - super.sparkConf.set(SQLConf.DISABLED_V2_FILE_DATA_SOURCE_READERS, "orc") + super.sparkConf.set(SQLConf.DISABLED_V2_FILE_READS, "orc") override def checkFilterPredicate( df: DataFrame, 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 15e6500b894da..fe1be9030837e 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_FILE_DATA_SOURCE_READERS, "orc") + super.sparkConf.set(SQLConf.DISABLED_V2_FILE_READS, "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 a2f254326885c..959d6287cbdaa 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 @@ -684,5 +684,5 @@ class OrcQuerySuite extends OrcQueryTest with SharedSQLContext { class OrcV1QuerySuite extends OrcQuerySuite { override protected def sparkConf: SparkConf = - super.sparkConf.set(SQLConf.DISABLED_V2_FILE_DATA_SOURCE_READERS, "orc") + super.sparkConf.set(SQLConf.DISABLED_V2_FILE_READS, "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 index d5e9c811de3dc..bb081b6ed78df 100644 --- 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 @@ -61,12 +61,12 @@ class FileDataSourceV2FallBackSuite extends QueryTest with ParquetTest with Shar withTempPath { file => val path = file.getCanonicalPath df.write.parquet(path) - withSQLConf(SQLConf.DISABLED_V2_FILE_DATA_SOURCE_READERS.key -> "foo,parquet,bar") { + withSQLConf(SQLConf.DISABLED_V2_FILE_READS.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") { + withSQLConf(SQLConf.DISABLED_V2_FILE_READS.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) From 5dd34c49644a75dd9cd2c66581f00b5b51ea39ea Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Sun, 6 Jan 2019 22:11:16 +0800 Subject: [PATCH 03/31] address comments --- .../sql/execution/DataSourceScanExec.scala | 39 ------------------- .../spark/sql/execution/command/tables.scala | 2 +- .../datasources/orc/OrcFileFormat.scala | 2 +- .../datasources/orc/OrcFilters.scala | 7 +--- .../datasources/v2/FileDataSourceV2.scala | 11 ++---- .../datasources/v2/FilePartitionReader.scala | 34 ++++++++-------- .../streaming/MetadataLogFileIndex.scala | 1 - 7 files changed, 24 insertions(+), 72 deletions(-) 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 985daa5eeebba..7466700564c97 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 @@ -438,45 +438,6 @@ case class FileSourceScanExec( 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/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 314c52388f0ea..d24e66e583857 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 @@ -215,7 +215,7 @@ case class AlterTableAddColumnsCommand( /** * ALTER TABLE ADD COLUMNS command does not support temporary view/table, * view, or datasource table with text, orc formats or external provider. - * For datasource table, it currently only supports parquet, json, csv. + * For datasource table, it currently only supports parquet, json, csv, orc. */ private def verifyAlterTableAddColumn( conf: SQLConf, 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 14779cdba4178..5a72dbac047ec 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 @@ -81,7 +81,7 @@ class OrcFileFormat with DataSourceRegister with Serializable { - override def shortName(): String = "orc" + override def shortName(): String = "orcV1" override def toString: String = "ORC" 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 210eef4e4a7cc..ee545d19f21cf 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 @@ -82,14 +82,9 @@ private[sql] object OrcFilters { */ def createFilter(schema: StructType, filters: Seq[Filter]): Option[SearchArgument] = { val dataTypeMap = schema.map(f => f.name -> f.dataType).toMap - - // 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 convertible = convertibleFilters(schema, filters) - for { // Combines all convertible filters using `And` to produce a single conjunction - conjunction <- buildTree(convertible) + conjunction <- buildTree(convertibleFilters(schema, filters)) // Then tries to build a single ORC `SearchArgument` for the conjunction predicate builder <- buildSearchArgument(dataTypeMap, conjunction, newBuilder) } yield builder.build() 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 80e815e048197..2b9cfe0e681c9 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 @@ -31,16 +31,13 @@ trait FileDataSourceV2 extends TableProvider 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 [[SupportsBatchRead]] is implemented while [[SupportsBatchWrite]] 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. + * 1. File datasource V2 implementations cause regression. Users can disable the problematic data + * source via SQL configuration and fall back to FileFormat. + * 2. Catalog support is required, which is still under development for data source V2. */ def fallBackFileFormat: Class[_ <: FileFormat] - lazy val sparkSession = - SparkSession.getActiveSession.getOrElse(SparkSession.getDefaultSession.get) + lazy val sparkSession = SparkSession.active def getFileIndex( options: DataSourceOptions, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala index f588722040150..81709c137f85b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala @@ -24,52 +24,52 @@ import org.apache.spark.sql.sources.v2.reader.PartitionReader class FilePartitionReader[T]( readers: Iterator[PartitionReader[T]]) extends PartitionReader[T] with Logging { - private var currentFile: PartitionReader[T] = null + private var currentReader: PartitionReader[T] = null private val sqlConf = SQLConf.get - private val ignoreMissingFiles = sqlConf.ignoreMissingFiles - private val ignoreCorruptFiles = sqlConf.ignoreCorruptFiles + private def ignoreMissingFiles = sqlConf.ignoreMissingFiles + private def ignoreCorruptFiles = sqlConf.ignoreCorruptFiles override def next(): Boolean = { - if (currentFile == null) { + if (currentReader == null) { if (readers.hasNext) { if (ignoreMissingFiles || ignoreCorruptFiles) { try { - currentFile = readers.next() + currentReader = readers.next() } catch { case e: FileNotFoundException if ignoreMissingFiles => - logWarning(s"Skipped missing file: $currentFile", e) - currentFile = null + logWarning(s"Skipped missing file: $currentReader", e) + currentReader = 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 + s"Skipped the rest of the content in the corrupted file: $currentReader", e) + currentReader = null return false } } else { - currentFile = readers.next() + currentReader = readers.next() } } else { return false } } - if (currentFile.next()) { - return true + if (currentReader.next()) { + true } else { close() - currentFile = null + currentReader = null + next() } - next() } - override def get(): T = currentFile.get() + override def get(): T = currentReader.get() override def close(): Unit = { - if (currentFile != null) { - currentFile.close() + if (currentReader != null) { + currentReader.close() } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileIndex.scala index 89f1006fa51bf..5cacdd070b735 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileIndex.scala @@ -22,7 +22,6 @@ import scala.collection.mutable import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.types.StructType From 5f35e7555894d59072c90399653f9600122f0359 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Sun, 6 Jan 2019 22:18:57 +0800 Subject: [PATCH 04/31] remove partition pruning for now --- .../datasources/CatalogFileIndex.scala | 3 +- .../datasources/InMemoryFileIndex.scala | 43 +------------------ .../PartitioningAwareFileIndex.scala | 2 - .../PruneFileSourcePartitions.scala | 37 ---------------- 4 files changed, 3 insertions(+), 82 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CatalogFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CatalogFileIndex.scala index 93196b970bb19..a66a07673e25f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CatalogFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CatalogFileIndex.scala @@ -82,7 +82,7 @@ class CatalogFileIndex( val partitionSpec = PartitionSpec(partitionSchema, partitions) val timeNs = System.nanoTime() - startTime new PrunedInMemoryFileIndex( - sparkSession, fileStatusCache, partitionSpec, Option(timeNs)) + sparkSession, new Path(baseLocation.get), fileStatusCache, partitionSpec, Option(timeNs)) } else { new InMemoryFileIndex( sparkSession, rootPaths, table.storage.properties, userSpecifiedSchema = None) @@ -111,6 +111,7 @@ class CatalogFileIndex( */ private class PrunedInMemoryFileIndex( sparkSession: SparkSession, + tableBasePath: Path, fileStatusCache: FileStatusCache, override val partitionSpec: PartitionSpec, override val metadataOpsTimeNs: Option[Long]) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala index a6b7cee65c508..fe418e610da8f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala @@ -28,8 +28,7 @@ import org.apache.hadoop.mapred.{FileInputFormat, JobConf} import org.apache.spark.SparkContext import org.apache.spark.internal.Logging import org.apache.spark.metrics.source.HiveCatalogMetrics -import org.apache.spark.sql.{AnalysisException, SparkSession} -import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, InterpretedPredicate} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.streaming.FileStreamSink import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration @@ -133,46 +132,6 @@ class InMemoryFileIndex( } output } - - /** - * Returns a [[InMemoryFileIndex]] for this table restricted to the subset of partitions - * specified by the given partition-pruning filters. - * - * @param filters partition-pruning filters - */ - def filterPartitions(filters: Seq[Expression]): InMemoryFileIndex = { - if (partitionSchema.nonEmpty) { - val startTime = System.nanoTime() - val partitionColumnNames = partitionSchema.fields.map(_.name).toSet - - val nonPartitionPruningPredicates = filters.filterNot { - _.references.map(_.name).toSet.subsetOf(partitionColumnNames) - } - if (nonPartitionPruningPredicates.nonEmpty) { - throw new AnalysisException("Expected only partition pruning predicates: " + - nonPartitionPruningPredicates) - } - - val boundPredicate = - InterpretedPredicate.create(filters.reduce(And).transform { - case att: AttributeReference => - val index = partitionSchema.indexWhere(_.name == att.name) - BoundReference(index, partitionSchema(index).dataType, nullable = true) - }) - - val spec = partitionSpec() - val prunedPartitions = spec.partitions.filter { p => - boundPredicate.eval(p.values) - } - val prunedPartitionSpec = spec.copy(partitions = prunedPartitions) - - val timeNs = System.nanoTime() - startTime - new PrunedInMemoryFileIndex( - sparkSession, fileStatusCache, prunedPartitionSpec, Option(timeNs)) - } else { - this - } - } } object InMemoryFileIndex extends Logging { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala index 39e23550e8feb..5a2ac7a2a7744 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala @@ -55,8 +55,6 @@ abstract class PartitioningAwareFileIndex( def getSparkSession: SparkSession = this.sparkSession - def getUserSpecifiedSchema: Option[StructType] = this.userSpecifiedSchema - val hadoopConf: Configuration = sparkSession.sessionState.newHadoopConfWithOptions(parameters) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala index 61d60ae738f21..329b9539f52e3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala @@ -22,9 +22,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileTable} -import org.apache.spark.sql.sources.v2.SupportsBatchRead -import org.apache.spark.sql.types.StructType private[sql] object PruneFileSourcePartitions extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { @@ -77,39 +74,5 @@ private[sql] object PruneFileSourcePartitions extends Rule[LogicalPlan] { } else { op } - case op @ PhysicalOperation(projects, filters, - relation @DataSourceV2Relation(_, table: FileTable, output: Seq[AttributeReference], _, _)) - if filters.nonEmpty && table.getFileIndex.isInstanceOf[InMemoryFileIndex] && - ! table.getFileIndex.isInstanceOf[PrunedInMemoryFileIndex] - && table.getFileIndex.partitionSchema.nonEmpty => - // The attribute name of predicate could be different than the one in schema in case of - // case insensitive, we should change them to match the one in schema, so we donot need to - // worry about case sensitivity anymore. - val normalizedFilters = filters.filterNot(SubqueryExpression.hasSubquery).map { e => - e transform { - case a: AttributeReference => - a.withName(output.find(_.semanticEquals(a)).get.name) - } - } - val fileIndex = table.getFileIndex.asInstanceOf[InMemoryFileIndex] - val caseSensitive = fileIndex.getSparkSession.sessionState.conf.caseSensitiveAnalysis - val partitionColumns = - PartitioningUtils.partitionColumns(output, fileIndex.partitionSchema, caseSensitive) - val partitionSet = AttributeSet(partitionColumns) - val partitionKeyFilters = ExpressionSet(normalizedFilters - .filter(_.references.subsetOf(partitionSet))) - - if (partitionKeyFilters.nonEmpty) { - val prunedFileIndex = fileIndex.filterPartitions(partitionKeyFilters.toSeq) - val prunedTable = table.withNewFileIndex(prunedFileIndex) - val prunedRelation = relation.copy(table = prunedTable) - // Todo: Change table stats based on the sizeInBytes of pruned files - // Keep partition-pruning predicates so that they are visible in physical planning - val filterExpression = filters.reduceLeft(And) - val filter = Filter(filterExpression, prunedRelation) - Project(projects, filter) - } else { - op - } } } From cdb1f997d2c15b32544fcaa627bf37dda61ed0f5 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Mon, 7 Jan 2019 20:22:40 +0800 Subject: [PATCH 05/31] fix hive tests --- .../spark/sql/execution/command/ddl.scala | 3 +- .../sql/execution/datasources/rules.scala | 48 ++++++++++++++++--- .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- .../sql/hive/execution/SQLQuerySuite.scala | 2 +- 4 files changed, 45 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 096481f68275d..8ea2185af7c01 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -889,7 +889,8 @@ object DDLUtils { ParquetSchemaConverter.checkFieldNames(colNames) } case "parquet" => ParquetSchemaConverter.checkFieldNames(colNames) - case "orc" => OrcFileFormat.checkFieldNames(colNames) + case "orc" | "org.apache.spark.sql.execution.datasources.orc.orcfileformat" => + OrcFileFormat.checkFieldNames(colNames) case _ => } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 949aa665527ac..d9892d45d1fcd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -19,15 +19,21 @@ package org.apache.spark.sql.execution.datasources import java.util.Locale -import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession} +import scala.collection.JavaConverters._ + +import com.fasterxml.jackson.databind.ObjectMapper + +import org.apache.spark.sql.{AnalysisException, Dataset, SaveMode, SparkSession} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, Expression, InputFileBlockLength, InputFileBlockStart, InputFileName, RowOrdering} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.command.DDLUtils +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2Utils, FileDataSourceV2} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.InsertableRelation +import org.apache.spark.sql.sources.v2.{DataSourceOptions, SupportsBatchRead, TableProvider} import org.apache.spark.sql.types.{AtomicType, StructType} import org.apache.spark.sql.util.SchemaUtils @@ -50,13 +56,41 @@ class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] { // `dataSource.providingClass` may throw ClassNotFoundException, then the outer try-catch // will catch it and return the original plan, so that the analyzer can report table not // found later. - val isFileFormat = classOf[FileFormat].isAssignableFrom(dataSource.providingClass) - if (!isFileFormat || - dataSource.className.toLowerCase(Locale.ROOT) == DDLUtils.HIVE_PROVIDER) { - throw new AnalysisException("Unsupported data source type for direct query on files: " + - s"${u.tableIdentifier.database.get}") + val message = "Unsupported data source type for direct query on files: " + + s"${u.tableIdentifier.database.get}" + if (dataSource.className.toLowerCase(Locale.ROOT) == DDLUtils.HIVE_PROVIDER) { + throw new AnalysisException(message) + } + if (classOf[TableProvider].isAssignableFrom(dataSource.providingClass)) { + // TODO: refacotor and reduce duplicated code with DataFrameReader.load(). + val provider = + dataSource.providingClass.getConstructor().newInstance().asInstanceOf[TableProvider] + val sessionOptions = DataSourceV2Utils.extractSessionConfigs( + ds = provider, conf = sparkSession.sessionState.conf) + val pathsOption = { + val objectMapper = new ObjectMapper() + DataSourceOptions.PATHS_KEY -> + objectMapper.writeValueAsString(Array(u.tableIdentifier.table)) + } + val finalOptions = sessionOptions + pathsOption + val dsOptions = new DataSourceOptions(finalOptions.asJava) + val table = provider.getTable(dsOptions) + table match { + case s: SupportsBatchRead => + DataSourceV2Relation.create(provider, s, finalOptions, userSpecifiedSchema = None) + + // If the FileDataSourceV2 doesn't implement read path, fall back to FileFormat. + case fileSource: FileDataSourceV2 => + LogicalRelation(dataSource.copy(className = + fileSource.fallBackFileFormat.getCanonicalName).resolveRelation()) + + case _ => throw new AnalysisException(message) + } + } else if (classOf[FileFormat].isAssignableFrom(dataSource.providingClass)) { + LogicalRelation(dataSource.resolveRelation()) + } else { + throw new AnalysisException(message) } - LogicalRelation(dataSource.resolveRelation()) } catch { case _: ClassNotFoundException => u case e: Exception => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 03f4b8d83e353..300d2a7dddbd1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -242,7 +242,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log userSpecifiedSchema = Option(updatedTable.dataSchema), bucketSpec = None, options = options, - className = fileType).resolveRelation(), + className = fileFormatClass.getCanonicalName).resolveRelation(), table = updatedTable) catalogProxy.cacheTable(tableIdentifier, created) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 70efad103d13e..34aeedf6ca6f6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -428,7 +428,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { assert(r.options("path") === location) case None => // OK. } - assert(catalogTable.provider.get === format) + assert(catalogTable.provider.get.toLowerCase(Locale.ROOT).contains(format)) case r: HiveTableRelation => if (isDataSourceTable) { From c1e1f4b442ffe9ded7445029eb50f8878894e935 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Tue, 8 Jan 2019 15:46:10 +0800 Subject: [PATCH 06/31] add comment and revise --- .../datasources/v2/orc/OrcPartitionReaderFactory.scala | 6 +++--- .../spark/sql/execution/datasources/v2/orc/OrcScan.scala | 3 --- .../test/scala/org/apache/spark/sql/test/SQLTestUtils.scala | 5 +++-- 3 files changed, 6 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala index 33f26065baf07..6082b1a9fe890 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala @@ -49,9 +49,9 @@ case class OrcPartitionReaderFactory( partitionSchema: StructType) extends FilePartitionReaderFactory { private val sqlConf = SQLConf.get private val isCaseSensitive = sqlConf.caseSensitiveAnalysis - val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled - val copyToSpark = sqlConf.getConf(SQLConf.ORC_COPY_BATCH_TO_SPARK) - val capacity = sqlConf.orcVectorizedReaderBatchSize + private val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled + private val copyToSpark = sqlConf.getConf(SQLConf.ORC_COPY_BATCH_TO_SPARK) + private val capacity = sqlConf.orcVectorizedReaderBatchSize override def supportColumnarReads(partition: InputPartition): Boolean = { sqlConf.orcVectorizedReaderEnabled && sqlConf.wholeStageEnabled && diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala index 144f5c6da57a2..2828e4d65e184 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala @@ -16,13 +16,10 @@ */ package org.apache.spark.sql.execution.datasources.v2.orc -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.FileScan -import org.apache.spark.sql.sources.v2.DataSourceOptions import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index f914b9d64d731..5940916055442 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -230,11 +230,12 @@ private[sql] trait SQLTestUtilsBase SparkSession.setActiveSession(spark) val sc = spark.sparkContext // Set all the specified SQL configs to local properties, so that they can be available at - // the executor side. + // the executor side. This is because DataFrame.rdd() doesn't propagate SQL configs. Before + // the issue is fixed, set the SparkContext's properties here to unblock related tests. // Note: `allSparkConfigs` may `overlap` with pairs, so use `.toMap` to reduce duplicated keys. val allSparkConfigs = (spark.sessionState.conf.getAllConfs ++ pairs.toSeq).filter { case (key, _) => key.startsWith("spark") - }.toMap + } val originalLocalProps = allSparkConfigs.map { case (key, value) => val originalValue = sc.getLocalProperty(key) From ba66011f4e56161ae3ceccb329d332ce056911a2 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Tue, 8 Jan 2019 19:59:17 +0800 Subject: [PATCH 07/31] fix test failure --- .../apache/spark/sql/streaming/DataStreamWriter.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index 5733258a6b310..59c426bafc1ae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.streaming.InternalOutputModes import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.DataSource -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Utils, FileDataSourceV2} import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger import org.apache.spark.sql.execution.streaming.sources._ @@ -314,6 +314,14 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { w, df.sparkSession.sessionState.conf) options = sessionOptions ++ extraOptions w + // TODO: remove this when file source write path is finished. + case f: FileDataSourceV2 => + val ds = DataSource( + df.sparkSession, + className = f.fallBackFileFormat.getCanonicalName, + options = options, + partitionColumns = normalizedParCols.getOrElse(Nil)) + ds.createSink(outputMode) case _ => val ds = DataSource( df.sparkSession, From 91b146b412ddc24ed9ba4bcb79dff071deed2c2b Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Tue, 8 Jan 2019 22:16:13 +0800 Subject: [PATCH 08/31] fix checkAnswer --- .../org/apache/spark/sql/QueryTest.scala | 5 +++- .../apache/spark/sql/test/SQLTestUtils.scala | 23 +------------------ 2 files changed, 5 insertions(+), 23 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index cf25f1ce910db..d83deb17a0900 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -23,6 +23,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.columnar.InMemoryRelation @@ -234,7 +235,9 @@ object QueryTest { checkToRDD: Boolean = true): Option[String] = { val isSorted = df.logicalPlan.collect { case s: logical.Sort => s }.nonEmpty if (checkToRDD) { - df.rdd.count() // Also attempt to deserialize as an RDD [SPARK-15791] + SQLExecution.withSQLConfPropagated(df.sparkSession) { + df.rdd.count() // Also attempt to deserialize as an RDD [SPARK-15791] + } } val sparkAnswer = try df.collect().toSeq catch { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index 5940916055442..e918700732ba9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -228,28 +228,7 @@ private[sql] trait SQLTestUtilsBase protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { SparkSession.setActiveSession(spark) - val sc = spark.sparkContext - // Set all the specified SQL configs to local properties, so that they can be available at - // the executor side. This is because DataFrame.rdd() doesn't propagate SQL configs. Before - // the issue is fixed, set the SparkContext's properties here to unblock related tests. - // Note: `allSparkConfigs` may `overlap` with pairs, so use `.toMap` to reduce duplicated keys. - val allSparkConfigs = (spark.sessionState.conf.getAllConfs ++ pairs.toSeq).filter { - case (key, _) => key.startsWith("spark") - } - val originalLocalProps = allSparkConfigs.map { - case (key, value) => - val originalValue = sc.getLocalProperty(key) - sc.setLocalProperty(key, value) - (key, originalValue) - } - - try { - super.withSQLConf(pairs: _*)(f) - } finally { - for ((key, value) <- originalLocalProps) { - sc.setLocalProperty(key, value) - } - } + super.withSQLConf(pairs: _*)(f) } /** From 91475cae5c12a7a171d2ee770cf9b1799d04185d Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Wed, 9 Jan 2019 01:11:09 +0800 Subject: [PATCH 09/31] remove useless code --- .../src/main/scala/org/apache/spark/sql/DataFrameReader.scala | 4 ---- .../src/main/scala/org/apache/spark/sql/DataFrameWriter.scala | 3 ++- 2 files changed, 2 insertions(+), 5 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 e6aa94cd8ebcc..8642cdd8facfd 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 @@ -217,10 +217,6 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { case _: SupportsBatchRead => Dataset.ofRows(sparkSession, DataSourceV2Relation.create(table, finalOptions)) - // If the FileDataSourceV2 doesn't implement read path, fall back to FileFormat. - case fileSource: FileDataSourceV2 => - loadV1Source(fileSource.fallBackFileFormat.getCanonicalName, paths: _*) - case _ => loadV1Source(cls.getCanonicalName, paths: _*) } } else { 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 c593734918a64..b33b8997b9e81 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 @@ -277,7 +277,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } } - // If the FileDataSourceV2 doesn't implement write path, fall back to FileFormat. + // Since the FileDataSourceV2 doesn't implement write path, fall back to FileFormat. + // TODO: remove this when FileDataSourceV2 write path is implemented. case fileSource: FileDataSourceV2 => saveToV1Source(fileSource.fallBackFileFormat.getCanonicalName) From 38a27f2fc90e913c4f600cc4cf0bf2f8ee0b86cc Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Wed, 9 Jan 2019 19:04:03 +0800 Subject: [PATCH 10/31] use DataSource.providingClass for falling back to V1 when with catalog --- .../apache/spark/sql/DataFrameReader.scala | 8 ++-- .../apache/spark/sql/DataFrameWriter.scala | 13 ++--- .../spark/sql/execution/command/ddl.scala | 3 +- .../execution/datasources/DataSource.scala | 15 ++++-- .../datasources/DataSourceStrategy.scala | 20 -------- .../sql/execution/datasources/rules.scala | 48 +++---------------- .../apache/spark/sql/internal/HiveSerDe.scala | 1 - .../sql/streaming/DataStreamWriter.scala | 10 +--- .../apache/spark/sql/test/SQLTestUtils.scala | 3 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- .../sql/hive/execution/SQLQuerySuite.scala | 2 +- 11 files changed, 32 insertions(+), 93 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 8642cdd8facfd..3cd924379bf00 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 @@ -217,21 +217,21 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { case _: SupportsBatchRead => Dataset.ofRows(sparkSession, DataSourceV2Relation.create(table, finalOptions)) - case _ => loadV1Source(cls.getCanonicalName, paths: _*) + case _ => loadV1Source(paths: _*) } } else { - loadV1Source(cls.getCanonicalName, paths: _*) + loadV1Source(paths: _*) } } - private def loadV1Source(className: String, paths: String*) = { + private def loadV1Source(paths: String*) = { // Code path for data source v1. sparkSession.baseRelationToDataFrame( DataSource.apply( sparkSession, paths = paths, userSpecifiedSchema = userSpecifiedSchema, - className = className, + className = 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 b33b8997b9e81..54fb6d486f296 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 @@ -277,27 +277,22 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } } - // Since the FileDataSourceV2 doesn't implement write path, fall back to FileFormat. - // TODO: remove this when FileDataSourceV2 write path is implemented. - case fileSource: FileDataSourceV2 => - saveToV1Source(fileSource.fallBackFileFormat.getCanonicalName) - // 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(cls.getCanonicalName) + case _ => saveToV1Source() } } else { - saveToV1Source(cls.getCanonicalName) + saveToV1Source() } } - private def saveToV1Source(className: String): Unit = { + private def saveToV1Source(): Unit = { // Code path for data source v1. runCommand(df.sparkSession, "save") { DataSource( sparkSession = df.sparkSession, - className = className, + className = source, partitionColumns = partitioningColumns.getOrElse(Nil), options = extraOptions.toMap).planForWriting(mode, df.logicalPlan) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 8ea2185af7c01..096481f68275d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -889,8 +889,7 @@ object DDLUtils { ParquetSchemaConverter.checkFieldNames(colNames) } case "parquet" => ParquetSchemaConverter.checkFieldNames(colNames) - case "orc" | "org.apache.spark.sql.execution.datasources.orc.orcfileformat" => - OrcFileFormat.checkFieldNames(colNames) + case "orc" => OrcFileFormat.checkFieldNames(colNames) case _ => } } 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 6dfeb7876922e..6754f789fc115 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,7 +18,6 @@ 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} @@ -94,8 +93,18 @@ 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) + // `providingClass` is used for resolving data source relation for catalog tables. + // As now catalog for data source V2 is under development, here we fall back all the + // [[FileDataSourceV2]] to [[FileFormat]] to guarantee the current catalog works. + // [[FileDataSourceV2]] will still be used if we call the load()/save() method in + // [[DataFrameReader]]/[[DataFrameWriter]]. + cls.newInstance() match { + case f: FileDataSourceV2 => f.fallBackFileFormat + case _ => cls + } + } lazy val sourceInfo: SourceInfo = sourceSchema() private val caseInsensitiveOptions = CaseInsensitiveMap(options) private val equality = sparkSession.sessionState.conf.resolver 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 ff26393191f79..5fc2d86f3a9f5 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 @@ -232,26 +232,6 @@ class FallBackFileDataSourceToV1(sparkSession: SparkSession) extends Rule[Logica val relation = HadoopFsRelation(table.getFileIndex, table.getFileIndex.partitionSchema, table.schema(), None, v1FileFormat, d.options)(sparkSession) i.copy(table = LogicalRelation(relation)) - - case c @CreateTable(tableDesc, _, _) if DDLUtils.isDatasourceTable(tableDesc) => - val cls = DataSource.lookupDataSource(tableDesc.provider.get, 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 => - val newTableDesc = tableDesc.copy(provider = Some(f.fallBackFileFormat.getCanonicalName)) - c.copy(tableDesc = newTableDesc) - case _ => c - } - - case c: CreateTempViewUsing => - val cls = DataSource.lookupDataSource(c.provider, sparkSession.sessionState.conf) - cls.newInstance() match { - case f: FileDataSourceV2 => - c.copy(provider = f.fallBackFileFormat.getCanonicalName) - case _ => c - } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index d9892d45d1fcd..949aa665527ac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -19,21 +19,15 @@ package org.apache.spark.sql.execution.datasources import java.util.Locale -import scala.collection.JavaConverters._ - -import com.fasterxml.jackson.databind.ObjectMapper - -import org.apache.spark.sql.{AnalysisException, Dataset, SaveMode, SparkSession} +import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, Expression, InputFileBlockLength, InputFileBlockStart, InputFileName, RowOrdering} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.command.DDLUtils -import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2Utils, FileDataSourceV2} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.InsertableRelation -import org.apache.spark.sql.sources.v2.{DataSourceOptions, SupportsBatchRead, TableProvider} import org.apache.spark.sql.types.{AtomicType, StructType} import org.apache.spark.sql.util.SchemaUtils @@ -56,41 +50,13 @@ class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] { // `dataSource.providingClass` may throw ClassNotFoundException, then the outer try-catch // will catch it and return the original plan, so that the analyzer can report table not // found later. - val message = "Unsupported data source type for direct query on files: " + - s"${u.tableIdentifier.database.get}" - if (dataSource.className.toLowerCase(Locale.ROOT) == DDLUtils.HIVE_PROVIDER) { - throw new AnalysisException(message) - } - if (classOf[TableProvider].isAssignableFrom(dataSource.providingClass)) { - // TODO: refacotor and reduce duplicated code with DataFrameReader.load(). - val provider = - dataSource.providingClass.getConstructor().newInstance().asInstanceOf[TableProvider] - val sessionOptions = DataSourceV2Utils.extractSessionConfigs( - ds = provider, conf = sparkSession.sessionState.conf) - val pathsOption = { - val objectMapper = new ObjectMapper() - DataSourceOptions.PATHS_KEY -> - objectMapper.writeValueAsString(Array(u.tableIdentifier.table)) - } - val finalOptions = sessionOptions + pathsOption - val dsOptions = new DataSourceOptions(finalOptions.asJava) - val table = provider.getTable(dsOptions) - table match { - case s: SupportsBatchRead => - DataSourceV2Relation.create(provider, s, finalOptions, userSpecifiedSchema = None) - - // If the FileDataSourceV2 doesn't implement read path, fall back to FileFormat. - case fileSource: FileDataSourceV2 => - LogicalRelation(dataSource.copy(className = - fileSource.fallBackFileFormat.getCanonicalName).resolveRelation()) - - case _ => throw new AnalysisException(message) - } - } else if (classOf[FileFormat].isAssignableFrom(dataSource.providingClass)) { - LogicalRelation(dataSource.resolveRelation()) - } else { - throw new AnalysisException(message) + val isFileFormat = classOf[FileFormat].isAssignableFrom(dataSource.providingClass) + if (!isFileFormat || + dataSource.className.toLowerCase(Locale.ROOT) == DDLUtils.HIVE_PROVIDER) { + throw new AnalysisException("Unsupported data source type for direct query on files: " + + s"${u.tableIdentifier.database.get}") } + LogicalRelation(dataSource.resolveRelation()) } catch { case _: ClassNotFoundException => u case e: Exception => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala index bd25a6437033c..1363b460fc87b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala @@ -77,7 +77,6 @@ object HiveSerDe { case s if s.startsWith("org.apache.spark.sql.execution.datasources.parquet") => "parquet" case s if s.startsWith("org.apache.spark.sql.orc") => "orc" case s if s.startsWith("org.apache.spark.sql.hive.orc") => "orc" - case s if s.startsWith("org.apache.spark.sql.execution.datasources.orc") => "orc" case s if s.equals("orcfile") => "orc" case s if s.equals("parquetfile") => "parquet" case s if s.equals("avrofile") => "avro" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index 59c426bafc1ae..5733258a6b310 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.streaming.InternalOutputModes import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.DataSource -import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Utils, FileDataSourceV2} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger import org.apache.spark.sql.execution.streaming.sources._ @@ -314,14 +314,6 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { w, df.sparkSession.sessionState.conf) options = sessionOptions ++ extraOptions w - // TODO: remove this when file source write path is finished. - case f: FileDataSourceV2 => - val ds = DataSource( - df.sparkSession, - className = f.fallBackFileFormat.getCanonicalName, - options = options, - partitionColumns = normalizedParCols.getOrElse(Nil)) - ds.createSink(outputMode) case _ => val ds = DataSource( df.sparkSession, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index e918700732ba9..85963ec4ca699 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -24,7 +24,6 @@ import java.util.{Locale, UUID} import scala.concurrent.duration._ import scala.language.implicitConversions -import scala.util.Try import scala.util.control.NonFatal import org.apache.hadoop.fs.Path @@ -40,7 +39,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.PlanTestBase import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.execution.{FilterExec, SQLExecution} +import org.apache.spark.sql.execution.FilterExec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.UninterruptibleThread import org.apache.spark.util.Utils diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 300d2a7dddbd1..03f4b8d83e353 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -242,7 +242,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log userSpecifiedSchema = Option(updatedTable.dataSchema), bucketSpec = None, options = options, - className = fileFormatClass.getCanonicalName).resolveRelation(), + className = fileType).resolveRelation(), table = updatedTable) catalogProxy.cacheTable(tableIdentifier, created) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 34aeedf6ca6f6..70efad103d13e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -428,7 +428,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { assert(r.options("path") === location) case None => // OK. } - assert(catalogTable.provider.get.toLowerCase(Locale.ROOT).contains(format)) + assert(catalogTable.provider.get === format) case r: HiveTableRelation => if (isDataSourceTable) { From 9bbd87bf325e1e3cd19b5304d6be8d6621828ba1 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Wed, 9 Jan 2019 20:03:37 +0800 Subject: [PATCH 11/31] address comments --- .../spark/sql/execution/datasources/DataSource.scala | 5 +++-- .../sql/execution/datasources/DataSourceStrategy.scala | 8 +++----- .../datasources/orc/OrcPartitionDiscoverySuite.scala | 2 +- 3 files changed, 7 insertions(+), 8 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 6754f789fc115..d48261e783dc9 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 @@ -99,7 +99,8 @@ case class DataSource( // As now catalog for data source V2 is under development, here we fall back all the // [[FileDataSourceV2]] to [[FileFormat]] to guarantee the current catalog works. // [[FileDataSourceV2]] will still be used if we call the load()/save() method in - // [[DataFrameReader]]/[[DataFrameWriter]]. + // [[DataFrameReader]]/[[DataFrameWriter]], since they use method `lookupDataSource` + // instead of `providingClass`. cls.newInstance() match { case f: FileDataSourceV2 => f.fallBackFileFormat case _ => cls @@ -614,7 +615,7 @@ object DataSource extends Logging { val provider1 = backwardCompatibilityMap.getOrElse(provider, provider) match { case name if name.equalsIgnoreCase("orc") && conf.getConf(SQLConf.ORC_IMPLEMENTATION) == "native" => - 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 5fc2d86f3a9f5..937cb60df1aa7 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,9 +19,6 @@ package org.apache.spark.sql.execution.datasources import java.util.Locale import java.util.concurrent.Callable -import javax.activation.FileDataSource - -import scala.collection.JavaConverters._ import org.apache.hadoop.fs.Path @@ -222,11 +219,12 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast * 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. + * This is a temporary hack for making current data source V2 work. It should be removed + * when catalog of data source v2 is finished. */ class FallBackFileDataSourceToV1(sparkSession: SparkSession) extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case i @InsertIntoTable(d @ + case i @ InsertIntoTable(d @ DataSourceV2Relation(source: FileDataSourceV2, table: FileTable, _, _, _), _, _, _, _) => val v1FileFormat = source.fallBackFileFormat.getConstructor().newInstance() val relation = HadoopFsRelation(table.getFileIndex, table.getFileIndex.partitionSchema, 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 fe1be9030837e..63e78fa6ba2ef 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 @@ -76,7 +76,7 @@ abstract class OrcPartitionDiscoveryTest extends OrcTest { ps <- Seq("foo", "bar") } yield Row(i, i.toString, pi, ps)) - checkAnswer( + checkAnswer( sql("SELECT intField, pi FROM t"), for { i <- 1 to 10 From 38b8af551bbe48bbb13a533bca98f6789ff41910 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Wed, 9 Jan 2019 21:28:19 +0800 Subject: [PATCH 12/31] revise comment --- .../spark/sql/execution/datasources/DataSourceStrategy.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 937cb60df1aa7..ada7aa72f31f6 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 @@ -220,7 +220,7 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast * 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. It should be removed - * when catalog of data source v2 is finished. + * when write path of file data source v2 is finished. */ class FallBackFileDataSourceToV1(sparkSession: SparkSession) extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { From c91b84aa54bd760412cddb04fa1d772cc19da3db Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Thu, 10 Jan 2019 00:11:54 +0800 Subject: [PATCH 13/31] don't use FileIndex to get SparkSession, hadoopconf, and data source options --- .../PartitioningAwareFileIndex.scala | 8 +------- .../execution/datasources/v2/FileScan.scala | 12 +++--------- .../datasources/v2/FileScanBuilder.scala | 3 +-- .../execution/datasources/v2/FileTable.scala | 9 +++------ .../datasources/v2/orc/OrcDataSourceV2.scala | 14 +++++++++++--- .../datasources/v2/orc/OrcScan.scala | 11 +++++++---- .../datasources/v2/orc/OrcScanBuilder.scala | 19 +++++++++++++++---- .../datasources/v2/orc/OrcTable.scala | 13 +++++++------ 8 files changed, 48 insertions(+), 41 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala index 5a2ac7a2a7744..b2e4155e6f49e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution.datasources -import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.hadoop.conf.Configuration @@ -28,7 +27,6 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{expressions, InternalRow} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} -import org.apache.spark.sql.sources.v2.DataSourceOptions import org.apache.spark.sql.types.{StringType, StructType} /** @@ -51,11 +49,7 @@ abstract class PartitioningAwareFileIndex( override def partitionSchema: StructType = partitionSpec().partitionColumns - def dataSourceOptions(): DataSourceOptions = new DataSourceOptions(parameters.asJava) - - def getSparkSession: SparkSession = this.sparkSession - - val hadoopConf: Configuration = + protected val hadoopConf: Configuration = sparkSession.sessionState.newHadoopConfWithOptions(parameters) protected def leafFiles: mutable.LinkedHashMap[Path, FileStatus] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala index 3d812738bfa98..4246c28197eb6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala @@ -16,22 +16,17 @@ */ package org.apache.spark.sql.execution.datasources.v2 -import scala.collection.JavaConverters._ - -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.SparkSession 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.{Batch, InputPartition, Scan} import org.apache.spark.sql.types.StructType abstract class FileScan( - fileIndex: PartitioningAwareFileIndex, - schema: StructType, - readSchema: StructType) extends Scan with Batch { + sparkSession: SparkSession, + fileIndex: PartitioningAwareFileIndex) extends Scan with Batch { /** * Returns whether a file with `path` could be split or not. */ @@ -40,7 +35,6 @@ abstract class FileScan( } protected def partitions: Seq[FilePartition] = { - val sparkSession = fileIndex.getSparkSession val selectedPartitions = fileIndex.listFiles(Seq.empty, Seq.empty) val maxSplitBytes = PartitionedFileUtil.maxSplitBytes(sparkSession, selectedPartitions) val splitFiles = selectedPartitions.flatMap { partition => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala index d7f16bc2cc7fb..5dd343ba44b62 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala @@ -16,11 +16,10 @@ */ package org.apache.spark.sql.execution.datasources.v2 -import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.sources.v2.reader.{ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns} import org.apache.spark.sql.types.StructType -abstract class FileScanBuilder(fileIndex: PartitioningAwareFileIndex, schema: StructType) +abstract class FileScanBuilder(schema: StructType) extends ScanBuilder with SupportsPushDownRequiredColumns with SupportsPushDownFilters { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala index 699c41268ffba..b1786541a8051 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala @@ -18,18 +18,17 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.hadoop.fs.FileStatus -import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources.v2.{SupportsBatchRead, Table} import org.apache.spark.sql.types.StructType abstract class FileTable( + sparkSession: SparkSession, fileIndex: PartitioningAwareFileIndex, userSpecifiedSchema: Option[StructType]) extends Table with SupportsBatchRead { def getFileIndex: PartitioningAwareFileIndex = this.fileIndex - override def name(): String = fileIndex.dataSourceOptions().paths().mkString(";") - lazy val dataSchema: StructType = userSpecifiedSchema.orElse { inferSchema(fileIndex.allFiles()) }.getOrElse { @@ -38,7 +37,7 @@ abstract class FileTable( }.asNullable override def schema(): StructType = { - val caseSensitive = fileIndex.getSparkSession.sessionState.conf.caseSensitiveAnalysis + val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis PartitioningUtils.mergeDataAndPartitionSchema(dataSchema, fileIndex.partitionSchema, caseSensitive)._1 } @@ -49,6 +48,4 @@ abstract class FileTable( * Spark will require that user specify the schema manually. */ def inferSchema(files: Seq[FileStatus]): Option[StructType] - - def withNewFileIndex(newFileIndex: PartitioningAwareFileIndex): FileTable } 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 7b823fb1f3519..25e6719c24171 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,10 +28,18 @@ class OrcDataSourceV2 extends FileDataSourceV2 { override def shortName(): String = "orc" + private def geTableName(options: DataSourceOptions): String = { + shortName() + ":" + options.paths().mkString(";") + } override def getTable(options: DataSourceOptions): Table = { - OrcTable(getFileIndex(options, None), None) + val tableName = geTableName(options) + val fileIndex = getFileIndex(options, None) + OrcTable(tableName, sparkSession, fileIndex, None) } - override def getTable(options: DataSourceOptions, schema: StructType): Table = - OrcTable(getFileIndex(options, Some(schema)), Some(schema)) + override def getTable(options: DataSourceOptions, schema: StructType): Table = { + val tableName = geTableName(options) + val fileIndex = getFileIndex(options, Some(schema)) + OrcTable(tableName, sparkSession, fileIndex, Some(schema)) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala index 2828e4d65e184..51776c1945aa2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala @@ -16,8 +16,10 @@ */ package org.apache.spark.sql.execution.datasources.v2.orc +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.FileScan import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory @@ -25,16 +27,17 @@ import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration case class OrcScan( + sparkSession: SparkSession, + hadoopConf: Configuration, fileIndex: PartitioningAwareFileIndex, schema: StructType, dataSchema: StructType, - readSchema: StructType) - extends FileScan(fileIndex, schema, readSchema) { + readSchema: StructType) extends FileScan(sparkSession, fileIndex) { override def isSplitable(path: Path): Boolean = true override def createReaderFactory(): PartitionReaderFactory = { - val broadcastedConf = fileIndex.getSparkSession.sparkContext.broadcast( - new SerializableConfiguration(fileIndex.hadoopConf)) + val broadcastedConf = sparkSession.sparkContext.broadcast( + new SerializableConfiguration(hadoopConf)) OrcPartitionReaderFactory(broadcastedConf, schema, dataSchema, readSchema, fileIndex.partitionSchema) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala index 6c967c71b8dca..51c098ba7a366 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala @@ -16,27 +16,38 @@ */ package org.apache.spark.sql.execution.datasources.v2.orc +import scala.collection.JavaConverters._ + import org.apache.orc.mapreduce.OrcInputFormat +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.orc.OrcFilters import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.sources.v2.DataSourceOptions import org.apache.spark.sql.sources.v2.reader.Scan import org.apache.spark.sql.types.StructType case class OrcScanBuilder( + sparkSession: SparkSession, fileIndex: PartitioningAwareFileIndex, schema: StructType, - dataSchema: StructType) extends FileScanBuilder(fileIndex, schema) { - override def build(): Scan = OrcScan(fileIndex, schema, dataSchema, readSchema) + dataSchema: StructType, + options: DataSourceOptions) extends FileScanBuilder(schema) { + lazy val hadoopConf = + sparkSession.sessionState.newHadoopConfWithOptions(options.asMap().asScala.toMap) + + override def build(): Scan = { + OrcScan(sparkSession, hadoopConf, fileIndex, schema, dataSchema, readSchema) + } private var _pushedFilters: Array[Filter] = Array.empty override def pushFilters(filters: Array[Filter]): Array[Filter] = { - if (fileIndex.getSparkSession.sessionState.conf.orcFilterPushDown) { + if (sparkSession.sessionState.conf.orcFilterPushDown) { OrcFilters.createFilter(schema, filters).foreach { f => - OrcInputFormat.setSearchArgument(fileIndex.hadoopConf, f, schema.fieldNames) + OrcInputFormat.setSearchArgument(hadoopConf, f, schema.fieldNames) } _pushedFilters = OrcFilters.convertibleFilters(schema, filters).toArray } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala index 8ee5045501b8f..719e757c33cbd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.datasources.v2.orc import org.apache.hadoop.fs.FileStatus +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.orc.OrcUtils import org.apache.spark.sql.execution.datasources.v2.FileTable @@ -25,14 +26,14 @@ import org.apache.spark.sql.sources.v2.DataSourceOptions import org.apache.spark.sql.types.StructType case class OrcTable( + name: String, + sparkSession: SparkSession, fileIndex: PartitioningAwareFileIndex, - userSpecifiedSchema: Option[StructType]) extends FileTable(fileIndex, userSpecifiedSchema) { + userSpecifiedSchema: Option[StructType]) + extends FileTable(sparkSession, fileIndex, userSpecifiedSchema) { override def newScanBuilder(options: DataSourceOptions): OrcScanBuilder = - new OrcScanBuilder(fileIndex, schema, dataSchema) + new OrcScanBuilder(sparkSession, fileIndex, schema, dataSchema, options) override def inferSchema(files: Seq[FileStatus]): Option[StructType] = - OrcUtils.readSchema(fileIndex.getSparkSession, files) - - override def withNewFileIndex(newFileIndex: PartitioningAwareFileIndex): FileTable = - copy(fileIndex = newFileIndex) + OrcUtils.readSchema(sparkSession, files) } From 2c43797bfafae97fe97823a796ab7f4d30918320 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Thu, 10 Jan 2019 19:28:52 +0800 Subject: [PATCH 14/31] move method to orc factory --- .../datasources/PartitioningUtils.scala | 39 +------------------ .../v2/orc/OrcPartitionReaderFactory.scala | 26 ++++++++++--- 2 files changed, 22 insertions(+), 43 deletions(-) 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 4e2fbcb26c6ca..a2e08180cc50e 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 @@ -557,18 +557,6 @@ object PartitioningUtils { }).asNullable } - def partitionColumns( - output: Seq[AttributeReference], - partitionSchema: StructType, - caseSensitive: Boolean): Seq[AttributeReference] = { - val equality = columnNameEquality(caseSensitive) - partitionSchema.fields.map { col => - output.find(a => equality(a.name, col.name)).getOrElse { - throw new AnalysisException(s"Partition column `$col` not found in $output") - } - } - } - def mergeDataAndPartitionSchema( dataSchema: StructType, partitionSchema: StructType, @@ -590,32 +578,7 @@ object PartitioningUtils { (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) - } - - private def getColName(f: StructField, caseSensitive: Boolean): String = { + def getColName(f: StructField, caseSensitive: Boolean): String = { if (caseSensitive) { f.name } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala index 6082b1a9fe890..32e3415690211 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala @@ -85,8 +85,7 @@ case class OrcPartitionReaderFactory( 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 requiredDataSchema = subtractSchema(readSchema, partitionSchema) val orcRecordReader = new OrcInputFormat[OrcStruct] .createRecordReader(fileSplit, taskAttemptContext) @@ -136,16 +135,33 @@ case class OrcPartitionReaderFactory( val batchReader = new OrcColumnarBatchReader( enableOffHeapColumnVector && taskContext.isDefined, copyToSpark, capacity) batchReader.initialize(fileSplit, taskAttemptContext) - val partitionColIds = PartitioningUtils.requestedPartitionColumnIds( - partitionSchema, readSchema, isCaseSensitive) + val columnNameMap = partitionSchema.fields.map( + PartitioningUtils.getColName(_, isCaseSensitive)).zipWithIndex.toMap + val requestedPartitionColIds = readSchema.fields.map { field => + columnNameMap.getOrElse(PartitioningUtils.getColName(field, isCaseSensitive), -1) + } batchReader.initBatch( reader.getSchema, readSchema.fields, requestedColIds, - partitionColIds, + requestedPartitionColIds, file.partitionValues) new PartitionRecordReader(batchReader) } } + + /** + * 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. + */ + private def subtractSchema(original: StructType, other: StructType): StructType = { + val otherNameSet = other.fields.map(PartitioningUtils.getColName(_, isCaseSensitive)).toSet + val fields = original.fields.filterNot { field => + otherNameSet.contains(PartitioningUtils.getColName(field, isCaseSensitive)) + } + + StructType(fields) + } + } From af9de22b259fe981b82d65f581ff478b37ff4ecf Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Thu, 10 Jan 2019 19:39:48 +0800 Subject: [PATCH 15/31] use SparkSession in OrcPartitionReaderFactory --- .../datasources/v2/orc/OrcPartitionReaderFactory.scala | 2 +- .../spark/sql/execution/datasources/v2/orc/OrcScan.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala index 32e3415690211..cdfc14aeff48f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala @@ -42,12 +42,12 @@ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration case class OrcPartitionReaderFactory( + sqlConf: SQLConf, broadcastedConf: Broadcast[SerializableConfiguration], schema: StructType, dataSchema: StructType, readSchema: StructType, partitionSchema: StructType) extends FilePartitionReaderFactory { - private val sqlConf = SQLConf.get private val isCaseSensitive = sqlConf.caseSensitiveAnalysis private val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled private val copyToSpark = sqlConf.getConf(SQLConf.ORC_COPY_BATCH_TO_SPARK) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala index 51776c1945aa2..8d51dbdd3c2e7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala @@ -37,8 +37,8 @@ case class OrcScan( override def createReaderFactory(): PartitionReaderFactory = { val broadcastedConf = sparkSession.sparkContext.broadcast( - new SerializableConfiguration(hadoopConf)) - OrcPartitionReaderFactory(broadcastedConf, schema, + new SerializableConfiguration(hadoopConf)) + OrcPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, schema, dataSchema, readSchema, fileIndex.partitionSchema) } } From 2f7b3c66cadcc7209213b8e5bbeb914f54041f18 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Fri, 11 Jan 2019 01:38:46 +0800 Subject: [PATCH 16/31] update the latest master --- .../datasources/v2/orc/OrcPartitionReaderFactory.scala | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala index cdfc14aeff48f..6fbf10d6a6151 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala @@ -49,8 +49,6 @@ case class OrcPartitionReaderFactory( readSchema: StructType, partitionSchema: StructType) extends FilePartitionReaderFactory { private val isCaseSensitive = sqlConf.caseSensitiveAnalysis - private val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled - private val copyToSpark = sqlConf.getConf(SQLConf.ORC_COPY_BATCH_TO_SPARK) private val capacity = sqlConf.orcVectorizedReaderBatchSize override def supportColumnarReads(partition: InputPartition): Boolean = { @@ -130,10 +128,7 @@ case class OrcPartitionReaderFactory( val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) val taskAttemptContext = new TaskAttemptContextImpl(taskConf, attemptId) - val taskContext = Option(TaskContext.get()) - - val batchReader = new OrcColumnarBatchReader( - enableOffHeapColumnVector && taskContext.isDefined, copyToSpark, capacity) + val batchReader = new OrcColumnarBatchReader(capacity) batchReader.initialize(fileSplit, taskAttemptContext) val columnNameMap = partitionSchema.fields.map( PartitioningUtils.getColName(_, isCaseSensitive)).zipWithIndex.toMap From 91aeafba0fdd1dbdfba8005263cd8da31e8674b1 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Fri, 11 Jan 2019 11:39:53 +0800 Subject: [PATCH 17/31] fix regression of merging to latest master --- .../src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala index 1363b460fc87b..bd25a6437033c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala @@ -77,6 +77,7 @@ object HiveSerDe { case s if s.startsWith("org.apache.spark.sql.execution.datasources.parquet") => "parquet" case s if s.startsWith("org.apache.spark.sql.orc") => "orc" case s if s.startsWith("org.apache.spark.sql.hive.orc") => "orc" + case s if s.startsWith("org.apache.spark.sql.execution.datasources.orc") => "orc" case s if s.equals("orcfile") => "orc" case s if s.equals("parquetfile") => "parquet" case s if s.equals("avrofile") => "avro" From 5ebeda020f4f8b3afbfe311a5088abb81c12d161 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Fri, 11 Jan 2019 13:07:44 +0800 Subject: [PATCH 18/31] don't change OrcFileFormat's short name --- .../spark/sql/execution/datasources/orc/OrcFileFormat.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 5a72dbac047ec..14779cdba4178 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 @@ -81,7 +81,7 @@ class OrcFileFormat with DataSourceRegister with Serializable { - override def shortName(): String = "orcV1" + override def shortName(): String = "orc" override def toString: String = "ORC" From 3b50f2d2d55740ba80a2e808291972347b23c86e Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Fri, 11 Jan 2019 21:04:16 +0800 Subject: [PATCH 19/31] address comments --- .../apache/spark/sql/DataFrameReader.scala | 3 +- .../sql/execution/PartitionedFileUtil.scala | 5 +- .../datasources/FilePartitionUtil.scala | 6 +- .../datasources/orc/OrcFilters.scala | 8 +- .../datasources/v2/FilePartitionReader.scala | 8 +- .../v2/FilePartitionReaderFactory.scala | 21 +++- .../v2/PartitionRecordReader.scala | 2 +- .../datasources/v2/orc/OrcDataSourceV2.scala | 7 +- .../v2/orc/OrcPartitionReaderFactory.scala | 5 +- .../datasources/v2/orc/OrcScanBuilder.scala | 3 +- .../datasources/orc/OrcFilterSuite.scala | 75 ------------- .../datasources/orc/OrcV1FilterSuite.scala | 104 ++++++++++++++++++ 12 files changed, 148 insertions(+), 99 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.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 3cd924379bf00..15b1640edc97b 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,8 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { "read files of Hive data source directly.") } - val disabledV2Readers = sparkSession.sessionState.conf.disabledV2FileReads.split(",") + val disabledV2Readers = + sparkSession.sessionState.conf.disabledV2FileReads.toLowerCase(Locale.ROOT).split(",") val lookupCls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf) val cls = lookupCls.newInstance() match { case f: FileDataSourceV2 if disabledV2Readers.contains(f.shortName()) => f.fallBackFileFormat 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 index 06585271c1650..d13e3060e3877 100644 --- 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 @@ -21,7 +21,7 @@ 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, _} +import org.apache.spark.sql.execution.datasources._ object PartitionedFileUtil { def splitFiles( @@ -46,8 +46,7 @@ object PartitionedFileUtil { def getPartitionedFile( file: FileStatus, filePath: Path, - partitionValues: InternalRow - ): PartitionedFile = { + partitionValues: InternalRow): PartitionedFile = { val hosts = getBlockHosts(getBlockLocations(file), 0, file.getLen) PartitionedFile(partitionValues, filePath.toUri.toString, 0, file.getLen, hosts) } 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 c29dc0a32c95a..5137efd89d7b3 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 @@ -34,10 +34,8 @@ object FilePartitionUtil extends Logging { /** 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. + // Copy to a new Array. + val newPartition = FilePartition(partitions.size, currentFiles.toArray.toSeq) partitions += newPartition } currentFiles.clear() 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 ee545d19f21cf..cd2a68a53bab7 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 @@ -84,14 +84,16 @@ private[sql] object OrcFilters { val dataTypeMap = schema.map(f => f.name -> f.dataType).toMap for { // Combines all convertible filters using `And` to produce a single conjunction - conjunction <- buildTree(convertibleFilters(schema, filters)) + conjunction <- buildTree(convertibleFilters(schema, dataTypeMap, filters)) // Then tries to build a single ORC `SearchArgument` for the conjunction predicate builder <- buildSearchArgument(dataTypeMap, conjunction, newBuilder) } yield builder.build() } - def convertibleFilters(schema: StructType, filters: Seq[Filter]): Seq[Filter] = { - val dataTypeMap = schema.map(f => f.name -> f.dataType).toMap + def convertibleFilters( + schema: StructType, + dataTypeMap: Map[String, DataType], + filters: Seq[Filter]): Seq[Filter] = { for { filter <- filters _ <- buildSearchArgument(dataTypeMap, filter, newBuilder()) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala index 81709c137f85b..d76d69dba31df 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala @@ -22,9 +22,9 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.v2.reader.PartitionReader -class FilePartitionReader[T]( - readers: Iterator[PartitionReader[T]]) extends PartitionReader[T] with Logging { - private var currentReader: PartitionReader[T] = null +class FilePartitionReader[T](readers: Iterator[PartitionedFileReader[T]]) + extends PartitionReader[T] with Logging { + private var currentReader: PartitionedFileReader[T] = null private val sqlConf = SQLConf.get private def ignoreMissingFiles = sqlConf.ignoreMissingFiles @@ -36,6 +36,7 @@ class FilePartitionReader[T]( if (ignoreMissingFiles || ignoreCorruptFiles) { try { currentReader = readers.next() + logInfo(s"Reading file $currentReader") } catch { case e: FileNotFoundException if ignoreMissingFiles => logWarning(s"Skipped missing file: $currentReader", e) @@ -51,6 +52,7 @@ class FilePartitionReader[T]( } } else { currentReader = readers.next() + logInfo(s"Reading file $currentReader") } } else { return false diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala index 16d224ffc4070..7859b3f6bc024 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala @@ -25,14 +25,18 @@ abstract class FilePartitionReaderFactory extends PartitionReaderFactory { override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { assert(partition.isInstanceOf[FilePartition]) val filePartition = partition.asInstanceOf[FilePartition] - val iter = filePartition.files.toIterator.map(buildReader) + val iter = filePartition.files.toIterator.map { file => + new PartitionedFileReader(file, buildReader(file)) + } new FilePartitionReader[InternalRow](iter) } override def createColumnarReader(partition: InputPartition): PartitionReader[ColumnarBatch] = { assert(partition.isInstanceOf[FilePartition]) val filePartition = partition.asInstanceOf[FilePartition] - val iter = filePartition.files.toIterator.map(buildColumnarReader) + val iter = filePartition.files.toIterator.map { file => + new PartitionedFileReader(file, buildColumnarReader(file)) + } new FilePartitionReader[ColumnarBatch](iter) } @@ -42,3 +46,16 @@ abstract class FilePartitionReaderFactory extends PartitionReaderFactory { throw new UnsupportedOperationException("Cannot create columnar reader.") } } + +// A compound class for combining a input partitioned file and its corresponding file. +private[v2] class PartitionedFileReader[T]( + file: PartitionedFile, + reader: PartitionReader[T]) extends PartitionReader[T] { + override def next(): Boolean = reader.next() + + override def get(): T = reader.get() + + override def close(): Unit = reader.close() + + override def toString: String = file.toString +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionRecordReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionRecordReader.scala index f716fa679bffd..ff78ef3220c17 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionRecordReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionRecordReader.scala @@ -30,7 +30,7 @@ class PartitionRecordReader[T]( override def close(): Unit = rowReader.close() } -class PartitionRecordDReaderWithProject[X, T]( +class PartitionRecordReaderWithProject[X, T]( private[this] var rowReader: RecordReader[_, X], project: X => T) extends PartitionReader[T] { override def next(): Boolean = rowReader.nextKeyValue() 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 25e6719c24171..1eb77f22304cb 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,17 +28,18 @@ class OrcDataSourceV2 extends FileDataSourceV2 { override def shortName(): String = "orc" - private def geTableName(options: DataSourceOptions): String = { + private def getTableName(options: DataSourceOptions): String = { shortName() + ":" + options.paths().mkString(";") } + override def getTable(options: DataSourceOptions): Table = { - val tableName = geTableName(options) + val tableName = getTableName(options) val fileIndex = getFileIndex(options, None) OrcTable(tableName, sparkSession, fileIndex, None) } override def getTable(options: DataSourceOptions, schema: StructType): Table = { - val tableName = geTableName(options) + val tableName = getTableName(options) val fileIndex = getFileIndex(options, Some(schema)) OrcTable(tableName, sparkSession, fileIndex, Some(schema)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala index 6fbf10d6a6151..976b45afd6d73 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala @@ -27,14 +27,13 @@ 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 import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.execution.datasources.{PartitionedFile, PartitioningUtils} import org.apache.spark.sql.execution.datasources.orc.{OrcColumnarBatchReader, OrcDeserializer, OrcUtils} -import org.apache.spark.sql.execution.datasources.v2.{EmptyPartitionReader, FilePartitionReaderFactory, PartitionRecordDReaderWithProject, PartitionRecordReader} +import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader} import org.apache.spark.sql.types.{AtomicType, StructType} @@ -98,7 +97,7 @@ case class OrcPartitionReaderFactory( (value: OrcStruct) => unsafeProjection(joinedRow(deserializer.deserialize(value), file.partitionValues)) } - new PartitionRecordDReaderWithProject(orcRecordReader, projection) + new PartitionRecordReaderWithProject(orcRecordReader, projection) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala index 51c098ba7a366..9dcd6d8bc7f74 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala @@ -49,7 +49,8 @@ case class OrcScanBuilder( OrcFilters.createFilter(schema, filters).foreach { f => OrcInputFormat.setSearchArgument(hadoopConf, f, schema.fieldNames) } - _pushedFilters = OrcFilters.convertibleFilters(schema, filters).toArray + val dataTypeMap = schema.map(f => f.name -> f.dataType).toMap + _pushedFilters = OrcFilters.convertibleFilters(schema, dataTypeMap, filters).toArray } filters } 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 067e24095aca1..d0d422ffd36bb 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 @@ -24,7 +24,6 @@ import scala.collection.JavaConverters._ import org.apache.orc.storage.ql.io.sarg.{PredicateLeaf, SearchArgument} -import org.apache.spark.SparkConf import org.apache.spark.sql.{AnalysisException, Column, DataFrame} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ @@ -429,77 +428,3 @@ class OrcFilterSuite extends OrcTest with SharedSQLContext { } } -class OrcV1FilterSuite extends OrcFilterSuite { - - override protected def sparkConf: SparkConf = - super.sparkConf.set(SQLConf.DISABLED_V2_FILE_READS, "orc") - - override def checkFilterPredicate( - df: DataFrame, - predicate: Predicate, - checker: (SearchArgument) => Unit): Unit = { - val output = predicate.collect { case a: Attribute => a }.distinct - val query = df - .select(output.map(e => Column(e)): _*) - .where(Column(predicate)) - - var maybeRelation: Option[HadoopFsRelation] = None - val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => - maybeRelation = Some(orcRelation) - 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 maybeFilter = OrcFilters.createFilter(query.schema, selectedFilters) - assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $selectedFilters") - checker(maybeFilter.get) - } - - override def checkFilterPredicate - (predicate: Predicate, filterOperator: PredicateLeaf.Operator) - (implicit df: DataFrame): Unit = { - def checkComparisonOperator(filter: SearchArgument) = { - val operator = filter.getLeaves.asScala - assert(operator.map(_.getOperator).contains(filterOperator)) - } - checkFilterPredicate(df, predicate, checkComparisonOperator) - } - - override def checkFilterPredicate - (predicate: Predicate, stringExpr: String) - (implicit df: DataFrame): Unit = { - def checkLogicalOperator(filter: SearchArgument) = { - assert(filter.toString == stringExpr) - } - checkFilterPredicate(df, predicate, checkLogicalOperator) - } - - override def checkNoFilterPredicate - (predicate: Predicate, noneSupported: Boolean = false) - (implicit df: DataFrame): Unit = { - val output = predicate.collect { case a: Attribute => a }.distinct - val query = df - .select(output.map(e => Column(e)): _*) - .where(Column(predicate)) - - var maybeRelation: Option[HadoopFsRelation] = None - val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => - maybeRelation = Some(orcRelation) - 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 maybeFilter = OrcFilters.createFilter(query.schema, selectedFilters) - assert(maybeFilter.isEmpty, s"Could generate filter predicate for $selectedFilters") - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala new file mode 100644 index 0000000000000..a8c06eb7638f4 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.orc + +import scala.collection.JavaConverters._ + +import org.apache.orc.storage.ql.io.sarg.{PredicateLeaf, SearchArgument} + +import org.apache.spark.SparkConf +import org.apache.spark.sql.{Column, DataFrame} +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.{Attribute, Predicate} +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.internal.SQLConf + +class OrcV1FilterSuite extends OrcFilterSuite { + + override protected def sparkConf: SparkConf = + super.sparkConf.set(SQLConf.DISABLED_V2_FILE_READS, "orc") + + override def checkFilterPredicate( + df: DataFrame, + predicate: Predicate, + checker: (SearchArgument) => Unit): Unit = { + val output = predicate.collect { case a: Attribute => a }.distinct + val query = df + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) + + var maybeRelation: Option[HadoopFsRelation] = None + val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { + case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => + maybeRelation = Some(orcRelation) + 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 maybeFilter = OrcFilters.createFilter(query.schema, selectedFilters) + assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $selectedFilters") + checker(maybeFilter.get) + } + + override def checkFilterPredicate + (predicate: Predicate, filterOperator: PredicateLeaf.Operator) + (implicit df: DataFrame): Unit = { + def checkComparisonOperator(filter: SearchArgument) = { + val operator = filter.getLeaves.asScala + assert(operator.map(_.getOperator).contains(filterOperator)) + } + checkFilterPredicate(df, predicate, checkComparisonOperator) + } + + override def checkFilterPredicate + (predicate: Predicate, stringExpr: String) + (implicit df: DataFrame): Unit = { + def checkLogicalOperator(filter: SearchArgument) = { + assert(filter.toString == stringExpr) + } + checkFilterPredicate(df, predicate, checkLogicalOperator) + } + + override def checkNoFilterPredicate + (predicate: Predicate, noneSupported: Boolean = false) + (implicit df: DataFrame): Unit = { + val output = predicate.collect { case a: Attribute => a }.distinct + val query = df + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) + + var maybeRelation: Option[HadoopFsRelation] = None + val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { + case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => + maybeRelation = Some(orcRelation) + 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 maybeFilter = OrcFilters.createFilter(query.schema, selectedFilters) + assert(maybeFilter.isEmpty, s"Could generate filter predicate for $selectedFilters") + } +} From ea8f178c7b5bfc735245a575e9c0861b36acff2e Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Sat, 12 Jan 2019 02:57:30 +0800 Subject: [PATCH 20/31] address more comments --- .../datasources/DataSourceStrategy.scala | 2 +- .../datasources/FilePartitionUtil.scala | 18 ++++++++++++++++++ .../execution/datasources/FileScanRDD.scala | 15 +-------------- 3 files changed, 20 insertions(+), 15 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 ada7aa72f31f6..5dc7cdbe45cca 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 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast class FallBackFileDataSourceToV1(sparkSession: SparkSession) extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case i @ InsertIntoTable(d @ - DataSourceV2Relation(source: FileDataSourceV2, table: FileTable, _, _, _), _, _, _, _) => + DataSourceV2Relation(source: FileDataSourceV2, table: FileTable, _, _, _), _, _, _, _) => val v1FileFormat = source.fallBackFileFormat.getConstructor().newInstance() val relation = HadoopFsRelation(table.getFileIndex, table.getFileIndex.partitionSchema, table.schema(), None, v1FileFormat, d.options)(sparkSession) 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 5137efd89d7b3..913c30d4f91fc 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 @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.execution.datasources +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import org.apache.spark.internal.Logging @@ -55,4 +56,21 @@ object FilePartitionUtil extends Logging { closePartition() partitions } + + def getPreferredLocations(files: Seq[PartitionedFile]): Array[String] = { + // 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 + }.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 6c3b0b5e04771..73c9fcc8e5d9b 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 @@ -61,20 +61,7 @@ case class PartitionedFile( case class FilePartition(index: Int, files: Seq[PartitionedFile]) extends RDDPartition with InputPartition { override def preferredLocations(): Array[String] = { - // 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 - }.toArray + FilePartitionUtil.getPreferredLocations(files) } } From aff3788530e974594e95be7b99e132e71119a141 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Sat, 12 Jan 2019 20:51:50 +0800 Subject: [PATCH 21/31] check file existence --- .../spark/sql/execution/datasources/v2/FileDataSourceV2.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 2b9cfe0e681c9..712e57863bd64 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 @@ -46,7 +46,7 @@ trait FileDataSourceV2 extends TableProvider with DataSourceRegister { val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(options.asMap().asScala.toMap) val rootPathsSpecified = DataSource.checkAndGlobPathIfNecessary(filePaths, hadoopConf, - checkEmptyGlobPath = false, checkFilesExist = false) + checkEmptyGlobPath = true, checkFilesExist = true) val fileStatusCache = FileStatusCache.getOrCreate(sparkSession) new InMemoryFileIndex(sparkSession, rootPathsSpecified, options.asMap().asScala.toMap, userSpecifiedSchema, fileStatusCache) From 575643b790d84cd60bdae0c8061632615902b7d5 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Mon, 14 Jan 2019 00:46:08 +0800 Subject: [PATCH 22/31] address comments --- .../apache/spark/sql/internal/SQLConf.scala | 1 - .../datasources/DataSourceStrategy.scala | 18 -------- .../FallBackFileDataSourceToV1.scala | 41 +++++++++++++++++++ .../datasources/v2/orc/OrcDataSourceV2.scala | 2 +- .../datasources/v2/orc/OrcScanBuilder.scala | 1 + 5 files changed, 43 insertions(+), 20 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileDataSourceToV1.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 55fa520689659..fbc003a4009ef 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 @@ -1420,7 +1420,6 @@ object SQLConf { .createWithDefault(100) val DISABLED_V2_FILE_READS = buildConf("spark.sql.files.disabledV2Reads") - .internal() .doc("A comma-separated list of FileDataSourceV2 short names for which read paths " + "are disabled. Reads from these sources will fall back to the V1 sources.") .stringConf 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 5dc7cdbe45cca..12b60c4d3925a 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,24 +215,6 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast } } -/** - * 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. It should be removed - * when write path of file data source v2 is finished. - */ -class FallBackFileDataSourceToV1(sparkSession: SparkSession) extends Rule[LogicalPlan] { - override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case i @ InsertIntoTable(d @ - DataSourceV2Relation(source: FileDataSourceV2, table: FileTable, _, _, _), _, _, _, _) => - val v1FileFormat = source.fallBackFileFormat.getConstructor().newInstance() - val relation = HadoopFsRelation(table.getFileIndex, table.getFileIndex.partitionSchema, - table.schema(), None, v1FileFormat, d.options)(sparkSession) - i.copy(table = LogicalRelation(relation)) - } -} - /** * Replaces [[UnresolvedCatalogRelation]] with concrete relation logical plans. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileDataSourceToV1.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileDataSourceToV1.scala new file mode 100644 index 0000000000000..a4de430b33627 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileDataSourceToV1.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileDataSourceV2, FileTable} + +/** + * 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. It should be removed + * when write path of file data source v2 is finished. + */ +class FallBackFileDataSourceToV1(sparkSession: SparkSession) extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case i @ InsertIntoTable(d @ + DataSourceV2Relation(source: FileDataSourceV2, table: FileTable, _, _, _), _, _, _, _) => + val v1FileFormat = source.fallBackFileFormat.getConstructor().newInstance() + val relation = HadoopFsRelation(table.getFileIndex, table.getFileIndex.partitionSchema, + table.schema(), None, v1FileFormat, d.options)(sparkSession) + i.copy(table = LogicalRelation(relation)) + } +} 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 1eb77f22304cb..db1f2f7934221 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,7 @@ package org.apache.spark.sql.execution.datasources.v2.orc import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.datasources.orc._ +import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.sources.v2.{DataSourceOptions, Table} import org.apache.spark.sql.types.StructType diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala index 9dcd6d8bc7f74..bf93ed051c1cc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.spark.sql.execution.datasources.v2.orc import scala.collection.JavaConverters._ From ac8acddd7da37f092d1cf0e891b068596e09144c Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Mon, 14 Jan 2019 14:45:02 +0800 Subject: [PATCH 23/31] update sqlconf --- .../scala/org/apache/spark/sql/internal/SQLConf.scala | 11 ++++++----- .../scala/org/apache/spark/sql/DataFrameReader.scala | 6 ++++-- .../datasources/orc/OrcPartitionDiscoverySuite.scala | 2 +- .../sql/execution/datasources/orc/OrcQuerySuite.scala | 2 +- .../execution/datasources/orc/OrcV1FilterSuite.scala | 2 +- .../sources/v2/FileDataSourceV2FallBackSuite.scala | 4 ++-- 6 files changed, 15 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 fbc003a4009ef..91032f0225c10 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 @@ -1419,14 +1419,15 @@ object SQLConf { .timeConf(TimeUnit.MILLISECONDS) .createWithDefault(100) - val DISABLED_V2_FILE_READS = buildConf("spark.sql.files.disabledV2Reads") - .doc("A comma-separated list of FileDataSourceV2 short names for which read paths " + - "are disabled. Reads from these sources will fall back to the V1 sources.") + val DISABLED_V2_BATCH_READERS = buildConf("spark.sql.sources.disabledV2BatchReaders") + .internal() + .doc("A comma-separated list of data source short names or fully qualified data source" + + " register class names for which read paths are disabled. Reads from these sources will" + + " fall back to the V1 sources.") .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" + " StreamWriteSupport is disabled. Writes to these sources will fall back to the V1 Sinks.") .stringConf @@ -2008,7 +2009,7 @@ class SQLConf extends Serializable with Logging { def continuousStreamingExecutorPollIntervalMs: Long = getConf(CONTINUOUS_STREAMING_EXECUTOR_POLL_INTERVAL_MS) - def disabledV2FileReads: String = getConf(DISABLED_V2_FILE_READS) + def disabledV2BatchReaders: String = getConf(DISABLED_V2_BATCH_READERS) def disabledV2StreamingWriters: String = getConf(DISABLED_V2_STREAMING_WRITERS) 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 15b1640edc97b..97cad463b2123 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,10 +193,12 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { } val disabledV2Readers = - sparkSession.sessionState.conf.disabledV2FileReads.toLowerCase(Locale.ROOT).split(",") + sparkSession.sessionState.conf.disabledV2BatchReaders.toLowerCase(Locale.ROOT).split(",") val lookupCls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf) val cls = lookupCls.newInstance() match { - case f: FileDataSourceV2 if disabledV2Readers.contains(f.shortName()) => f.fallBackFileFormat + case f: FileDataSourceV2 if disabledV2Readers.contains(f.shortName()) || + disabledV2Readers.contains(lookupCls.getCanonicalName) => + f.fallBackFileFormat case _ => lookupCls } 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 63e78fa6ba2ef..d9f8e3267496f 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_FILE_READS, "orc") + super.sparkConf.set(SQLConf.DISABLED_V2_BATCH_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 959d6287cbdaa..cecd1e4ab7a1f 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 @@ -684,5 +684,5 @@ class OrcQuerySuite extends OrcQueryTest with SharedSQLContext { class OrcV1QuerySuite extends OrcQuerySuite { override protected def sparkConf: SparkConf = - super.sparkConf.set(SQLConf.DISABLED_V2_FILE_READS, "orc") + super.sparkConf.set(SQLConf.DISABLED_V2_BATCH_READERS, "orc") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala index a8c06eb7638f4..deffa24cbb305 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.internal.SQLConf class OrcV1FilterSuite extends OrcFilterSuite { override protected def sparkConf: SparkConf = - super.sparkConf.set(SQLConf.DISABLED_V2_FILE_READS, "orc") + super.sparkConf.set(SQLConf.DISABLED_V2_BATCH_READERS, "orc") override def checkFilterPredicate( df: DataFrame, 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 index bb081b6ed78df..070309def7cce 100644 --- 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 @@ -61,12 +61,12 @@ class FileDataSourceV2FallBackSuite extends QueryTest with ParquetTest with Shar withTempPath { file => val path = file.getCanonicalPath df.write.parquet(path) - withSQLConf(SQLConf.DISABLED_V2_FILE_READS.key -> "foo,parquet,bar") { + withSQLConf(SQLConf.DISABLED_V2_BATCH_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_READS.key -> "foo,bar") { + withSQLConf(SQLConf.DISABLED_V2_BATCH_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) From 13d615b5a21f52acdcf5994a712af45ced8ea213 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Mon, 14 Jan 2019 22:40:01 +0800 Subject: [PATCH 24/31] address comments --- .../apache/spark/sql/DataFrameReader.scala | 2 +- .../sql/execution/DataSourceScanExec.scala | 4 +- .../sql/execution/PartitionedFileUtil.scala | 12 ----- ...scala => FallbackFileDataSourceToV1.scala} | 6 +-- ...artitionUtil.scala => FilePartition.scala} | 51 +++++++++++++------ .../execution/datasources/FileScanRDD.scala | 11 ---- .../execution/datasources/v2/FileScan.scala | 4 +- .../internal/BaseSessionStateBuilder.scala | 2 +- .../v2/FileDataSourceV2FallBackSuite.scala | 13 +++-- .../sql/hive/HiveSessionStateBuilder.scala | 2 +- 10 files changed, 56 insertions(+), 51 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/{FallBackFileDataSourceToV1.scala => FallbackFileDataSourceToV1.scala} (89%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/{FilePartitionUtil.scala => FilePartition.scala} (72%) 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 97cad463b2123..d7d8e268c30db 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 @@ -197,7 +197,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { val lookupCls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf) val cls = lookupCls.newInstance() match { case f: FileDataSourceV2 if disabledV2Readers.contains(f.shortName()) || - disabledV2Readers.contains(lookupCls.getCanonicalName) => + disabledV2Readers.contains(lookupCls.getCanonicalName.toLowerCase(Locale.ROOT)) => f.fallBackFileFormat case _ => lookupCls } 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 7466700564c97..f6f3fb18c97fd 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 @@ -411,7 +411,7 @@ case class FileSourceScanExec( fsRelation: HadoopFsRelation): RDD[InternalRow] = { val openCostInBytes = fsRelation.sparkSession.sessionState.conf.filesOpenCostInBytes val maxSplitBytes = - PartitionedFileUtil.maxSplitBytes(fsRelation.sparkSession, selectedPartitions) + FilePartition.maxSplitBytes(fsRelation.sparkSession, selectedPartitions) logInfo(s"Planning scan with bin packing, max size: $maxSplitBytes bytes, " + s"open cost is considered as scanning $openCostInBytes bytes.") @@ -433,7 +433,7 @@ case class FileSourceScanExec( }.toArray.sortBy(_.length)(implicitly[Ordering[Long]].reverse) val partitions = - FilePartitionUtil.getFilePartitions(relation.sparkSession, splitFiles, maxSplitBytes) + FilePartition.getFilePartitions(relation.sparkSession, splitFiles, maxSplitBytes) new FileScanRDD(fsRelation.sparkSession, readFile, partitions) } 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 index d13e3060e3877..3196624f7c7c3 100644 --- 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 @@ -51,18 +51,6 @@ object PartitionedFileUtil { 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] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileDataSourceToV1.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallbackFileDataSourceToV1.scala similarity index 89% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileDataSourceToV1.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallbackFileDataSourceToV1.scala index a4de430b33627..34234a2f585f3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileDataSourceToV1.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallbackFileDataSourceToV1.scala @@ -26,10 +26,10 @@ import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, File * 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. It should be removed - * when write path of file data source v2 is finished. + * SPARK-23817: This is a temporary hack for making current data source V2 work. It should be + * removed when write path of file data source v2 is finished. */ -class FallBackFileDataSourceToV1(sparkSession: SparkSession) extends Rule[LogicalPlan] { +class FallbackFileDataSourceToV1(sparkSession: SparkSession) extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case i @ InsertIntoTable(d @ DataSourceV2Relation(source: FileDataSourceV2, table: FileTable, _, _, _), _, _, _, _) => 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/FilePartition.scala similarity index 72% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartitionUtil.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala index 913c30d4f91fc..4b1ade8e29575 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/FilePartition.scala @@ -19,10 +19,36 @@ package org.apache.spark.sql.execution.datasources import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import org.apache.spark.Partition import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.sources.v2.reader.InputPartition -object FilePartitionUtil extends Logging { +/** + * A collection of file blocks that should be read as a single task + * (possibly from multiple partitioned directories). + */ +case class FilePartition(index: Int, files: Seq[PartitionedFile]) + extends Partition with InputPartition { + override def preferredLocations(): Array[String] = { + // 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 + }.toArray + } +} + +object FilePartition extends Logging { def getFilePartitions( sparkSession: SparkSession, @@ -57,20 +83,15 @@ object FilePartitionUtil extends Logging { partitions } - def getPreferredLocations(files: Seq[PartitionedFile]): Array[String] = { - // 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 - } - } + 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 - // 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 - }.toArray + Math.min(defaultMaxSplitBytes, Math.max(openCostInBytes, bytesPerCore)) } } 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 73c9fcc8e5d9b..d92ea2e068953 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 @@ -54,17 +54,6 @@ case class PartitionedFile( } } -/** - * A collection of file blocks that should be read as a single task - * (possibly from multiple partitioned directories). - */ -case class FilePartition(index: Int, files: Seq[PartitionedFile]) - extends RDDPartition with InputPartition { - override def preferredLocations(): Array[String] = { - FilePartitionUtil.getPreferredLocations(files) - } -} - /** * An RDD that scans a list of file partitions. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala index 4246c28197eb6..3615b15be6fd5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala @@ -36,7 +36,7 @@ abstract class FileScan( protected def partitions: Seq[FilePartition] = { val selectedPartitions = fileIndex.listFiles(Seq.empty, Seq.empty) - val maxSplitBytes = PartitionedFileUtil.maxSplitBytes(sparkSession, selectedPartitions) + val maxSplitBytes = FilePartition.maxSplitBytes(sparkSession, selectedPartitions) val splitFiles = selectedPartitions.flatMap { partition => partition.files.flatMap { file => val filePath = file.getPath @@ -50,7 +50,7 @@ abstract class FileScan( ) }.toArray.sortBy(_.length)(implicitly[Ordering[Long]].reverse) } - FilePartitionUtil.getFilePartitions(sparkSession, splitFiles, maxSplitBytes) + FilePartition.getFilePartitions(sparkSession, splitFiles, maxSplitBytes) } override def planInputPartitions(): Array[InputPartition] = { 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 569a04921d4ca..2319d6f421dbb 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 @@ -159,7 +159,7 @@ abstract class BaseSessionStateBuilder( override val extendedResolutionRules: Seq[Rule[LogicalPlan]] = new FindDataSourceTable(session) +: new ResolveSQLOnFile(session) +: - new FallBackFileDataSourceToV1(session) +: + new FallbackFileDataSourceToV1(session) +: customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = 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 index 070309def7cce..bf0f80027255e 100644 --- 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 @@ -61,9 +61,16 @@ class FileDataSourceV2FallBackSuite extends QueryTest with ParquetTest with Shar withTempPath { file => val path = file.getCanonicalPath df.write.parquet(path) - withSQLConf(SQLConf.DISABLED_V2_BATCH_READERS.key -> "foo,parquet,bar") { - // Reading file should fall back to v1 and succeed. - checkAnswer(spark.read.format(dummyParquetReaderV2).load(path), df) + Seq( + "foo,parquet,bar", + "ParQuet,bar,foo", + s"foobar,$dummyParquetReaderV2" + ).foreach { fallbackReaders => + withSQLConf(SQLConf.DISABLED_V2_BATCH_READERS.key -> fallbackReaders) { + // Reading file should fall back to v1 and succeed. + checkAnswer(spark.read.format(dummyParquetReaderV2).load(path), df) + checkAnswer(sql(s"SELECT * FROM parquet.`$path`"), df) + } } withSQLConf(SQLConf.DISABLED_V2_BATCH_READERS.key -> "foo,bar") { 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 caa71e7cce133..be9eca15846e2 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,7 +71,7 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session new ResolveHiveSerdeTable(session) +: new FindDataSourceTable(session) +: new ResolveSQLOnFile(session) +: - new FallBackFileDataSourceToV1(session) +: + new FallbackFileDataSourceToV1(session) +: customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = From a1e66f3d461d3f38499f807eb9ec6fd9e7643d4b Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Tue, 15 Jan 2019 13:33:18 +0800 Subject: [PATCH 25/31] change conf to spark.sql.sources.useV1Readers --- .../main/scala/org/apache/spark/sql/internal/SQLConf.scala | 4 ++-- .../src/main/scala/org/apache/spark/sql/DataFrameReader.scala | 2 +- .../datasources/orc/OrcPartitionDiscoverySuite.scala | 2 +- .../spark/sql/execution/datasources/orc/OrcQuerySuite.scala | 2 +- .../sql/execution/datasources/orc/OrcV1FilterSuite.scala | 2 +- .../spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala | 4 ++-- 6 files changed, 8 insertions(+), 8 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 91032f0225c10..dcec54f5715ff 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 @@ -1419,7 +1419,7 @@ object SQLConf { .timeConf(TimeUnit.MILLISECONDS) .createWithDefault(100) - val DISABLED_V2_BATCH_READERS = buildConf("spark.sql.sources.disabledV2BatchReaders") + val USE_V1_READERS = buildConf("spark.sql.sources.useV1Readers") .internal() .doc("A comma-separated list of data source short names or fully qualified data source" + " register class names for which read paths are disabled. Reads from these sources will" + @@ -2009,7 +2009,7 @@ class SQLConf extends Serializable with Logging { def continuousStreamingExecutorPollIntervalMs: Long = getConf(CONTINUOUS_STREAMING_EXECUTOR_POLL_INTERVAL_MS) - def disabledV2BatchReaders: String = getConf(DISABLED_V2_BATCH_READERS) + def useV1Readers: String = getConf(USE_V1_READERS) def disabledV2StreamingWriters: String = getConf(DISABLED_V2_STREAMING_WRITERS) 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 d7d8e268c30db..6ec107bd8f6be 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,7 +193,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { } val disabledV2Readers = - sparkSession.sessionState.conf.disabledV2BatchReaders.toLowerCase(Locale.ROOT).split(",") + sparkSession.sessionState.conf.useV1Readers.toLowerCase(Locale.ROOT).split(",") val lookupCls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf) val cls = lookupCls.newInstance() match { case f: FileDataSourceV2 if disabledV2Readers.contains(f.shortName()) || 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 d9f8e3267496f..86f98ab83102e 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_BATCH_READERS, "orc") + super.sparkConf.set(SQLConf.USE_V1_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 cecd1e4ab7a1f..5386ca0124d3d 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 @@ -684,5 +684,5 @@ class OrcQuerySuite extends OrcQueryTest with SharedSQLContext { class OrcV1QuerySuite extends OrcQuerySuite { override protected def sparkConf: SparkConf = - super.sparkConf.set(SQLConf.DISABLED_V2_BATCH_READERS, "orc") + super.sparkConf.set(SQLConf.USE_V1_READERS, "orc") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala index deffa24cbb305..5d0bb1c091e2d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.internal.SQLConf class OrcV1FilterSuite extends OrcFilterSuite { override protected def sparkConf: SparkConf = - super.sparkConf.set(SQLConf.DISABLED_V2_BATCH_READERS, "orc") + super.sparkConf.set(SQLConf.USE_V1_READERS, "orc") override def checkFilterPredicate( df: DataFrame, 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 index bf0f80027255e..7980004314c83 100644 --- 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 @@ -66,14 +66,14 @@ class FileDataSourceV2FallBackSuite extends QueryTest with ParquetTest with Shar "ParQuet,bar,foo", s"foobar,$dummyParquetReaderV2" ).foreach { fallbackReaders => - withSQLConf(SQLConf.DISABLED_V2_BATCH_READERS.key -> fallbackReaders) { + withSQLConf(SQLConf.USE_V1_READERS.key -> fallbackReaders) { // Reading file should fall back to v1 and succeed. checkAnswer(spark.read.format(dummyParquetReaderV2).load(path), df) checkAnswer(sql(s"SELECT * FROM parquet.`$path`"), df) } } - withSQLConf(SQLConf.DISABLED_V2_BATCH_READERS.key -> "foo,bar") { + withSQLConf(SQLConf.USE_V1_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) From c4b94c883d2e9833d98b1cf68651e9308449f60e Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Tue, 15 Jan 2019 22:12:11 +0800 Subject: [PATCH 26/31] update conf --- .../main/scala/org/apache/spark/sql/internal/SQLConf.scala | 4 ++-- .../src/main/scala/org/apache/spark/sql/DataFrameReader.scala | 2 +- .../datasources/orc/OrcPartitionDiscoverySuite.scala | 2 +- .../spark/sql/execution/datasources/orc/OrcQuerySuite.scala | 2 +- .../sql/execution/datasources/orc/OrcV1FilterSuite.scala | 2 +- .../spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala | 4 ++-- 6 files changed, 8 insertions(+), 8 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 dcec54f5715ff..4b46edc4f2e30 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 @@ -1419,7 +1419,7 @@ object SQLConf { .timeConf(TimeUnit.MILLISECONDS) .createWithDefault(100) - val USE_V1_READERS = buildConf("spark.sql.sources.useV1Readers") + val USE_V1_SOURCE_READER_LIST = buildConf("spark.sql.sources.read.useV1SourceList") .internal() .doc("A comma-separated list of data source short names or fully qualified data source" + " register class names for which read paths are disabled. Reads from these sources will" + @@ -2009,7 +2009,7 @@ class SQLConf extends Serializable with Logging { def continuousStreamingExecutorPollIntervalMs: Long = getConf(CONTINUOUS_STREAMING_EXECUTOR_POLL_INTERVAL_MS) - def useV1Readers: String = getConf(USE_V1_READERS) + def userV1SourceReaderList: String = getConf(USE_V1_SOURCE_READER_LIST) def disabledV2StreamingWriters: String = getConf(DISABLED_V2_STREAMING_WRITERS) 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 6ec107bd8f6be..5364abf3f60f8 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,7 +193,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { } val disabledV2Readers = - sparkSession.sessionState.conf.useV1Readers.toLowerCase(Locale.ROOT).split(",") + sparkSession.sessionState.conf.userV1SourceReaderList.toLowerCase(Locale.ROOT).split(",") val lookupCls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf) val cls = lookupCls.newInstance() match { case f: FileDataSourceV2 if disabledV2Readers.contains(f.shortName()) || 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 86f98ab83102e..4a695ac74c476 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.USE_V1_READERS, "orc") + super.sparkConf.set(SQLConf.USE_V1_SOURCE_READER_LIST, "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 5386ca0124d3d..d0b386b88c590 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 @@ -684,5 +684,5 @@ class OrcQuerySuite extends OrcQueryTest with SharedSQLContext { class OrcV1QuerySuite extends OrcQuerySuite { override protected def sparkConf: SparkConf = - super.sparkConf.set(SQLConf.USE_V1_READERS, "orc") + super.sparkConf.set(SQLConf.USE_V1_SOURCE_READER_LIST, "orc") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala index 5d0bb1c091e2d..cf5bbb3fff706 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.internal.SQLConf class OrcV1FilterSuite extends OrcFilterSuite { override protected def sparkConf: SparkConf = - super.sparkConf.set(SQLConf.USE_V1_READERS, "orc") + super.sparkConf.set(SQLConf.USE_V1_SOURCE_READER_LIST, "orc") override def checkFilterPredicate( df: DataFrame, 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 index 7980004314c83..5bed2fee4f2d2 100644 --- 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 @@ -66,14 +66,14 @@ class FileDataSourceV2FallBackSuite extends QueryTest with ParquetTest with Shar "ParQuet,bar,foo", s"foobar,$dummyParquetReaderV2" ).foreach { fallbackReaders => - withSQLConf(SQLConf.USE_V1_READERS.key -> fallbackReaders) { + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> fallbackReaders) { // Reading file should fall back to v1 and succeed. checkAnswer(spark.read.format(dummyParquetReaderV2).load(path), df) checkAnswer(sql(s"SELECT * FROM parquet.`$path`"), df) } } - withSQLConf(SQLConf.USE_V1_READERS.key -> "foo,bar") { + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.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) From 471ff1bf45cae6c141aab1623b642dc0120b6964 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Wed, 16 Jan 2019 19:30:04 +0800 Subject: [PATCH 27/31] use readSchema in supportColumnarReads; use dataSchema in buildColumnarReader --- .../datasources/v2/orc/OrcPartitionReaderFactory.scala | 7 +++---- .../spark/sql/execution/datasources/v2/orc/OrcScan.scala | 3 +-- .../sql/execution/datasources/v2/orc/OrcScanBuilder.scala | 2 +- 3 files changed, 5 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala index 976b45afd6d73..a8b59e89545d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala @@ -43,7 +43,6 @@ import org.apache.spark.util.SerializableConfiguration case class OrcPartitionReaderFactory( sqlConf: SQLConf, broadcastedConf: Broadcast[SerializableConfiguration], - schema: StructType, dataSchema: StructType, readSchema: StructType, partitionSchema: StructType) extends FilePartitionReaderFactory { @@ -52,8 +51,8 @@ case class OrcPartitionReaderFactory( override def supportColumnarReads(partition: InputPartition): Boolean = { sqlConf.orcVectorizedReaderEnabled && sqlConf.wholeStageEnabled && - schema.length <= sqlConf.wholeStageMaxNumFields && - schema.forall(_.dataType.isInstanceOf[AtomicType]) + readSchema.length <= sqlConf.wholeStageMaxNumFields && + readSchema.forall(_.dataType.isInstanceOf[AtomicType]) } override def buildReader(file: PartitionedFile): PartitionReader[InternalRow] = { @@ -111,7 +110,7 @@ case class OrcPartitionReaderFactory( val reader = OrcFile.createReader(filePath, readerOptions) val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( - isCaseSensitive, schema, readSchema, reader, conf) + isCaseSensitive, dataSchema, readSchema, reader, conf) if (requestedColIdsOrEmptyFile.isEmpty) { new EmptyPartitionReader diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala index 8d51dbdd3c2e7..cc56ff83b967b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala @@ -30,7 +30,6 @@ case class OrcScan( sparkSession: SparkSession, hadoopConf: Configuration, fileIndex: PartitioningAwareFileIndex, - schema: StructType, dataSchema: StructType, readSchema: StructType) extends FileScan(sparkSession, fileIndex) { override def isSplitable(path: Path): Boolean = true @@ -38,7 +37,7 @@ case class OrcScan( override def createReaderFactory(): PartitionReaderFactory = { val broadcastedConf = sparkSession.sparkContext.broadcast( new SerializableConfiguration(hadoopConf)) - OrcPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, schema, + OrcPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, dataSchema, readSchema, fileIndex.partitionSchema) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala index bf93ed051c1cc..4391b6b2958eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala @@ -40,7 +40,7 @@ case class OrcScanBuilder( sparkSession.sessionState.newHadoopConfWithOptions(options.asMap().asScala.toMap) override def build(): Scan = { - OrcScan(sparkSession, hadoopConf, fileIndex, schema, dataSchema, readSchema) + OrcScan(sparkSession, hadoopConf, fileIndex, dataSchema, readSchema) } private var _pushedFilters: Array[Filter] = Array.empty From 7da03ea9d613b03fd5ec8d988d170eb55565e5c0 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Wed, 16 Jan 2019 20:18:01 +0800 Subject: [PATCH 28/31] address more comments --- .../org/apache/spark/sql/internal/SQLConf.scala | 4 ++-- .../org/apache/spark/sql/DataFrameReader.scala | 6 +++--- .../org/apache/spark/sql/DataFrameWriter.scala | 2 +- .../execution/datasources/DataSourceStrategy.scala | 3 +-- .../datasources/v2/FilePartitionReaderFactory.scala | 2 +- .../v2/orc/OrcPartitionReaderFactory.scala | 13 +++++++++++-- .../sql/execution/datasources/v2/orc/OrcScan.scala | 2 +- .../datasources/v2/orc/OrcScanBuilder.scala | 2 ++ 8 files changed, 22 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 4b46edc4f2e30..ebc8c3705ea28 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 @@ -1422,8 +1422,8 @@ object SQLConf { val USE_V1_SOURCE_READER_LIST = buildConf("spark.sql.sources.read.useV1SourceList") .internal() .doc("A comma-separated list of data source short names or fully qualified data source" + - " register class names for which read paths are disabled. Reads from these sources will" + - " fall back to the V1 sources.") + " register class names for which data source V2 read paths are disabled. Reads from these" + + " sources will fall back to the V1 sources.") .stringConf .createWithDefault("") 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 5364abf3f60f8..bbc1f2356724b 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,12 +192,12 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { "read files of Hive data source directly.") } - val disabledV2Readers = + val useV1Sources = sparkSession.sessionState.conf.userV1SourceReaderList.toLowerCase(Locale.ROOT).split(",") val lookupCls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf) val cls = lookupCls.newInstance() match { - case f: FileDataSourceV2 if disabledV2Readers.contains(f.shortName()) || - disabledV2Readers.contains(lookupCls.getCanonicalName.toLowerCase(Locale.ROOT)) => + case f: FileDataSourceV2 if useV1Sources.contains(f.shortName()) || + useV1Sources.contains(lookupCls.getCanonicalName.toLowerCase(Locale.ROOT)) => f.fallBackFileFormat case _ => lookupCls } 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 54fb6d486f296..228dcb94b9acc 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 @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{AppendData, InsertIntoTable, 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.{DataSourceV2Relation, DataSourceV2Utils, FileDataSourceV2, WriteToDataSourceV2} +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2Utils, WriteToDataSourceV2} import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.sources.v2._ import org.apache.spark.sql.sources.v2.writer.SupportsSaveMode 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 12b60c4d3925a..b5cf8c9515bfb 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,10 +37,8 @@ import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoTab 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, FileDataSourceV2, FileTable} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ -import org.apache.spark.sql.sources.v2.DataSourceOptions import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -215,6 +213,7 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast } } + /** * Replaces [[UnresolvedCatalogRelation]] with concrete relation logical plans. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala index 7859b3f6bc024..101a70ee92ce5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala @@ -47,7 +47,7 @@ abstract class FilePartitionReaderFactory extends PartitionReaderFactory { } } -// A compound class for combining a input partitioned file and its corresponding file. +// A compound class for combining file and its corresponding reader. private[v2] class PartitionedFileReader[T]( file: PartitionedFile, reader: PartitionReader[T]) extends PartitionReader[T] { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala index a8b59e89545d6..f6fc0ca839081 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala @@ -40,12 +40,21 @@ import org.apache.spark.sql.types.{AtomicType, StructType} import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration +/** + * A factory used to create Orc readers. + * + * @param sqlConf SQL configuration. + * @param broadcastedConf Broadcast serializable Hadoop Configuration. + * @param dataSchema Schema of orc files. + * @param partitionSchema Schema of partitions. + * @param readSchema Required schema in the batch scan. + */ case class OrcPartitionReaderFactory( sqlConf: SQLConf, broadcastedConf: Broadcast[SerializableConfiguration], dataSchema: StructType, - readSchema: StructType, - partitionSchema: StructType) extends FilePartitionReaderFactory { + partitionSchema: StructType, + readSchema: StructType) extends FilePartitionReaderFactory { private val isCaseSensitive = sqlConf.caseSensitiveAnalysis private val capacity = sqlConf.orcVectorizedReaderBatchSize diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala index cc56ff83b967b..a792ad318b398 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala @@ -38,6 +38,6 @@ case class OrcScan( val broadcastedConf = sparkSession.sparkContext.broadcast( new SerializableConfiguration(hadoopConf)) OrcPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, - dataSchema, readSchema, fileIndex.partitionSchema) + dataSchema, fileIndex.partitionSchema, readSchema) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala index 4391b6b2958eb..eb27bbd3abeaa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala @@ -48,6 +48,8 @@ case class OrcScanBuilder( override def pushFilters(filters: Array[Filter]): Array[Filter] = { if (sparkSession.sessionState.conf.orcFilterPushDown) { OrcFilters.createFilter(schema, filters).foreach { f => + // The pushed filters will be set in `hadoopConf`. After that, we can simply use the + // changed `hadoopConf` in executors. OrcInputFormat.setSearchArgument(hadoopConf, f, schema.fieldNames) } val dataTypeMap = schema.map(f => f.name -> f.dataType).toMap From 0ce4a306af281120c06d5407f38f6ea94ccbc027 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Wed, 16 Jan 2019 23:13:32 +0800 Subject: [PATCH 29/31] update to latest master and address comments --- .../spark/sql/sources/v2/DataSourceOptions.java | 10 ++++++++++ .../org/apache/spark/sql/DataFrameReader.scala | 3 ++- .../org/apache/spark/sql/DataFrameWriter.scala | 3 ++- ...ceToV1.scala => FallbackOrcDataSourceV2.scala} | 15 ++++++++------- .../datasources/v2/FileDataSourceV2.scala | 2 +- .../sql/internal/BaseSessionStateBuilder.scala | 2 +- .../datasources/orc/OrcFilterSuite.scala | 4 ++-- .../spark/sql/hive/HiveSessionStateBuilder.scala | 2 +- 8 files changed, 27 insertions(+), 14 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/{FallbackFileDataSourceToV1.scala => FallbackOrcDataSourceV2.scala} (74%) diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceOptions.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceOptions.java index 1c5e3a0cd31e7..00af0bf1b172c 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceOptions.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceOptions.java @@ -163,6 +163,11 @@ public double getDouble(String key, double defaultValue) { */ public static final String DATABASE_KEY = "database"; + /** + * The option key for whether to check existence of files for a table. + */ + public static final String CHECK_FILES_EXIST_KEY = "check_files_exist"; + /** * Returns all the paths specified by both the singular path option and the multiple * paths option. @@ -197,4 +202,9 @@ public Optional tableName() { public Optional databaseName() { return get(DATABASE_KEY); } + + public Boolean checkFilesExist() { + Optional result = get(CHECK_FILES_EXIST_KEY); + return result.isPresent() && result.get().equals("true"); + } } 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 bbc1f2356724b..2b1521730bc07 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 @@ -210,7 +210,8 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { val objectMapper = new ObjectMapper() DataSourceOptions.PATHS_KEY -> objectMapper.writeValueAsString(paths.toArray) } - val finalOptions = sessionOptions ++ extraOptions.toMap + pathsOption + val checkFilesExistsOption = DataSourceOptions.CHECK_FILES_EXIST_KEY -> "true" + val finalOptions = sessionOptions ++ extraOptions.toMap + pathsOption + checkFilesExistsOption val dsOptions = new DataSourceOptions(finalOptions.asJava) val table = userSpecifiedSchema match { case Some(schema) => provider.getTable(dsOptions, schema) 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 228dcb94b9acc..d9404cd929925 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 @@ -248,7 +248,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val provider = cls.getConstructor().newInstance().asInstanceOf[TableProvider] val sessionOptions = DataSourceV2Utils.extractSessionConfigs( provider, session.sessionState.conf) - val options = sessionOptions ++ extraOptions + val checkFilesExistsOption = DataSourceOptions.CHECK_FILES_EXIST_KEY -> "false" + val options = sessionOptions ++ extraOptions + checkFilesExistsOption val dsOptions = new DataSourceOptions(options.asJava) provider.getTable(dsOptions) match { case table: SupportsBatchWrite => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallbackFileDataSourceToV1.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallbackOrcDataSourceV2.scala similarity index 74% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallbackFileDataSourceToV1.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallbackOrcDataSourceV2.scala index 34234a2f585f3..b9785cf28433e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallbackFileDataSourceToV1.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallbackOrcDataSourceV2.scala @@ -20,20 +20,21 @@ package org.apache.spark.sql.execution.datasources import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileDataSourceV2, FileTable} +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.OrcTable /** - * Replace the V2 data source of table in [[InsertIntoTable]] to V1 [[FileFormat]]. + * Replace the ORC 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. + * since there is no corresponding physical plan. * SPARK-23817: This is a temporary hack for making current data source V2 work. It should be * removed when write path of file data source v2 is finished. */ -class FallbackFileDataSourceToV1(sparkSession: SparkSession) extends Rule[LogicalPlan] { +class FallbackOrcDataSourceV2(sparkSession: SparkSession) extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case i @ InsertIntoTable(d @ - DataSourceV2Relation(source: FileDataSourceV2, table: FileTable, _, _, _), _, _, _, _) => - val v1FileFormat = source.fallBackFileFormat.getConstructor().newInstance() + case i @ InsertIntoTable(d @DataSourceV2Relation(table: OrcTable, _, _), _, _, _, _) => + val v1FileFormat = classOf[OrcFileFormat].getConstructor().newInstance() val relation = HadoopFsRelation(table.getFileIndex, table.getFileIndex.partitionSchema, table.schema(), None, v1FileFormat, d.options)(sparkSession) i.copy(table = LogicalRelation(relation)) 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 712e57863bd64..a0c932cbb0e09 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 @@ -46,7 +46,7 @@ trait FileDataSourceV2 extends TableProvider with DataSourceRegister { val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(options.asMap().asScala.toMap) val rootPathsSpecified = DataSource.checkAndGlobPathIfNecessary(filePaths, hadoopConf, - checkEmptyGlobPath = true, checkFilesExist = true) + checkEmptyGlobPath = true, checkFilesExist = options.checkFilesExist()) val fileStatusCache = FileStatusCache.getOrCreate(sparkSession) new InMemoryFileIndex(sparkSession, rootPathsSpecified, options.asMap().asScala.toMap, userSpecifiedSchema, fileStatusCache) 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 2319d6f421dbb..a605dc640dc96 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 @@ -159,7 +159,7 @@ abstract class BaseSessionStateBuilder( override val extendedResolutionRules: Seq[Rule[LogicalPlan]] = new FindDataSourceTable(session) +: new ResolveSQLOnFile(session) +: - new FallbackFileDataSourceToV1(session) +: + new FallbackOrcDataSourceV2(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 d0d422ffd36bb..cccd8e9ee8bd1 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 @@ -56,7 +56,7 @@ class OrcFilterSuite extends OrcTest with SharedSQLContext { query.queryExecution.optimizedPlan match { case PhysicalOperation(_, filters, - DataSourceV2Relation(_, orcTable: OrcTable, _, options, _)) => + DataSourceV2Relation(orcTable: OrcTable, _, options)) => assert(filters.nonEmpty, "No filter is analyzed from the given query") val scanBuilder = orcTable.newScanBuilder(new DataSourceOptions(options.asJava)) scanBuilder.pushFilters(filters.flatMap(DataSourceStrategy.translateFilter).toArray) @@ -100,7 +100,7 @@ class OrcFilterSuite extends OrcTest with SharedSQLContext { query.queryExecution.optimizedPlan match { case PhysicalOperation(_, filters, - DataSourceV2Relation(_, orcTable: OrcTable, _, options, _)) => + DataSourceV2Relation(orcTable: OrcTable, _, options)) => assert(filters.nonEmpty, "No filter is analyzed from the given query") val scanBuilder = orcTable.newScanBuilder(new DataSourceOptions(options.asJava)) scanBuilder.pushFilters(filters.flatMap(DataSourceStrategy.translateFilter).toArray) 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 be9eca15846e2..132b0e4db0d71 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,7 +71,7 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session new ResolveHiveSerdeTable(session) +: new FindDataSourceTable(session) +: new ResolveSQLOnFile(session) +: - new FallbackFileDataSourceToV1(session) +: + new FallbackOrcDataSourceV2(session) +: customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = From ff8608e9f99070a852f09885f5465cd09edba3e0 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Thu, 17 Jan 2019 00:44:43 +0800 Subject: [PATCH 30/31] update --- .../sql/execution/datasources/FallbackOrcDataSourceV2.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallbackOrcDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallbackOrcDataSourceV2.scala index b9785cf28433e..cefdfd14b3b30 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallbackOrcDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallbackOrcDataSourceV2.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.execution.datasources.v2.orc.OrcTable class FallbackOrcDataSourceV2(sparkSession: SparkSession) extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case i @ InsertIntoTable(d @DataSourceV2Relation(table: OrcTable, _, _), _, _, _, _) => - val v1FileFormat = classOf[OrcFileFormat].getConstructor().newInstance() + val v1FileFormat = new OrcFileFormat val relation = HadoopFsRelation(table.getFileIndex, table.getFileIndex.partitionSchema, table.schema(), None, v1FileFormat, d.options)(sparkSession) i.copy(table = LogicalRelation(relation)) From 6e875323a430cee190a458b8842adea44bb4e0b7 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Thu, 17 Jan 2019 12:08:24 +0800 Subject: [PATCH 31/31] fix test case --- .../v2/FileDataSourceV2FallBackSuite.scala | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) 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 index 5bed2fee4f2d2..f57c581fd800e 100644 --- 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 @@ -21,7 +21,9 @@ 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.ScanBuilder import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.StructType class DummyReadOnlyFileDataSourceV2 extends FileDataSourceV2 { @@ -30,6 +32,16 @@ class DummyReadOnlyFileDataSourceV2 extends FileDataSourceV2 { override def shortName(): String = "parquet" override def getTable(options: DataSourceOptions): Table = { + new DummyReadOnlyFileTable + } +} + +class DummyReadOnlyFileTable extends Table with SupportsBatchRead { + override def name(): String = "dummy" + + override def schema(): StructType = StructType(Nil) + + override def newScanBuilder(options: DataSourceOptions): ScanBuilder = { throw new AnalysisException("Dummy file reader") } } @@ -50,13 +62,13 @@ class FileDataSourceV2FallBackSuite extends QueryTest with ParquetTest with Shar // Dummy File reader should fail as expected. val exception = intercept[AnalysisException] { - spark.read.format(dummyParquetReaderV2).load(path) + spark.read.format(dummyParquetReaderV2).load(path).collect() } assert(exception.message.equals("Dummy file reader")) } } - test("Fall back read path to v1 with configuration DISABLED_V2_FILE_DATA_SOURCE_READERS") { + test("Fall back read path to v1 with configuration USE_V1_SOURCE_READER_LIST") { val df = spark.range(10).toDF() withTempPath { file => val path = file.getCanonicalPath @@ -76,7 +88,7 @@ class FileDataSourceV2FallBackSuite extends QueryTest with ParquetTest with Shar withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.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) + spark.read.format(dummyParquetReaderV2).load(path).collect() } assert(exception.message.equals("Dummy file reader")) }