diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 1fb0a338299b..dc0bce90c3f3 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -310,7 +310,7 @@ private[kafka010] class KafkaSource( currentPartitionOffsets = Some(untilPartitionOffsets) } - sqlContext.internalCreateDataFrame(rdd, schema) + markAsStreaming(sqlContext.internalCreateDataFrame(rdd, schema)) } /** Stop this source and free any resources it has allocated. */ diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index 7b6396e0291c..b7bd65d537bc 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -612,6 +612,33 @@ class KafkaSourceSuite extends KafkaSourceTest { assert(query.exception.isEmpty) } + test("getBatch should return a streaming DataFrame") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 5) + testUtils.sendMessages(topic, Array("-1")) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + val kafka = spark + .readStream + .format("kafka") + .option("subscribe", topic) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + + val mapped = kafka.map(kv => kv._2.toInt + 1) + testStream(mapped)( + StartStream(trigger = ProcessingTime(1)), + makeSureGetOffsetCalled, + AddKafkaData(Set(topic), 1, 2, 3), + CheckAnswer(2, 3, 4), + AssertOnQuery { query => + query.lastExecution.logical.isStreaming + } + ) + } + test("get offsets from case insensitive parameters") { for ((optionKey, optionValue, answer) <- Seq( (STARTING_OFFSETS_OPTION_KEY, "earLiEst", EarliestOffsetRangeLimit), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index c8ed4190a13a..78179ca357d5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -1128,11 +1128,12 @@ case class DecimalAggregates(conf: CatalystConf) extends Rule[LogicalPlan] { */ object ConvertToLocalRelation extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case Project(projectList, LocalRelation(output, data)) + case Project(projectList, lr @ LocalRelation(output, data, dataFromStreaming)) if !projectList.exists(hasUnevaluableExpr) => val projection = new InterpretedProjection(projectList, output) projection.initialize(0) - LocalRelation(projectList.map(_.toAttribute), data.map(projection)) + LocalRelation(projectList.map(_.toAttribute), data.map(projection), + dataFromStreaming = dataFromStreaming) } private def hasUnevaluableExpr(expr: Expression): Boolean = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala index 1faabcfcb73b..712bb5d67f40 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala @@ -43,7 +43,15 @@ object LocalRelation { } } -case class LocalRelation(output: Seq[Attribute], data: Seq[InternalRow] = Nil) +/** + * @param dataFromStreaming indicate if this relation comes from a streaming source. + * In a streaming query, stream relation will be cut into a + * series of batch relations. + */ +case class LocalRelation( + output: Seq[Attribute], + data: Seq[InternalRow] = Nil, + var dataFromStreaming: Boolean = false) extends LeafNode with analysis.MultiInstanceRelation { // A local relation must have resolved output. @@ -68,12 +76,14 @@ case class LocalRelation(output: Seq[Attribute], data: Seq[InternalRow] = Nil) override def sameResult(plan: LogicalPlan): Boolean = { plan.canonicalized match { - case LocalRelation(otherOutput, otherData) => + case LocalRelation(otherOutput, otherData, _) => otherOutput.map(_.dataType) == output.map(_.dataType) && otherData == data case _ => false } } + override def isStreaming: Boolean = dataFromStreaming + override def computeStats(conf: CatalystConf): Statistics = Statistics(sizeInBytes = output.map(n => BigInt(n.dataType.defaultSize)).sum * data.length) 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 3e975ef6a3c2..433028cffdfc 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 @@ -371,7 +371,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { case (true, SaveMode.Overwrite) => // Get all input data source or hive relations of the query. val srcRelations = df.logicalPlan.collect { - case LogicalRelation(src: BaseRelation, _, _) => src + case LogicalRelation(src: BaseRelation, _, _, _) => src case relation: CatalogRelation if DDLUtils.isHiveTable(relation.tableMeta) => relation.tableMeta.identifier } @@ -379,7 +379,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val tableRelation = df.sparkSession.table(tableIdentWithDB).queryExecution.analyzed EliminateSubqueryAliases(tableRelation) match { // check if the table is a data source table (the relation is a BaseRelation). - case LogicalRelation(dest: BaseRelation, _, _) if srcRelations.contains(dest) => + case LogicalRelation(dest: BaseRelation, _, _, _) if srcRelations.contains(dest) => throw new AnalysisException( s"Cannot overwrite table $tableName that is also being read from") // check hive table relation when overwrite mode diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 520663f62440..2f964430b7b9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -2731,7 +2731,7 @@ class Dataset[T] private[sql]( */ def inputFiles: Array[String] = { val files: Seq[String] = queryExecution.optimizedPlan.collect { - case LogicalRelation(fsBasedRelation: FileRelation, _, _) => + case LogicalRelation(fsBasedRelation: FileRelation, _, _, _) => fsBasedRelation.inputFiles case fr: FileRelation => fr.inputFiles diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 49336f424822..27d118c6a78f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -127,12 +127,18 @@ case class ExternalRDDScanExec[T]( } } -/** Logical plan node for scanning data from an RDD of InternalRow. */ +/** Logical plan node for scanning data from an RDD of InternalRow. + * + * @param dataFromStreaming indicate if this relation comes from a streaming source. + * In a streaming query, stream relation will be cut into a + * series of batch relations. + */ case class LogicalRDD( output: Seq[Attribute], rdd: RDD[InternalRow], outputPartitioning: Partitioning = UnknownPartitioning(0), - outputOrdering: Seq[SortOrder] = Nil)(session: SparkSession) + outputOrdering: Seq[SortOrder] = Nil, + var dataFromStreaming: Boolean = false)(session: SparkSession) extends LeafNode with MultiInstanceRelation { override protected final def otherCopyArgs: Seq[AnyRef] = session :: Nil @@ -163,11 +169,13 @@ case class LogicalRDD( override def sameResult(plan: LogicalPlan): Boolean = { plan.canonicalized match { - case LogicalRDD(_, otherRDD, _, _) => rdd.id == otherRDD.id + case LogicalRDD(_, otherRDD, _, _, _) => rdd.id == otherRDD.id case _ => false } } + override def isStreaming: Boolean = dataFromStreaming + override protected def stringArgs: Iterator[Any] = Iterator(output) @transient override def computeStats(conf: CatalystConf): Statistics = Statistics( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala index 3c046ce49428..a2117979ab34 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala @@ -96,7 +96,7 @@ case class OptimizeMetadataOnlyQuery( child transform { case plan if plan eq relation => relation match { - case l @ LogicalRelation(fsRelation: HadoopFsRelation, _, _) => + case l @ LogicalRelation(fsRelation: HadoopFsRelation, _, _, _) => val partAttrs = getPartitionAttrs(fsRelation.partitionSchema.map(_.name), l) val partitionData = fsRelation.location.listFiles(Nil, Nil) LocalRelation(partAttrs, partitionData.map(_.values)) @@ -132,7 +132,7 @@ case class OptimizeMetadataOnlyQuery( object PartitionedRelation { def unapply(plan: LogicalPlan): Option[(AttributeSet, LogicalPlan)] = plan match { - case l @ LogicalRelation(fsRelation: HadoopFsRelation, _, _) + case l @ LogicalRelation(fsRelation: HadoopFsRelation, _, _, _) if fsRelation.partitionSchema.nonEmpty => val partAttrs = getPartitionAttrs(fsRelation.partitionSchema.map(_.name), l) Some(AttributeSet(partAttrs), l) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 9e58e8ce3d5f..c0e6d6351b93 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -231,8 +231,9 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case EventTimeWatermark(columnName, delay, child) => EventTimeWatermarkExec(columnName, delay, planLater(child)) :: Nil - case PhysicalAggregation( - namedGroupingExpressions, aggregateExpressions, rewrittenResultExpressions, child) => + case agg @ PhysicalAggregation( + namedGroupingExpressions, aggregateExpressions, rewrittenResultExpressions, child) + if agg.isStreaming => aggregate.AggUtils.planStreamingAggregation( namedGroupingExpressions, @@ -411,7 +412,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.window.WindowExec(windowExprs, partitionSpec, orderSpec, planLater(child)) :: Nil case logical.Sample(lb, ub, withReplacement, seed, child) => execution.SampleExec(lb, ub, withReplacement, seed, planLater(child)) :: Nil - case logical.LocalRelation(output, data) => + case logical.LocalRelation(output, data, _) => LocalTableScanExec(output, data) :: Nil case logical.LocalLimit(IntegerLiteral(limit), child) => execution.LocalLimitExec(limit, planLater(child)) :: Nil 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 c9384e44255b..6be8e1d83f5c 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 @@ -421,7 +421,7 @@ case class DataSource( } val fileIndex = catalogTable.map(_.identifier).map { tableIdent => sparkSession.table(tableIdent).queryExecution.analyzed.collect { - case LogicalRelation(t: HadoopFsRelation, _, _) => t.location + case LogicalRelation(t: HadoopFsRelation, _, _, _) => t.location }.head } // For partitioned relation r, r.schema's column ordering can be different from the column 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 bddf5af23e06..7bf942a5a2fb 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 @@ -140,12 +140,12 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { if query.resolved && DDLUtils.isDatasourceTable(tableDesc) => CreateDataSourceTableAsSelectCommand(tableDesc, mode, query) - case InsertIntoTable(l @ LogicalRelation(_: InsertableRelation, _, _), + case InsertIntoTable(l @ LogicalRelation(_: InsertableRelation, _, _, _), parts, query, overwrite, false) if parts.isEmpty => InsertIntoDataSourceCommand(l, query, overwrite) case InsertIntoTable( - l @ LogicalRelation(t: HadoopFsRelation, _, table), parts, query, overwrite, false) => + l @ LogicalRelation(t: HadoopFsRelation, _, table, _), parts, query, overwrite, false) => // If the InsertIntoTable command is for a partitioned HadoopFsRelation and // the user has specified static partitions, we add a Project operator on top of the query // to include those constant column values in the query result. @@ -181,7 +181,7 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { val outputPath = t.location.rootPaths.head val inputPaths = actualQuery.collect { - case LogicalRelation(r: HadoopFsRelation, _, _) => r.location.rootPaths + case LogicalRelation(r: HadoopFsRelation, _, _, _) => r.location.rootPaths }.flatten val mode = if (overwrite) SaveMode.Overwrite else SaveMode.Append @@ -264,7 +264,7 @@ class FindDataSourceTable(sparkSession: SparkSession) extends Rule[LogicalPlan] */ object DataSourceStrategy extends Strategy with Logging { def apply(plan: LogicalPlan): Seq[execution.SparkPlan] = plan match { - case PhysicalOperation(projects, filters, l @ LogicalRelation(t: CatalystScan, _, _)) => + case PhysicalOperation(projects, filters, l @ LogicalRelation(t: CatalystScan, _, _, _)) => pruneFilterProjectRaw( l, projects, @@ -272,21 +272,22 @@ object DataSourceStrategy extends Strategy with Logging { (requestedColumns, allPredicates, _) => toCatalystRDD(l, requestedColumns, t.buildScan(requestedColumns, allPredicates))) :: Nil - case PhysicalOperation(projects, filters, l @ LogicalRelation(t: PrunedFilteredScan, _, _)) => + case PhysicalOperation( + projects, filters, l @ LogicalRelation(t: PrunedFilteredScan, _, _, _)) => pruneFilterProject( l, projects, filters, (a, f) => toCatalystRDD(l, a, t.buildScan(a.map(_.name).toArray, f))) :: Nil - case PhysicalOperation(projects, filters, l @ LogicalRelation(t: PrunedScan, _, _)) => + case PhysicalOperation(projects, filters, l @ LogicalRelation(t: PrunedScan, _, _, _)) => pruneFilterProject( l, projects, filters, (a, _) => toCatalystRDD(l, a, t.buildScan(a.map(_.name).toArray))) :: Nil - case l @ LogicalRelation(baseRelation: TableScan, _, _) => + case l @ LogicalRelation(baseRelation: TableScan, _, _, _) => RowDataSourceScanExec( l.output, toCatalystRDD(l, baseRelation.buildScan()), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index 17f7e0e601c0..16b22717b8d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -52,7 +52,7 @@ import org.apache.spark.sql.execution.SparkPlan object FileSourceStrategy extends Strategy with Logging { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case PhysicalOperation(projects, filters, - l @ LogicalRelation(fsRelation: HadoopFsRelation, _, table)) => + l @ LogicalRelation(fsRelation: HadoopFsRelation, _, table, _)) => // Filters on this relation fall into four categories based on where we can use them to avoid // reading unneeded data: // - partition keys only - used to prune directories to read diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala index 04a764bee2ef..c885ad0927b4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala @@ -30,11 +30,16 @@ import org.apache.spark.util.Utils * Note that sometimes we need to use `LogicalRelation` to replace an existing leaf node without * changing the output attributes' IDs. The `expectedOutputAttributes` parameter is used for * this purpose. See https://issues.apache.org/jira/browse/SPARK-10741 for more details. + * + * @param dataFromStreaming indicate if this relation comes from a streaming source. + * In a streaming query, stream relation will be cut into a + * series of batch relations. */ case class LogicalRelation( relation: BaseRelation, expectedOutputAttributes: Option[Seq[Attribute]] = None, - catalogTable: Option[CatalogTable] = None) + catalogTable: Option[CatalogTable] = None, + var dataFromStreaming: Boolean = false) extends LeafNode with MultiInstanceRelation { override val output: Seq[AttributeReference] = { @@ -53,7 +58,8 @@ case class LogicalRelation( // Logical Relations are distinct if they have different output for the sake of transformations. override def equals(other: Any): Boolean = other match { - case l @ LogicalRelation(otherRelation, _, _) => relation == otherRelation && output == l.output + case l @ LogicalRelation(otherRelation, _, _, _) => + relation == otherRelation && output == l.output case _ => false } @@ -63,11 +69,13 @@ case class LogicalRelation( override def sameResult(otherPlan: LogicalPlan): Boolean = { otherPlan.canonicalized match { - case LogicalRelation(otherRelation, _, _) => relation == otherRelation + case LogicalRelation(otherRelation, _, _, _) => relation == otherRelation case _ => false } } + override def isStreaming: Boolean = dataFromStreaming + // When comparing two LogicalRelations from within LogicalPlan.sameResult, we only need // LogicalRelation.cleanArgs to return Seq(relation), since expectedOutputAttribute's // expId can be different but the relation is still the same. 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 8566a8061034..994ce093108b 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 @@ -35,6 +35,7 @@ private[sql] object PruneFileSourcePartitions extends Rule[LogicalPlan] { _, _), _, + _, _)) if filters.nonEmpty && fsRelation.partitionSchemaOption.isDefined => // The attribute name of predicate could be different than the one in schema in case of 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 8b598cc60e77..c62bb82c5dbf 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 @@ -382,10 +382,10 @@ case class PreprocessTableInsertion(conf: SQLConf) extends Rule[LogicalPlan] { case relation: CatalogRelation => val metadata = relation.tableMeta preprocess(i, metadata.identifier.quotedString, metadata.partitionColumnNames) - case LogicalRelation(h: HadoopFsRelation, _, catalogTable) => + case LogicalRelation(h: HadoopFsRelation, _, catalogTable, _) => val tblName = catalogTable.map(_.identifier.quotedString).getOrElse("unknown") preprocess(i, tblName, h.partitionSchema.map(_.name)) - case LogicalRelation(_: InsertableRelation, _, catalogTable) => + case LogicalRelation(_: InsertableRelation, _, catalogTable, _) => val tblName = catalogTable.map(_.identifier.quotedString).getOrElse("unknown") preprocess(i, tblName, Nil) case _ => i @@ -415,10 +415,10 @@ object PreWriteCheck extends (LogicalPlan => Unit) { def apply(plan: LogicalPlan): Unit = { plan.foreach { - case InsertIntoTable(l @ LogicalRelation(relation, _, _), partition, query, _, _) => + case InsertIntoTable(l @ LogicalRelation(relation, _, _, _), partition, query, _, _) => // Get all input data source relations of the query. val srcRelations = query.collect { - case LogicalRelation(src, _, _) => src + case LogicalRelation(src, _, _, _) => src } if (srcRelations.contains(relation)) { failAnalysis("Cannot insert into table that is also being read from.") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index a9e64c640042..374789c6023d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -170,8 +170,10 @@ class FileStreamSource( partitionColumns = partitionColumns, className = fileFormatClassName, options = optionsWithPartitionBasePath) - Dataset.ofRows(sparkSession, LogicalRelation(newDataSource.resolveRelation( - checkFilesExist = false))) + markAsStreaming( + Dataset.ofRows(sparkSession, LogicalRelation(newDataSource.resolveRelation( + checkFilesExist = false), dataFromStreaming = true)) + ) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala index 75ffe90f2bb7..fef6a4c007a9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala @@ -17,7 +17,10 @@ package org.apache.spark.sql.execution.streaming -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation +import org.apache.spark.sql.execution.LogicalRDD +import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.types.StructType /** @@ -56,11 +59,32 @@ trait Source { */ def getBatch(start: Option[Offset], end: Offset): DataFrame + /** + * In a streaming query, stream relation will be cut into a series of batch relations. + * We need to mark the batch relation as streaming, i.e. data coming from a stream source, + * so we can apply those streaming strategies to it. + */ + def markAsStreaming(df: DataFrame): DataFrame = { + val markAsStreaming = df.logicalPlan transform { + case logicalRDD @ LogicalRDD(_, _, _, _, false) => + logicalRDD.dataFromStreaming = true + logicalRDD + case logicalRelation @ LogicalRelation(_, _, _, false) => + logicalRelation.dataFromStreaming = true + logicalRelation + case localRelation @ LocalRelation(_, _, false) => + localRelation.dataFromStreaming = true + localRelation + } + + Dataset.ofRows(df.sparkSession, markAsStreaming).toDF() + } + /** * Informs the source that Spark has completed processing all data for offsets less than or * equal to `end` and will only request offsets greater than `end` in the future. */ - def commit(end: Offset) : Unit = {} + def commit(end: Offset): Unit = {} /** Stop this source and free any resources it has allocated. */ def stop(): Unit diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 60d5283e6b21..702a2c9e58de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -567,7 +567,9 @@ class StreamExecution( replacements ++= output.zip(newPlan.output) newPlan }.getOrElse { - LocalRelation(output) + new LocalRelation(output) { + override def isStreaming = true + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index 6d34d51d31c1..dad4e5455201 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -24,7 +24,7 @@ import scala.collection.mutable.{ArrayBuffer, ListBuffer} import scala.util.control.NonFatal import org.apache.spark.internal.Logging -import org.apache.spark.sql._ +import org.apache.spark.sql.{DataFrame, _} import org.apache.spark.sql.catalyst.CatalystConf import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.expressions.Attribute @@ -120,12 +120,14 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) logDebug( s"MemoryBatch [$startOrdinal, $endOrdinal]: ${newBlocks.flatMap(_.collect()).mkString(", ")}") - newBlocks - .map(_.toDF()) - .reduceOption(_ union _) - .getOrElse { - sys.error("No data selected!") - } + markAsStreaming( + newBlocks + .map(_.toDF()) + .reduceOption(_ union _) + .getOrElse { + sys.error("No data selected!") + } + ) } override def commit(end: Offset): Unit = synchronized { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala index 58bff27a05bf..08b4a6e2827f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala @@ -131,12 +131,13 @@ class TextSocketSource(host: String, port: Int, includeTimestamp: Boolean, sqlCo // Underlying MemoryStream has schema (String, Timestamp); strip out the timestamp // if requested. - if (includeTimestamp) { + val df = if (includeTimestamp) { rawBatch.toDF("value", "timestamp") } else { // Strip out timestamp rawBatch.select("_1").toDF("value") } + markAsStreaming(df) } override def commit(end: Offset): Unit = synchronized { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala index 58c310596ca6..223c3d7729a5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala @@ -42,14 +42,14 @@ class OptimizeMetadataOnlyQuerySuite extends QueryTest with SharedSQLContext { private def assertMetadataOnlyQuery(df: DataFrame): Unit = { val localRelations = df.queryExecution.optimizedPlan.collect { - case l @ LocalRelation(_, _) => l + case l @ LocalRelation(_, _, _) => l } assert(localRelations.size == 1) } private def assertNotMetadataOnlyQuery(df: DataFrame): Unit = { val localRelations = df.queryExecution.optimizedPlan.collect { - case l @ LocalRelation(_, _) => l + case l @ LocalRelation(_, _, _) => l } assert(localRelations.size == 0) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala index aecfd3062147..5828f9783da4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala @@ -40,7 +40,7 @@ class SparkPlannerSuite extends SharedSQLContext { case Union(children) => planned += 1 UnionExec(children.map(planLater)) :: planLater(NeverPlanned) :: Nil - case LocalRelation(output, data) => + case LocalRelation(output, data, _) => planned += 1 LocalTableScanExec(output, data) :: planLater(NeverPlanned) :: Nil case NeverPlanned => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala index f36162858bf7..615bd132d2f3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala @@ -552,7 +552,7 @@ class FileSourceStrategySuite extends QueryTest with SharedSQLContext with Predi if (buckets > 0) { val bucketed = df.queryExecution.analyzed transform { - case l @ LogicalRelation(r: HadoopFsRelation, _, _) => + case l @ LogicalRelation(r: HadoopFsRelation, _, _, _) => l.copy(relation = r.copy(bucketSpec = Some(BucketSpec(numBuckets = buckets, "c1" :: Nil, Nil)))(r.sparkSession)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index 9a3328fcecee..1db460478cba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -63,7 +63,8 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex var maybeRelation: Option[HadoopFsRelation] = None val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(relation: HadoopFsRelation, _, _)) => + case PhysicalOperation( + _, filters, LogicalRelation(relation: HadoopFsRelation, _, _, _)) => maybeRelation = Some(relation) filters }.flatten.reduceLeftOption(_ && _) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 2b20b9716bf8..cc38c004a608 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -648,7 +648,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha val queryExecution = spark.read.parquet(dir.getCanonicalPath).queryExecution queryExecution.analyzed.collectFirst { case LogicalRelation( - HadoopFsRelation(location: PartitioningAwareFileIndex, _, _, _, _, _), _, _) => + HadoopFsRelation(location: PartitioningAwareFileIndex, _, _, _, _, _), _, _, _) => assert(location.partitionSpec() === PartitionSpec.emptySpec) }.getOrElse { fail(s"Expecting a matching HadoopFsRelation, but got:\n$queryExecution") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala index 5174a0415304..abb7dc9b8952 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala @@ -22,10 +22,13 @@ import java.net.ServerSocket import java.sql.Timestamp import java.util.concurrent.LinkedBlockingQueue +import scala.collection.mutable + import org.scalatest.BeforeAndAfterEach import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.internal.SQLConf._ import org.apache.spark.sql.streaming.StreamTest import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{StringType, StructField, StructType, TimestampType} @@ -60,29 +63,31 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before source = provider.createSource(sqlContext, "", None, "", parameters) - failAfter(streamingTimeout) { - serverThread.enqueue("hello") - while (source.getOffset.isEmpty) { - Thread.sleep(10) - } - val offset1 = source.getOffset.get - val batch1 = source.getBatch(None, offset1) - assert(batch1.as[String].collect().toSeq === Seq("hello")) + withAdditionalConf(Map(UNSUPPORTED_OPERATION_CHECK_ENABLED.key -> "false")) {() => + failAfter(streamingTimeout) { + serverThread.enqueue("hello") + while (source.getOffset.isEmpty) { + Thread.sleep(10) + } + val offset1 = source.getOffset.get + val batch1 = source.getBatch(None, offset1) + assert(batch1.as[String].collect().toSeq === Seq("hello")) - serverThread.enqueue("world") - while (source.getOffset.get === offset1) { - Thread.sleep(10) - } - val offset2 = source.getOffset.get - val batch2 = source.getBatch(Some(offset1), offset2) - assert(batch2.as[String].collect().toSeq === Seq("world")) + serverThread.enqueue("world") + while (source.getOffset.get === offset1) { + Thread.sleep(10) + } + val offset2 = source.getOffset.get + val batch2 = source.getBatch(Some(offset1), offset2) + assert(batch2.as[String].collect().toSeq === Seq("world")) - val both = source.getBatch(None, offset2) - assert(both.as[String].collect().sorted.toSeq === Seq("hello", "world")) + val both = source.getBatch(None, offset2) + assert(both.as[String].collect().sorted.toSeq === Seq("hello", "world")) - // Try stopping the source to make sure this does not block forever. - source.stop() - source = null + // Try stopping the source to make sure this does not block forever. + source.stop() + source = null + } } } @@ -99,31 +104,33 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before source = provider.createSource(sqlContext, "", None, "", parameters) - failAfter(streamingTimeout) { - serverThread.enqueue("hello") - while (source.getOffset.isEmpty) { - Thread.sleep(10) - } - val offset1 = source.getOffset.get - val batch1 = source.getBatch(None, offset1) - val batch1Seq = batch1.as[(String, Timestamp)].collect().toSeq - assert(batch1Seq.map(_._1) === Seq("hello")) - val batch1Stamp = batch1Seq(0)._2 - - serverThread.enqueue("world") - while (source.getOffset.get === offset1) { - Thread.sleep(10) + withAdditionalConf(Map(UNSUPPORTED_OPERATION_CHECK_ENABLED.key -> "false")) { () => + failAfter(streamingTimeout) { + serverThread.enqueue("hello") + while (source.getOffset.isEmpty) { + Thread.sleep(10) + } + val offset1 = source.getOffset.get + val batch1 = source.getBatch(None, offset1) + val batch1Seq = batch1.as[(String, Timestamp)].collect().toSeq + assert(batch1Seq.map(_._1) === Seq("hello")) + val batch1Stamp = batch1Seq(0)._2 + + serverThread.enqueue("world") + while (source.getOffset.get === offset1) { + Thread.sleep(10) + } + val offset2 = source.getOffset.get + val batch2 = source.getBatch(Some(offset1), offset2) + val batch2Seq = batch2.as[(String, Timestamp)].collect().toSeq + assert(batch2Seq.map(_._1) === Seq("world")) + val batch2Stamp = batch2Seq(0)._2 + assert(!batch2Stamp.before(batch1Stamp)) + + // Try stopping the source to make sure this does not block forever. + source.stop() + source = null } - val offset2 = source.getOffset.get - val batch2 = source.getBatch(Some(offset1), offset2) - val batch2Seq = batch2.as[(String, Timestamp)].collect().toSeq - assert(batch2Seq.map(_._1) === Seq("world")) - val batch2Stamp = batch2Seq(0)._2 - assert(!batch2Stamp.before(batch1Stamp)) - - // Try stopping the source to make sure this does not block forever. - source.stop() - source = null } } @@ -164,19 +171,42 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before val parameters = Map("host" -> "localhost", "port" -> serverThread.port.toString) source = provider.createSource(sqlContext, "", None, "", parameters) - failAfter(streamingTimeout) { - serverThread.enqueue("hello") - while (source.getOffset.isEmpty) { - Thread.sleep(10) + withAdditionalConf(Map(UNSUPPORTED_OPERATION_CHECK_ENABLED.key -> "false")) { () => + failAfter(streamingTimeout) { + serverThread.enqueue("hello") + while (source.getOffset.isEmpty) { + Thread.sleep(10) + } + val batch = source.getBatch(None, source.getOffset.get).as[String] + batch.collect() + val numRowsMetric = + batch.queryExecution.executedPlan.collectLeaves().head.metrics.get("numOutputRows") + assert(numRowsMetric.nonEmpty) + assert(numRowsMetric.get.value === 1) + source.stop() + source = null } - val batch = source.getBatch(None, source.getOffset.get).as[String] - batch.collect() - val numRowsMetric = - batch.queryExecution.executedPlan.collectLeaves().head.metrics.get("numOutputRows") - assert(numRowsMetric.nonEmpty) - assert(numRowsMetric.get.value === 1) - source.stop() - source = null + } + } + + def withAdditionalConf(additionalConf: Map[String, String] = Map.empty)(f: () => Unit): Unit = { + val resetConfValues = mutable.Map[String, Option[String]]() + val conf = sqlContext.sparkSession.conf + additionalConf.foreach(pair => { + val value = if (conf.contains(pair._1)) { + Some(conf.get(pair._1)) + } else { + None + } + resetConfValues(pair._1) = value + conf.set(pair._1, pair._2) + }) + + f() + + resetConfValues.foreach { + case (key, Some(value)) => conf.set(key, value) + case (key, None) => conf.unset(key) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 5463728ca0c1..cc1b7bc81f1d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -229,7 +229,7 @@ class JDBCSuite extends SparkFunSuite // Check whether the tables are fetched in the expected degree of parallelism def checkNumPartitions(df: DataFrame, expectedNumPartitions: Int): Unit = { val jdbcRelations = df.queryExecution.analyzed.collect { - case LogicalRelation(r: JDBCRelation, _, _) => r + case LogicalRelation(r: JDBCRelation, _, _, _) => r } assert(jdbcRelations.length == 1) assert(jdbcRelations.head.parts.length == expectedNumPartitions, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala index be56c964a18f..461e4fb85c55 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala @@ -324,7 +324,7 @@ class FilteredScanSuite extends DataSourceTest with SharedSQLContext with Predic val table = spark.table("oneToTenFiltered") val relation = table.queryExecution.logical.collectFirst { - case LogicalRelation(r, _, _) => r + case LogicalRelation(r, _, _, _) => r }.get assert( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala index 60adee4599b0..77dc085b2ad0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala @@ -134,7 +134,7 @@ class PathOptionSuite extends DataSourceTest with SharedSQLContext { private def getPathOption(tableName: String): Option[String] = { spark.table(tableName).queryExecution.analyzed.collect { - case LogicalRelation(r: TestOptionsRelation, _, _) => r.pathOption + case LogicalRelation(r: TestOptionsRelation, _, _, _) => r.pathOption }.head } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index f67444fbc49d..2761517be344 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -93,7 +93,8 @@ class FileStreamSinkSuite extends StreamTest { // Verify that MetadataLogFileIndex is being used and the correct partitioning schema has // been inferred val hadoopdFsRelations = outputDf.queryExecution.analyzed.collect { - case LogicalRelation(baseRelation, _, _) if baseRelation.isInstanceOf[HadoopFsRelation] => + case LogicalRelation(baseRelation, _, _, _) + if baseRelation.isInstanceOf[HadoopFsRelation] => baseRelation.asInstanceOf[HadoopFsRelation] } assert(hadoopdFsRelations.size === 1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index f705da3d6a70..4d6755eccd32 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -1084,7 +1084,8 @@ class FileStreamSourceSuite extends FileStreamSourceTest { withSQLConf( SQLConf.FILE_SOURCE_LOG_COMPACT_INTERVAL.key -> "2", // Force deleting the old logs - SQLConf.FILE_SOURCE_LOG_CLEANUP_DELAY.key -> "1" + SQLConf.FILE_SOURCE_LOG_CLEANUP_DELAY.key -> "1", + SQLConf.UNSUPPORTED_OPERATION_CHECK_ENABLED.key -> "false" ) { val fileStream = createFileStream("text", src.getCanonicalPath) val filtered = fileStream.filter($"value" contains "keep") @@ -1310,8 +1311,8 @@ class FileStreamSourceSuite extends FileStreamSourceTest { classOf[ExistsThrowsExceptionFileSystem].getName) // add the metadata entries as a pre-req val dir = new File(temp, "dir") // use non-existent directory to test whether log make the dir - val metadataLog = - new FileStreamSourceLog(FileStreamSourceLog.VERSION, spark, dir.getAbsolutePath) + val metadataLog = + new FileStreamSourceLog(FileStreamSourceLog.VERSION, spark, dir.getAbsolutePath) assert(metadataLog.add(0, Array(FileEntry(s"$scheme:///file1", 100L, 0)))) val newSource = new FileStreamSource(spark, s"$scheme:///", "parquet", StructType(Nil), Nil, @@ -1320,6 +1321,20 @@ class FileStreamSourceSuite extends FileStreamSourceTest { newSource.getBatch(None, FileStreamSourceOffset(1)) } } + + test("getBatch should return a streaming DataFrame") { + withTempDir { temp => + spark.conf.set( + s"fs.$scheme.impl", + classOf[ExistsThrowsExceptionFileSystem].getName) + val dir = new File(temp, "dir") + + val newSource = new FileStreamSource(spark, s"$scheme:///", "parquet", StructType(Nil), Nil, + dir.getAbsolutePath, Map.empty) + val df = newSource.getBatch(None, FileStreamSourceOffset(1)) + assert(df.isStreaming, "FileStreamSource should return a streaming DataFrame") + } + } } class FileStreamSourceStressTestSuite extends FileStreamSourceTest { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index e867fc40f7f1..a82da168a7f4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -67,6 +67,22 @@ class StreamSuite extends StreamTest { CheckAnswer(Row(1, 1, "one"), Row(2, 2, "two"), Row(4, 4, "four"))) } + test("join with batch table which has an aggregation") { + // Make a table and ensure it will be broadcast. + val smallTable = Seq((1, "one"), (2, "two"), (4, "four"), (2, "two")) + .toDF("number", "word").groupBy("word").count() + + // Join the input stream with a table. + val inputData = MemoryStream[Int] + val joined = inputData.toDS().toDF().join(smallTable, $"value" === $"count") + + testStream(joined)( + AddData(inputData, 1), + CheckAnswer(Row(1, "one", 1), Row(1, "four", 1)), + AddData(inputData, 2), + CheckAnswer(Row(1, "one", 1), Row(1, "four", 1), Row(2, "two", 2))) + } + test("union two streams") { val inputData1 = MemoryStream[Int] val inputData2 = MemoryStream[Int] 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 2e060ab9f680..cb3c32a79726 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 @@ -84,7 +84,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log tableRelationCache.getIfPresent(tableIdentifier) match { case null => None // Cache miss - case logical @ LogicalRelation(relation: HadoopFsRelation, _, _) => + case logical @ LogicalRelation(relation: HadoopFsRelation, _, _, _) => val cachedRelationFileFormatClass = relation.fileFormat.getClass expectedFileFormat match { 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 f02b7218d6ee..b18e3c82360c 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 @@ -589,7 +589,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv Row(3) :: Row(4) :: Nil) table("test_parquet_ctas").queryExecution.optimizedPlan match { - case LogicalRelation(p: HadoopFsRelation, _, _) => // OK + case LogicalRelation(p: HadoopFsRelation, _, _, _) => // OK case _ => fail(s"test_parquet_ctas should have be converted to ${classOf[HadoopFsRelation]}") } 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 236135dcff52..5dfbdb4357b4 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 @@ -523,7 +523,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { val catalogTable = sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) relation match { - case LogicalRelation(r: HadoopFsRelation, _, _) => + case LogicalRelation(r: HadoopFsRelation, _, _, _) => if (!isDataSourceTable) { fail( s"${classOf[CatalogRelation].getCanonicalName} is expected, but found " + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala index 222c24927a76..de6f0d67f173 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala @@ -45,7 +45,7 @@ class OrcFilterSuite extends QueryTest with OrcTest { var maybeRelation: Option[HadoopFsRelation] = None val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _)) => + case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => maybeRelation = Some(orcRelation) filters }.flatten.reduceLeftOption(_ && _) @@ -89,7 +89,7 @@ class OrcFilterSuite extends QueryTest with OrcTest { var maybeRelation: Option[HadoopFsRelation] = None val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _)) => + case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => maybeRelation = Some(orcRelation) filters }.flatten.reduceLeftOption(_ && _) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index 81af24979d82..9be7530ec848 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -284,7 +284,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { ) table("test_parquet_ctas").queryExecution.optimizedPlan match { - case LogicalRelation(_: HadoopFsRelation, _, _) => // OK + case LogicalRelation(_: HadoopFsRelation, _, _, _) => // OK case _ => fail( "test_parquet_ctas should be converted to " + s"${classOf[HadoopFsRelation ].getCanonicalName }") @@ -369,7 +369,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { assertResult(2) { analyzed.collect { - case r @ LogicalRelation(_: HadoopFsRelation, _, _) => r + case r @ LogicalRelation(_: HadoopFsRelation, _, _, _) => r }.size } } @@ -378,7 +378,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { def collectHadoopFsRelation(df: DataFrame): HadoopFsRelation = { val plan = df.queryExecution.analyzed plan.collectFirst { - case LogicalRelation(r: HadoopFsRelation, _, _) => r + case LogicalRelation(r: HadoopFsRelation, _, _, _) => r }.getOrElse { fail(s"Expecting a HadoopFsRelation 2, but got:\n$plan") } @@ -453,7 +453,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { // Converted test_parquet should be cached. sessionState.catalog.getCachedDataSourceTable(tableIdentifier) match { case null => fail("Converted test_parquet should be cached in the cache.") - case LogicalRelation(_: HadoopFsRelation, _, _) => // OK + case LogicalRelation(_: HadoopFsRelation, _, _, _) => // OK case other => fail( "The cached test_parquet should be a Parquet Relation. " +