-
Notifications
You must be signed in to change notification settings - Fork 28.3k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[SPARK-27190][SQL] add table capability for streaming
## What changes were proposed in this pull request? This is a followup of #24012 , to add the corresponding capabilities for streaming. ## How was this patch tested? existing tests Closes #24129 from cloud-fan/capability. Authored-by: Wenchen Fan <wenchen@databricks.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
- Loading branch information
Showing
24 changed files
with
389 additions
and
231 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
35 changes: 0 additions & 35 deletions
35
sql/core/src/main/java/org/apache/spark/sql/sources/v2/SupportsContinuousRead.java
This file was deleted.
Oops, something went wrong.
35 changes: 0 additions & 35 deletions
35
sql/core/src/main/java/org/apache/spark/sql/sources/v2/SupportsMicroBatchRead.java
This file was deleted.
Oops, something went wrong.
34 changes: 0 additions & 34 deletions
34
sql/core/src/main/java/org/apache/spark/sql/sources/v2/SupportsStreamingWrite.java
This file was deleted.
Oops, something went wrong.
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
64 changes: 64 additions & 0 deletions
64
...ain/scala/org/apache/spark/sql/execution/datasources/v2/V2StreamingScanSupportCheck.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,64 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.spark.sql.execution.datasources.v2 | ||
|
||
import org.apache.spark.sql.AnalysisException | ||
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
import org.apache.spark.sql.execution.streaming.{StreamingRelation, StreamingRelationV2} | ||
import org.apache.spark.sql.sources.v2.TableCapability.{CONTINUOUS_READ, MICRO_BATCH_READ} | ||
|
||
/** | ||
* This rules adds some basic table capability check for streaming scan, without knowing the actual | ||
* streaming execution mode. | ||
*/ | ||
object V2StreamingScanSupportCheck extends (LogicalPlan => Unit) { | ||
import DataSourceV2Implicits._ | ||
|
||
override def apply(plan: LogicalPlan): Unit = { | ||
plan.foreach { | ||
case r: StreamingRelationV2 if !r.table.supportsAny(MICRO_BATCH_READ, CONTINUOUS_READ) => | ||
throw new AnalysisException( | ||
s"Table ${r.table.name()} does not support either micro-batch or continuous scan.") | ||
case _ => | ||
} | ||
|
||
val streamingSources = plan.collect { | ||
case r: StreamingRelationV2 => r.table | ||
} | ||
val v1StreamingRelations = plan.collect { | ||
case r: StreamingRelation => r | ||
} | ||
|
||
if (streamingSources.length + v1StreamingRelations.length > 1) { | ||
val allSupportsMicroBatch = streamingSources.forall(_.supports(MICRO_BATCH_READ)) | ||
// v1 streaming data source only supports micro-batch. | ||
val allSupportsContinuous = streamingSources.forall(_.supports(CONTINUOUS_READ)) && | ||
v1StreamingRelations.isEmpty | ||
if (!allSupportsMicroBatch && !allSupportsContinuous) { | ||
val microBatchSources = | ||
streamingSources.filter(_.supports(MICRO_BATCH_READ)).map(_.name()) ++ | ||
v1StreamingRelations.map(_.sourceName) | ||
val continuousSources = streamingSources.filter(_.supports(CONTINUOUS_READ)).map(_.name()) | ||
throw new AnalysisException( | ||
"The streaming sources in a query do not have a common supported execution mode.\n" + | ||
"Sources support micro-batch: " + microBatchSources.mkString(", ") + "\n" + | ||
"Sources support continuous: " + continuousSources.mkString(", ")) | ||
} | ||
} | ||
} | ||
} |
Oops, something went wrong.