Skip to content

[SPARK-23817][SQL] Create file source V2 framework and migrate ORC read path #23383

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 31 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
31 commits
Select commit Hold shift + click to select a range
5d5f59e
orc data source v2
gengliangwang Jan 3, 2019
9ac801e
better data source fallback
gengliangwang Jan 6, 2019
5dd34c4
address comments
gengliangwang Jan 6, 2019
5f35e75
remove partition pruning for now
gengliangwang Jan 6, 2019
cdb1f99
fix hive tests
gengliangwang Jan 7, 2019
c1e1f4b
add comment and revise
gengliangwang Jan 8, 2019
ba66011
fix test failure
gengliangwang Jan 8, 2019
91b146b
fix checkAnswer
gengliangwang Jan 8, 2019
91475ca
remove useless code
gengliangwang Jan 8, 2019
38a27f2
use DataSource.providingClass for falling back to V1 when with catalog
gengliangwang Jan 9, 2019
9bbd87b
address comments
gengliangwang Jan 9, 2019
38b8af5
revise comment
gengliangwang Jan 9, 2019
c91b84a
don't use FileIndex to get SparkSession, hadoopconf, and data source …
gengliangwang Jan 9, 2019
2c43797
move method to orc factory
gengliangwang Jan 10, 2019
af9de22
use SparkSession in OrcPartitionReaderFactory
gengliangwang Jan 10, 2019
2f7b3c6
update the latest master
gengliangwang Jan 10, 2019
91aeafb
fix regression of merging to latest master
gengliangwang Jan 11, 2019
5ebeda0
don't change OrcFileFormat's short name
gengliangwang Jan 11, 2019
3b50f2d
address comments
gengliangwang Jan 11, 2019
ea8f178
address more comments
gengliangwang Jan 11, 2019
aff3788
check file existence
gengliangwang Jan 12, 2019
575643b
address comments
gengliangwang Jan 13, 2019
ac8acdd
update sqlconf
gengliangwang Jan 14, 2019
13d615b
address comments
gengliangwang Jan 14, 2019
a1e66f3
change conf to spark.sql.sources.useV1Readers
gengliangwang Jan 15, 2019
c4b94c8
update conf
gengliangwang Jan 15, 2019
471ff1b
use readSchema in supportColumnarReads; use dataSchema in buildColumn…
gengliangwang Jan 16, 2019
7da03ea
address more comments
gengliangwang Jan 16, 2019
0ce4a30
update to latest master and address comments
gengliangwang Jan 16, 2019
ff8608e
update
gengliangwang Jan 16, 2019
6e87532
fix test case
gengliangwang Jan 17, 2019
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -1419,8 +1419,15 @@ object SQLConf {
.timeConf(TimeUnit.MILLISECONDS)
.createWithDefault(100)

val DISABLED_V2_STREAMING_WRITERS = buildConf("spark.sql.streaming.disabledV2Writers")
val USE_V1_SOURCE_READER_LIST = buildConf("spark.sql.sources.read.useV1SourceList")
.internal()
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Given the overall DSv2 risk, shall we make this public and add migration doc officially, @gatorsmile ?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the streaming flag is also internal, if we want to change it, let's change them together in a followup PR.

.doc("A comma-separated list of data source short names or fully qualified data source" +
" 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("")

val DISABLED_V2_STREAMING_WRITERS = buildConf("spark.sql.streaming.disabledV2Writers")
.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
Expand Down Expand Up @@ -2002,6 +2009,8 @@ class SQLConf extends Serializable with Logging {
def continuousStreamingExecutorPollIntervalMs: Long =
getConf(CONTINUOUS_STREAMING_EXECUTOR_POLL_INTERVAL_MS)

def userV1SourceReaderList: String = getConf(USE_V1_SOURCE_READER_LIST)

def disabledV2StreamingWriters: String = getConf(DISABLED_V2_STREAMING_WRITERS)

def disabledV2StreamingMicroBatchReaders: String =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -197,4 +202,9 @@ public Optional<String> tableName() {
public Optional<String> databaseName() {
return get(DATABASE_KEY);
}

public Boolean checkFilesExist() {
Optional<String> result = get(CHECK_FILES_EXIST_KEY);
return result.isPresent() && result.get().equals("true");
}
}
18 changes: 14 additions & 4 deletions sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -193,7 +192,16 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging {
"read files of Hive data source directly.")
}

val cls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf)
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 useV1Sources.contains(f.shortName()) ||
useV1Sources.contains(lookupCls.getCanonicalName.toLowerCase(Locale.ROOT)) =>
f.fallBackFileFormat
case _ => lookupCls
}

if (classOf[TableProvider].isAssignableFrom(cls)) {
val provider = cls.getConstructor().newInstance().asInstanceOf[TableProvider]
val sessionOptions = DataSourceV2Utils.extractSessionConfigs(
Expand All @@ -202,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)
Expand All @@ -211,6 +220,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging {
table match {
case _: SupportsBatchRead =>
Dataset.ofRows(sparkSession, DataSourceV2Relation.create(table, finalOptions))

case _ => loadV1Source(paths: _*)
}
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -410,107 +409,35 @@ 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 =
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.")

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 =
FilePartition.getFilePartitions(relation.sparkSession, splitFiles, maxSplitBytes)

new FileScanRDD(fsRelation.sparkSession, readFile, partitions)
}

private def getBlockLocations(file: FileStatus): Array[BlockLocation] = file match {
case f: LocatedFileStatus => f.getBlockLocations
case f => Array.empty[BlockLocation]
}

// Given locations of all blocks of a single file, `blockLocations`, and an `(offset, length)`
// pair that represents a segment of the same file, find out the block that contains the largest
// fraction the segment, and returns location hosts of that block. If no such block can be found,
// returns an empty array.
private def getBlockHosts(
blockLocations: Array[BlockLocation], offset: Long, length: Long): Array[String] = {
val candidates = blockLocations.map {
// The fragment starts from a position within this block
case b if b.getOffset <= offset && offset < b.getOffset + b.getLength =>
b.getHosts -> (b.getOffset + b.getLength - offset).min(length)

// The fragment ends at a position within this block
case b if offset <= b.getOffset && offset + length < b.getLength =>
b.getHosts -> (offset + length - b.getOffset).min(length)

// The fragment fully contains this block
case b if offset <= b.getOffset && b.getOffset + b.getLength <= offset + length =>
b.getHosts -> b.getLength

// The fragment doesn't intersect with this block
case b =>
b.getHosts -> 0L
}.filter { case (hosts, size) =>
size > 0L
}

if (candidates.isEmpty) {
Array.empty[String]
} else {
val (hosts, _) = candidates.maxBy { case (_, size) => size }
hosts
}
}

override def doCanonicalize(): FileSourceScanExec = {
FileSourceScanExec(
relation,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,94 @@
/*
* 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._

object PartitionedFileUtil {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Now, we have PartitionedFileUtil and FilePartitionUtil. For most code readers, they are not clear about the difference. I am afraid these util objects will be misused. Could you add the description for both the utility objects and the functions?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have renamed FilePartitionUtil as FilePartition, also put the class FilePartition into the same file. I think they are much more clear now.

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)
}

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
}
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -214,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,
Expand All @@ -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(
Expand Down
Loading