[SPARK-28089][SQL] File source v2: support reading output of file streaming Sink
## What changes were proposed in this pull request? File source V1 supports reading output of FileStreamSink as batch. https://github.com/apache/spark/pull/11897 We should support this in file source V2 as well. When reading with paths, we first check if there is metadata log of FileStreamSink. If yes, we use `MetadataLogFileIndex` for listing files; Otherwise, we use `InMemoryFileIndex`. ## How was this patch tested? Unit test Closes #24900 from gengliangwang/FileStreamV2. Authored-by: Gengliang Wang <gengliang.wang@databricks.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
This commit is contained in:
parent
b276788d57
commit
f5107614d6
|
@ -20,11 +20,12 @@ import java.util
|
||||||
|
|
||||||
import scala.collection.JavaConverters._
|
import scala.collection.JavaConverters._
|
||||||
|
|
||||||
import org.apache.hadoop.fs.FileStatus
|
import org.apache.hadoop.fs.{FileStatus, Path}
|
||||||
|
|
||||||
import org.apache.spark.sql.{AnalysisException, SparkSession}
|
import org.apache.spark.sql.{AnalysisException, SparkSession}
|
||||||
import org.apache.spark.sql.catalog.v2.expressions.Transform
|
import org.apache.spark.sql.catalog.v2.expressions.Transform
|
||||||
import org.apache.spark.sql.execution.datasources._
|
import org.apache.spark.sql.execution.datasources._
|
||||||
|
import org.apache.spark.sql.execution.streaming.{FileStreamSink, MetadataLogFileIndex}
|
||||||
import org.apache.spark.sql.sources.v2.{SupportsRead, SupportsWrite, Table, TableCapability}
|
import org.apache.spark.sql.sources.v2.{SupportsRead, SupportsWrite, Table, TableCapability}
|
||||||
import org.apache.spark.sql.sources.v2.TableCapability._
|
import org.apache.spark.sql.sources.v2.TableCapability._
|
||||||
import org.apache.spark.sql.types.{DataType, StructType}
|
import org.apache.spark.sql.types.{DataType, StructType}
|
||||||
|
@ -44,14 +45,23 @@ abstract class FileTable(
|
||||||
val caseSensitiveMap = options.asCaseSensitiveMap.asScala.toMap
|
val caseSensitiveMap = options.asCaseSensitiveMap.asScala.toMap
|
||||||
// Hadoop Configurations are case sensitive.
|
// Hadoop Configurations are case sensitive.
|
||||||
val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(caseSensitiveMap)
|
val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(caseSensitiveMap)
|
||||||
|
if (FileStreamSink.hasMetadata(paths, hadoopConf, sparkSession.sessionState.conf)) {
|
||||||
|
// We are reading from the results of a streaming query. We will load files from
|
||||||
|
// the metadata log instead of listing them using HDFS APIs.
|
||||||
|
new MetadataLogFileIndex(sparkSession, new Path(paths.head),
|
||||||
|
options.asScala.toMap, userSpecifiedSchema)
|
||||||
|
} else {
|
||||||
|
// This is a non-streaming file based datasource.
|
||||||
val rootPathsSpecified = DataSource.checkAndGlobPathIfNecessary(paths, hadoopConf,
|
val rootPathsSpecified = DataSource.checkAndGlobPathIfNecessary(paths, hadoopConf,
|
||||||
checkEmptyGlobPath = true, checkFilesExist = true)
|
checkEmptyGlobPath = true, checkFilesExist = true)
|
||||||
val fileStatusCache = FileStatusCache.getOrCreate(sparkSession)
|
val fileStatusCache = FileStatusCache.getOrCreate(sparkSession)
|
||||||
new InMemoryFileIndex(
|
new InMemoryFileIndex(
|
||||||
sparkSession, rootPathsSpecified, caseSensitiveMap, userSpecifiedSchema, fileStatusCache)
|
sparkSession, rootPathsSpecified, caseSensitiveMap, userSpecifiedSchema, fileStatusCache)
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
lazy val dataSchema: StructType = userSpecifiedSchema.map { schema =>
|
lazy val dataSchema: StructType = {
|
||||||
|
val schema = userSpecifiedSchema.map { schema =>
|
||||||
val partitionSchema = fileIndex.partitionSchema
|
val partitionSchema = fileIndex.partitionSchema
|
||||||
val resolver = sparkSession.sessionState.conf.resolver
|
val resolver = sparkSession.sessionState.conf.resolver
|
||||||
StructType(schema.filterNot(f => partitionSchema.exists(p => resolver(p.name, f.name))))
|
StructType(schema.filterNot(f => partitionSchema.exists(p => resolver(p.name, f.name))))
|
||||||
|
@ -60,7 +70,12 @@ abstract class FileTable(
|
||||||
}.getOrElse {
|
}.getOrElse {
|
||||||
throw new AnalysisException(
|
throw new AnalysisException(
|
||||||
s"Unable to infer schema for $formatName. It must be specified manually.")
|
s"Unable to infer schema for $formatName. It must be specified manually.")
|
||||||
}.asNullable
|
}
|
||||||
|
fileIndex match {
|
||||||
|
case _: MetadataLogFileIndex => schema
|
||||||
|
case _ => schema.asNullable
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
override lazy val schema: StructType = {
|
override lazy val schema: StructType = {
|
||||||
val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis
|
val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis
|
||||||
|
|
|
@ -25,17 +25,19 @@ import scala.collection.JavaConverters._
|
||||||
|
|
||||||
import org.apache.hadoop.fs.Path
|
import org.apache.hadoop.fs.Path
|
||||||
|
|
||||||
|
import org.apache.spark.SparkConf
|
||||||
import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd}
|
import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd}
|
||||||
import org.apache.spark.sql.{AnalysisException, DataFrame}
|
import org.apache.spark.sql.{AnalysisException, DataFrame}
|
||||||
import org.apache.spark.sql.execution.DataSourceScanExec
|
import org.apache.spark.sql.execution.DataSourceScanExec
|
||||||
import org.apache.spark.sql.execution.datasources._
|
import org.apache.spark.sql.execution.datasources._
|
||||||
|
import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2Relation, FileScan, FileTable}
|
||||||
import org.apache.spark.sql.execution.streaming._
|
import org.apache.spark.sql.execution.streaming._
|
||||||
import org.apache.spark.sql.functions._
|
import org.apache.spark.sql.functions._
|
||||||
import org.apache.spark.sql.internal.SQLConf
|
import org.apache.spark.sql.internal.SQLConf
|
||||||
import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
|
import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
|
||||||
import org.apache.spark.util.Utils
|
import org.apache.spark.util.Utils
|
||||||
|
|
||||||
class FileStreamSinkSuite extends StreamTest {
|
abstract class FileStreamSinkSuite extends StreamTest {
|
||||||
import testImplicits._
|
import testImplicits._
|
||||||
|
|
||||||
override def beforeAll(): Unit = {
|
override def beforeAll(): Unit = {
|
||||||
|
@ -51,6 +53,8 @@ class FileStreamSinkSuite extends StreamTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected def checkQueryExecution(df: DataFrame): Unit
|
||||||
|
|
||||||
test("unpartitioned writing and batch reading") {
|
test("unpartitioned writing and batch reading") {
|
||||||
val inputData = MemoryStream[Int]
|
val inputData = MemoryStream[Int]
|
||||||
val df = inputData.toDF()
|
val df = inputData.toDF()
|
||||||
|
@ -121,8 +125,6 @@ class FileStreamSinkSuite extends StreamTest {
|
||||||
|
|
||||||
var query: StreamingQuery = null
|
var query: StreamingQuery = null
|
||||||
|
|
||||||
// TODO: test file source V2 as well.
|
|
||||||
withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> "parquet") {
|
|
||||||
try {
|
try {
|
||||||
query =
|
query =
|
||||||
ds.map(i => (i, i * 1000))
|
ds.map(i => (i, i * 1000))
|
||||||
|
@ -144,58 +146,18 @@ class FileStreamSinkSuite extends StreamTest {
|
||||||
.add(StructField("id", IntegerType))
|
.add(StructField("id", IntegerType))
|
||||||
assert(outputDf.schema === expectedSchema)
|
assert(outputDf.schema === expectedSchema)
|
||||||
|
|
||||||
// Verify that MetadataLogFileIndex is being used and the correct partitioning schema has
|
|
||||||
// been inferred
|
|
||||||
val hadoopdFsRelations = outputDf.queryExecution.analyzed.collect {
|
|
||||||
case LogicalRelation(baseRelation: HadoopFsRelation, _, _, _) => baseRelation
|
|
||||||
}
|
|
||||||
assert(hadoopdFsRelations.size === 1)
|
|
||||||
assert(hadoopdFsRelations.head.location.isInstanceOf[MetadataLogFileIndex])
|
|
||||||
assert(hadoopdFsRelations.head.partitionSchema.exists(_.name == "id"))
|
|
||||||
assert(hadoopdFsRelations.head.dataSchema.exists(_.name == "value"))
|
|
||||||
|
|
||||||
// Verify the data is correctly read
|
// Verify the data is correctly read
|
||||||
checkDatasetUnorderly(
|
checkDatasetUnorderly(
|
||||||
outputDf.as[(Int, Int)],
|
outputDf.as[(Int, Int)],
|
||||||
(1000, 1), (2000, 2), (3000, 3))
|
(1000, 1), (2000, 2), (3000, 3))
|
||||||
|
|
||||||
/** Check some condition on the partitions of the FileScanRDD generated by a DF */
|
checkQueryExecution(outputDf)
|
||||||
def checkFileScanPartitions(df: DataFrame)(func: Seq[FilePartition] => Unit): Unit = {
|
|
||||||
val getFileScanRDD = df.queryExecution.executedPlan.collect {
|
|
||||||
case scan: DataSourceScanExec if scan.inputRDDs().head.isInstanceOf[FileScanRDD] =>
|
|
||||||
scan.inputRDDs().head.asInstanceOf[FileScanRDD]
|
|
||||||
}.headOption.getOrElse {
|
|
||||||
fail(s"No FileScan in query\n${df.queryExecution}")
|
|
||||||
}
|
|
||||||
func(getFileScanRDD.filePartitions)
|
|
||||||
}
|
|
||||||
|
|
||||||
// Read without pruning
|
|
||||||
checkFileScanPartitions(outputDf) { partitions =>
|
|
||||||
// There should be as many distinct partition values as there are distinct ids
|
|
||||||
assert(partitions.flatMap(_.files.map(_.partitionValues)).distinct.size === 3)
|
|
||||||
}
|
|
||||||
|
|
||||||
// Read with pruning, should read only files in partition dir id=1
|
|
||||||
checkFileScanPartitions(outputDf.filter("id = 1")) { partitions =>
|
|
||||||
val filesToBeRead = partitions.flatMap(_.files)
|
|
||||||
assert(filesToBeRead.map(_.filePath).forall(_.contains("/id=1/")))
|
|
||||||
assert(filesToBeRead.map(_.partitionValues).distinct.size === 1)
|
|
||||||
}
|
|
||||||
|
|
||||||
// Read with pruning, should read only files in partition dir id=1 and id=2
|
|
||||||
checkFileScanPartitions(outputDf.filter("id in (1,2)")) { partitions =>
|
|
||||||
val filesToBeRead = partitions.flatMap(_.files)
|
|
||||||
assert(!filesToBeRead.map(_.filePath).exists(_.contains("/id=3/")))
|
|
||||||
assert(filesToBeRead.map(_.partitionValues).distinct.size === 2)
|
|
||||||
}
|
|
||||||
} finally {
|
} finally {
|
||||||
if (query != null) {
|
if (query != null) {
|
||||||
query.stop()
|
query.stop()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
test("partitioned writing and batch reading with 'basePath'") {
|
test("partitioned writing and batch reading with 'basePath'") {
|
||||||
withTempDir { outputDir =>
|
withTempDir { outputDir =>
|
||||||
|
@ -512,3 +474,92 @@ class FileStreamSinkSuite extends StreamTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
class FileStreamSinkV1Suite extends FileStreamSinkSuite {
|
||||||
|
override protected def sparkConf: SparkConf =
|
||||||
|
super
|
||||||
|
.sparkConf
|
||||||
|
.set(SQLConf.USE_V1_SOURCE_READER_LIST, "csv,json,orc,text,parquet")
|
||||||
|
.set(SQLConf.USE_V1_SOURCE_WRITER_LIST, "csv,json,orc,text,parquet")
|
||||||
|
|
||||||
|
override def checkQueryExecution(df: DataFrame): Unit = {
|
||||||
|
// Verify that MetadataLogFileIndex is being used and the correct partitioning schema has
|
||||||
|
// been inferred
|
||||||
|
val hadoopdFsRelations = df.queryExecution.analyzed.collect {
|
||||||
|
case LogicalRelation(baseRelation: HadoopFsRelation, _, _, _) => baseRelation
|
||||||
|
}
|
||||||
|
assert(hadoopdFsRelations.size === 1)
|
||||||
|
assert(hadoopdFsRelations.head.location.isInstanceOf[MetadataLogFileIndex])
|
||||||
|
assert(hadoopdFsRelations.head.partitionSchema.exists(_.name == "id"))
|
||||||
|
assert(hadoopdFsRelations.head.dataSchema.exists(_.name == "value"))
|
||||||
|
|
||||||
|
/** Check some condition on the partitions of the FileScanRDD generated by a DF */
|
||||||
|
def checkFileScanPartitions(df: DataFrame)(func: Seq[FilePartition] => Unit): Unit = {
|
||||||
|
val getFileScanRDD = df.queryExecution.executedPlan.collect {
|
||||||
|
case scan: DataSourceScanExec if scan.inputRDDs().head.isInstanceOf[FileScanRDD] =>
|
||||||
|
scan.inputRDDs().head.asInstanceOf[FileScanRDD]
|
||||||
|
}.headOption.getOrElse {
|
||||||
|
fail(s"No FileScan in query\n${df.queryExecution}")
|
||||||
|
}
|
||||||
|
func(getFileScanRDD.filePartitions)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Read without pruning
|
||||||
|
checkFileScanPartitions(df) { partitions =>
|
||||||
|
// There should be as many distinct partition values as there are distinct ids
|
||||||
|
assert(partitions.flatMap(_.files.map(_.partitionValues)).distinct.size === 3)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Read with pruning, should read only files in partition dir id=1
|
||||||
|
checkFileScanPartitions(df.filter("id = 1")) { partitions =>
|
||||||
|
val filesToBeRead = partitions.flatMap(_.files)
|
||||||
|
assert(filesToBeRead.map(_.filePath).forall(_.contains("/id=1/")))
|
||||||
|
assert(filesToBeRead.map(_.partitionValues).distinct.size === 1)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Read with pruning, should read only files in partition dir id=1 and id=2
|
||||||
|
checkFileScanPartitions(df.filter("id in (1,2)")) { partitions =>
|
||||||
|
val filesToBeRead = partitions.flatMap(_.files)
|
||||||
|
assert(!filesToBeRead.map(_.filePath).exists(_.contains("/id=3/")))
|
||||||
|
assert(filesToBeRead.map(_.partitionValues).distinct.size === 2)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
class FileStreamSinkV2Suite extends FileStreamSinkSuite {
|
||||||
|
override protected def sparkConf: SparkConf =
|
||||||
|
super
|
||||||
|
.sparkConf
|
||||||
|
.set(SQLConf.USE_V1_SOURCE_READER_LIST, "")
|
||||||
|
.set(SQLConf.USE_V1_SOURCE_WRITER_LIST, "")
|
||||||
|
|
||||||
|
override def checkQueryExecution(df: DataFrame): Unit = {
|
||||||
|
// Verify that MetadataLogFileIndex is being used and the correct partitioning schema has
|
||||||
|
// been inferred
|
||||||
|
val table = df.queryExecution.analyzed.collect {
|
||||||
|
case DataSourceV2Relation(table: FileTable, _, _) => table
|
||||||
|
}
|
||||||
|
assert(table.size === 1)
|
||||||
|
assert(table.head.fileIndex.isInstanceOf[MetadataLogFileIndex])
|
||||||
|
assert(table.head.fileIndex.partitionSchema.exists(_.name == "id"))
|
||||||
|
assert(table.head.dataSchema.exists(_.name == "value"))
|
||||||
|
|
||||||
|
/** Check some condition on the partitions of the FileScanRDD generated by a DF */
|
||||||
|
def checkFileScanPartitions(df: DataFrame)(func: Seq[FilePartition] => Unit): Unit = {
|
||||||
|
val fileScan = df.queryExecution.executedPlan.collect {
|
||||||
|
case batch: BatchScanExec if batch.scan.isInstanceOf[FileScan] =>
|
||||||
|
batch.scan.asInstanceOf[FileScan]
|
||||||
|
}.headOption.getOrElse {
|
||||||
|
fail(s"No FileScan in query\n${df.queryExecution}")
|
||||||
|
}
|
||||||
|
func(fileScan.planInputPartitions().map(_.asInstanceOf[FilePartition]))
|
||||||
|
}
|
||||||
|
|
||||||
|
// Read without pruning
|
||||||
|
checkFileScanPartitions(df) { partitions =>
|
||||||
|
// There should be as many distinct partition values as there are distinct ids
|
||||||
|
assert(partitions.flatMap(_.files.map(_.partitionValues)).distinct.size === 3)
|
||||||
|
}
|
||||||
|
// TODO: test partition pruning when file source V2 supports it.
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
|
@ -218,13 +218,14 @@ class StreamSuite extends StreamTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO: fix file source V2 as well.
|
|
||||||
withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> "parquet") {
|
|
||||||
val df = spark.readStream.format(classOf[FakeDefaultSource].getName).load()
|
val df = spark.readStream.format(classOf[FakeDefaultSource].getName).load()
|
||||||
|
Seq("", "parquet").foreach { useV1SourceReader =>
|
||||||
|
withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> useV1SourceReader) {
|
||||||
assertDF(df)
|
assertDF(df)
|
||||||
assertDF(df)
|
assertDF(df)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
test("Within the same streaming query, one StreamingRelation should only be transformed to one " +
|
test("Within the same streaming query, one StreamingRelation should only be transformed to one " +
|
||||||
"StreamingExecutionRelation") {
|
"StreamingExecutionRelation") {
|
||||||
|
|
|
@ -197,7 +197,6 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
test("deduplicate with file sink") {
|
test("deduplicate with file sink") {
|
||||||
withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> "parquet") {
|
|
||||||
withTempDir { output =>
|
withTempDir { output =>
|
||||||
withTempDir { checkpointDir =>
|
withTempDir { checkpointDir =>
|
||||||
val outputPath = output.getAbsolutePath
|
val outputPath = output.getAbsolutePath
|
||||||
|
@ -226,7 +225,6 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
test("SPARK-19841: watermarkPredicate should filter based on keys") {
|
test("SPARK-19841: watermarkPredicate should filter based on keys") {
|
||||||
val input = MemoryStream[(Int, Int)]
|
val input = MemoryStream[(Int, Int)]
|
||||||
|
|
|
@ -995,8 +995,6 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi
|
||||||
|
|
||||||
// Reading a file sink output in a batch query should detect the legacy _spark_metadata
|
// Reading a file sink output in a batch query should detect the legacy _spark_metadata
|
||||||
// directory and throw an error
|
// directory and throw an error
|
||||||
// TODO: test file source V2 as well.
|
|
||||||
withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> "parquet") {
|
|
||||||
val e = intercept[SparkException] {
|
val e = intercept[SparkException] {
|
||||||
spark.read.load(outputDir.getCanonicalPath).as[Int]
|
spark.read.load(outputDir.getCanonicalPath).as[Int]
|
||||||
}
|
}
|
||||||
|
@ -1050,7 +1048,6 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
test("ignore the escaped path check when the flag is off") {
|
test("ignore the escaped path check when the flag is off") {
|
||||||
withTempDir { tempDir =>
|
withTempDir { tempDir =>
|
||||||
|
|
Loading…
Reference in a new issue