-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-24630][SS] Support SQLStreaming in Spark #22575
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
Closed
Changes from all commits
Commits
Show all changes
9 commits
Select commit
Hold shift + click to select a range
af26ea7
Support SQLStreaming in Spark: Add keyword 'STREAM'; Support create s…
36c68a1
Support SQLStreaming in Spark: Add analysis rule for SQLStreaming
fba7bb4
[SPARK-24630][SS] Support SQLStreaming in Spark: add watermark to SQL…
a741185
[SPARK-24630][SS] Support SQLStreaming in Spark: remove stream keyword
0392de7
[SPARK-24630][SS] Support SQLStreaming in Spark: remove unused changes
6150463
[SPARK-24630][SS] Support SQLStreaming in Spark: change microseconds …
41f595d
Merge branch 'master' into sqlstreaming
e735938
[SPARK-24630][SS] Support SQLStreaming in Spark: change microseconds …
45acfb5
Merge branch 'master' into sqlstreaming
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or 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 hidden or 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 hidden or 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 hidden or 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
115 changes: 115 additions & 0 deletions
115
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/SQLStreamingSink.scala
This file contains hidden or 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,115 @@ | ||
| /* | ||
| * 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.streaming | ||
|
|
||
| import java.util.concurrent.TimeUnit | ||
|
|
||
| import org.apache.spark.sql._ | ||
| import org.apache.spark.sql.catalyst.catalog.CatalogTable | ||
| import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
| import org.apache.spark.sql.catalyst.streaming.InternalOutputModes | ||
| import org.apache.spark.sql.execution.command.RunnableCommand | ||
| import org.apache.spark.sql.execution.datasources.DataSource | ||
| import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils | ||
| import org.apache.spark.sql.sources.v2.StreamingWriteSupportProvider | ||
| import org.apache.spark.sql.streaming.Trigger | ||
| import org.apache.spark.util.Utils | ||
|
|
||
| /** | ||
| * The basic RunnableCommand for SQLStreaming, using Command.run to start a streaming query. | ||
| * | ||
| * @param sparkSession | ||
| * @param extraOptions | ||
| * @param partitionColumnNames | ||
| * @param child | ||
| */ | ||
| case class SQLStreamingSink(sparkSession: SparkSession, | ||
| table: CatalogTable, | ||
| child: LogicalPlan) | ||
| extends RunnableCommand { | ||
|
|
||
| private val sqlConf = sparkSession.sqlContext.conf | ||
|
|
||
| /** | ||
| * The given column name may not be equal to any of the existing column names if we were in | ||
| * case-insensitive context. Normalize the given column name to the real one so that we don't | ||
| * need to care about case sensitivity afterwards. | ||
| */ | ||
| private def normalize(df: DataFrame, columnName: String, columnType: String): String = { | ||
| val validColumnNames = df.logicalPlan.output.map(_.name) | ||
| validColumnNames.find(sparkSession.sessionState.analyzer.resolver(_, columnName)) | ||
| .getOrElse(throw new AnalysisException(s"$columnType column $columnName not found in " + | ||
| s"existing columns (${validColumnNames.mkString(", ")})")) | ||
| } | ||
|
|
||
| /** | ||
| * Parse spark.sqlstreaming.trigger.seconds to Trigger | ||
| */ | ||
| private def parseTrigger(): Trigger = { | ||
| val trigger = Utils.timeStringAsMs(sqlConf.sqlStreamTrigger) | ||
| Trigger.ProcessingTime(trigger, TimeUnit.MILLISECONDS) | ||
| } | ||
|
|
||
| /** | ||
| * Running by queryExecution.executeCollect() | ||
| * @param sparkSession | ||
| * @return return empty rdds, save as DDLCommands | ||
| */ | ||
| override def run(sparkSession: SparkSession): Seq[Row] = { | ||
|
|
||
| /////////////////////////////////////////////////////////////////////////////////////// | ||
| // Builder pattern config options | ||
| /////////////////////////////////////////////////////////////////////////////////////// | ||
| val df = Dataset.ofRows(sparkSession, child) | ||
| val outputMode = InternalOutputModes(sqlConf.sqlStreamOutputMode) | ||
| val normalizedParCols = table.partitionColumnNames.map { | ||
| normalize(df, _, "Partition") | ||
| } | ||
|
|
||
| val ds = DataSource.lookupDataSource(table.provider.get, sparkSession.sessionState.conf) | ||
| val disabledSources = sparkSession.sqlContext.conf.disabledV2StreamingWriters.split(",") | ||
| var options = table.storage.properties | ||
| val sink = ds.newInstance() match { | ||
| case w: StreamingWriteSupportProvider | ||
| if !disabledSources.contains(w.getClass.getCanonicalName) => | ||
| val sessionOptions = DataSourceV2Utils.extractSessionConfigs( | ||
| w, df.sparkSession.sessionState.conf) | ||
| options = sessionOptions ++ options | ||
| w | ||
| case _ => | ||
| val ds = DataSource( | ||
| df.sparkSession, | ||
| className = table.provider.get, | ||
| options = options, | ||
| partitionColumns = normalizedParCols) | ||
| ds.createSink(outputMode) | ||
| } | ||
|
|
||
| sparkSession.sessionState.streamingQueryManager.startQuery( | ||
| sqlConf.sqlStreamQueryName, | ||
| None, | ||
| df, | ||
| table.storage.properties, | ||
| sink, | ||
| outputMode, | ||
| trigger = parseTrigger() | ||
| ).awaitTermination() | ||
|
|
||
| Seq.empty[Row] | ||
| } | ||
| } | ||
This file contains hidden or 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
42 changes: 42 additions & 0 deletions
42
sql/hive/src/test/scala/org/apache/spark/sql/hive/StreamTableDDLCommandSuite.scala
This file contains hidden or 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,42 @@ | ||
| /* | ||
| * 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.hive | ||
|
|
||
| import org.apache.spark.sql.catalyst.TableIdentifier | ||
| import org.apache.spark.sql.hive.test.TestHiveSingleton | ||
| import org.apache.spark.sql.test.SQLTestUtils | ||
|
|
||
| class StreamTableDDLCommandSuite extends SQLTestUtils with TestHiveSingleton { | ||
| private val catalog = spark.sessionState.catalog | ||
|
|
||
| test("CTAS: create data source stream table") { | ||
| withTempPath { dir => | ||
| withTable("t") { | ||
| sql( | ||
| s"""CREATE TABLE t USING PARQUET | ||
| |OPTIONS ( | ||
| |PATH = '${dir.toURI}', | ||
| |location = '${dir.toURI}', | ||
| |isStreaming = 'true') | ||
| |AS SELECT 1 AS a, 2 AS b, 3 AS c | ||
| """.stripMargin) | ||
jackylee-ch marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| val streamTable = catalog.getTableMetadata(TableIdentifier("t")) | ||
| assert(streamTable.isStreaming) | ||
| } | ||
| } | ||
| } | ||
| } | ||
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Continuous processing mode is supported now, do you plan to support it? If so I think we can traverse the logical plan to find out whether this is a continuous query and create a ContinuousTrigger
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This feature is under discussion: https://docs.google.com/document/d/19degwnIIcuMSELv6BQ_1VQI5AIVcvGeqOm5xE2-aRA0