[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:
Gengliang Wang 2019-06-20 12:57:13 +08:00 committed by Wenchen Fan
parent b276788d57
commit f5107614d6
5 changed files with 223 additions and 161 deletions

View file

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

View file

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

View file

@ -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") {

View file

@ -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)]

View file

@ -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 =>