forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 1
Initial draft of Streaming Dataframe infrastructure #21
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
Merged
Merged
Changes from 1 commit
Commits
Show all changes
24 commits
Select commit
Hold shift + click to select a range
e238911
first draft
marmbrus d2706b5
working on state
marmbrus 7a3590f
working on stateful streaming
marmbrus c8a9238
now with event time windows
marmbrus dddd192
some refactoring after talking to ali
marmbrus a0a1e7b
docs
marmbrus 15bed31
start kinesis
marmbrus 89464a9
some renaming
marmbrus d133dbb
WIP: file source
marmbrus e3c4c83
Merge remote-tracking branch 'origin/master' into streaming-infra
marmbrus 90fa6d3
remove half-baked stateful implementation
marmbrus b1c1dc6
cleanup
marmbrus 9205068
more docs
marmbrus eab186d
rollback changes
marmbrus cd575db
some feedback
marmbrus d233eae
comments
marmbrus 3423dce
Merge remote-tracking branch 'marmbrus/streaming-df' into streaming-i…
marmbrus b0b20e5
Update circle.yml
marmbrus f5d9642
Update SparkBuild.scala
marmbrus f8911e4
Update circle.yml
marmbrus 6387781
Add newlines to satisfy Scalastyle
JoshRosen 6242bc2
revert CI changes
marmbrus 95fd978
update based on TD's comments
marmbrus f928595
style and docs
marmbrus 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
working on stateful streaming
- Loading branch information
commit 7a3590fe5dd659d1a47e645eec247e21280a3373
There are no files selected for viewing
152 changes: 152 additions & 0 deletions
152
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state.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,152 @@ | ||
| /* | ||
| * 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 org.apache.spark._ | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.sql.catalyst._ | ||
| import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection | ||
| import org.apache.spark.sql.types.{DoubleType, LongType} | ||
| import org.apache.spark.sql.{Strategy, SQLContext, Column, DataFrame} | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.catalyst.expressions._ | ||
| import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan, UnaryNode} | ||
| import org.apache.spark.sql.execution.{SparkPlanner, SparkPlan} | ||
|
|
||
| package object state { | ||
|
|
||
| trait StateStore { | ||
| def get(key: InternalRow): InternalRow | ||
| } | ||
|
|
||
| case class Window( | ||
| eventtime: Expression, | ||
| windowAttribute: AttributeReference, | ||
| step: Int, | ||
| closingTriggerDelay: Int, | ||
| child: LogicalPlan) extends UnaryNode { | ||
| override def output: Seq[Attribute] = windowAttribute +: child.output | ||
| override def missingInput: AttributeSet = super.missingInput - windowAttribute | ||
| } | ||
|
|
||
| implicit object MaxWatermark extends AccumulatorParam[Watermark] { | ||
| /** | ||
| * Add additional data to the accumulator value. Is allowed to modify and return `r` | ||
| * for efficiency (to avoid allocating objects). | ||
| * | ||
| * @param r the current value of the accumulator | ||
| * @param t the data to be added to the accumulator | ||
| * @return the new value of the accumulator | ||
| */ | ||
| override def addAccumulator(r: Watermark, t: Watermark): Watermark = if (r > t) r else t | ||
|
|
||
| /** | ||
| * Merge two accumulated values together. Is allowed to modify and return the first value | ||
| * for efficiency (to avoid allocating objects). | ||
| * | ||
| * @param r1 one set of accumulated data | ||
| * @param r2 another set of accumulated data | ||
| * @return both data sets merged together | ||
| */ | ||
| override def addInPlace(r1: Watermark, r2: Watermark): Watermark = if (r1 > r2) r1 else r2 | ||
|
|
||
| /** | ||
| * Return the "zero" (identity) value for an accumulator type, given its initial value. For | ||
| * example, if R was a vector of N dimensions, this would return a vector of N zeroes. | ||
| */ | ||
| override def zero(initialValue: Watermark): Watermark = new Watermark(-1) | ||
| } | ||
|
|
||
| case class WindowAggregate( | ||
| eventtime: Expression, | ||
| eventtimeMax: Accumulator[Watermark], | ||
| eventtimeWatermark: Watermark, | ||
| windowAttribute: AttributeReference, | ||
| step: Int, | ||
| groupingExpressions: Seq[Expression], | ||
| aggregateExpressions: Seq[NamedExpression], | ||
| child: SparkPlan) extends SparkPlan { | ||
| /** | ||
| * Overridden by concrete implementations of SparkPlan. | ||
| * Produces the result of the query as an RDD[InternalRow] | ||
| */ | ||
| override protected def doExecute(): RDD[InternalRow] = { | ||
| child.execute().mapPartitions { iter => | ||
| val window = | ||
| Multiply(Ceil(Divide(Cast(eventtime, DoubleType), Literal(step))), Literal(step)) | ||
| val windowAndGroupProjection = | ||
| GenerateUnsafeProjection.generate(window +: groupingExpressions, child.output) | ||
| iter.foreach { row => | ||
| val windowAndGroup = windowAndGroupProjection(row) | ||
| println(windowAndGroup.toSeq((windowAttribute +: groupingExpressions).map(_.dataType))) | ||
|
|
||
| eventtimeMax += new Watermark(windowAndGroup.getLong(0)) | ||
| } | ||
|
|
||
| Iterator.empty | ||
| } | ||
| } | ||
|
|
||
| override def output: Seq[Attribute] = | ||
| windowAttribute +: aggregateExpressions.map(_.toAttribute) | ||
|
|
||
| override def missingInput: AttributeSet = super.missingInput - windowAttribute | ||
|
|
||
| /** | ||
| * Returns a Seq of the children of this node. | ||
| * Children should not change. Immutability required for containsChild optimization | ||
| */ | ||
| override def children: Seq[SparkPlan] = child :: Nil | ||
| } | ||
|
|
||
| implicit class StatefulDataFrame(df: DataFrame) { | ||
| def window(eventTime: Column, step: Int, closingTriggerDelay: Int): DataFrame = | ||
| df.withPlan( | ||
| Window( | ||
| eventTime.expr, | ||
| new AttributeReference("window", LongType)(), | ||
| step, | ||
| closingTriggerDelay, | ||
| df.logicalPlan)) | ||
| } | ||
|
|
||
| class StatefulPlanner(sqlContext: SQLContext, maxWatermark: Accumulator[Watermark]) | ||
| extends SparkPlanner(sqlContext) { | ||
|
|
||
| override def strategies: Seq[Strategy] = WindowStrategy +: super.strategies | ||
|
|
||
| object WindowStrategy extends Strategy with Logging { | ||
| def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { | ||
| case Aggregate(grouping, aggregate, | ||
| Window(et, windowAttribute, step, trigger, child)) => | ||
| println(s"triggering ${maxWatermark.value - trigger}") | ||
|
|
||
| WindowAggregate( | ||
| et, | ||
| maxWatermark, | ||
| maxWatermark.value - trigger, | ||
| windowAttribute, | ||
| step, | ||
| grouping, | ||
| aggregate, | ||
| planLater(child)) :: Nil | ||
|
|
||
| case other => Nil | ||
| } | ||
| } | ||
| } | ||
| } |
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
49 changes: 49 additions & 0 deletions
49
sql/core/src/test/scala/org/apache/spark/sql/streaming/StatefulStreamSuite.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,49 @@ | ||
| /* | ||
| * 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.streaming | ||
|
|
||
| import org.apache.spark.sql.{Row, QueryTest} | ||
| import org.apache.spark.sql.functions._ | ||
|
|
||
| import org.apache.spark.sql.execution.streaming._ | ||
| import org.apache.spark.sql.execution.streaming.state._ | ||
| import org.apache.spark.sql.test.SharedSQLContext | ||
|
|
||
| class StatefulStreamSuite extends QueryTest with SharedSQLContext { | ||
|
|
||
| import testImplicits._ | ||
|
|
||
| test("windowed aggregation") { | ||
| val inputData = MemoryStream[Int] | ||
| val tenSecondCounts = | ||
| inputData.toDF("eventTime") | ||
| .window($"eventTime", step = 10, closingTriggerDelay = 20) | ||
| .groupBy($"eventTime" % 2) | ||
| .agg(count("*")) | ||
|
|
||
| testStream(tenSecondCounts)( | ||
| AddData(inputData, 1, 2, 3), | ||
| CheckAnswer(), | ||
| AddData(inputData, 11, 12), | ||
| CheckAnswer(), | ||
| AddData(inputData, 20), | ||
| CheckAnswer(), | ||
| AddData(inputData, 30), | ||
| CheckAnswer((0, 0, 1), (0, 1, 2))) | ||
| } | ||
| } |
83 changes: 83 additions & 0 deletions
83
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.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,83 @@ | ||
| /* | ||
| * 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.streaming | ||
|
|
||
| import org.apache.spark.sql.{Row, QueryTest} | ||
| import org.apache.spark.sql.execution.streaming._ | ||
| import org.apache.spark.sql.test.SharedSQLContext | ||
|
|
||
| class StreamSuite extends QueryTest with SharedSQLContext { | ||
|
|
||
| import testImplicits._ | ||
|
|
||
| test("map with recovery") { | ||
| val inputData = MemoryStream[Int] | ||
| val mapped = inputData.toDS().map(_ + 1) | ||
|
|
||
| testStream(mapped)( | ||
| AddData(inputData, 1, 2, 3), | ||
| CheckAnswer(2, 3, 4), | ||
| StopStream, | ||
| AddData(inputData, 4, 5, 6), | ||
| StartStream, | ||
| CheckAnswer(2, 3, 4, 5, 6, 7)) | ||
| } | ||
|
|
||
| test("join") { | ||
| // Make a table and ensure it will be broadcast. | ||
| val smallTable = Seq((1, "one"), (2, "two"), (4, "four")).toDF("number", "word") | ||
|
|
||
| // Join the input stream with a table. | ||
| val inputData = MemoryStream[Int] | ||
| val joined = inputData.toDS().toDF().join(smallTable, $"value" === $"number") | ||
|
|
||
| testStream(joined)( | ||
| AddData(inputData, 1, 2, 3), | ||
| CheckAnswer(Row(1, 1, "one"), Row(2, 2, "two")), | ||
| AddData(inputData, 4), | ||
| CheckAnswer(Row(1, 1, "one"), Row(2, 2, "two"), Row(4, 4, "four"))) | ||
| } | ||
|
|
||
| test("union two streams") { | ||
| val inputData1 = MemoryStream[Int] | ||
| val inputData2 = MemoryStream[Int] | ||
|
|
||
| val unioned = inputData1.toDS().union(inputData2.toDS()) | ||
|
|
||
| testStream(unioned)( | ||
| AddData(inputData1, 1, 3, 5), | ||
| CheckAnswer(1, 3, 5), | ||
| AddData(inputData2, 2, 4, 6), | ||
| CheckAnswer(1, 2, 3, 4, 5, 6), | ||
| StopStream, | ||
| AddData(inputData1, 7), | ||
| StartStream, | ||
| AddData(inputData2, 8), | ||
| CheckAnswer(1, 2, 3, 4, 5, 6, 7, 8)) | ||
| } | ||
|
|
||
| test("sql queries") { | ||
| val inputData = MemoryStream[Int] | ||
| inputData.toDF().registerTempTable("stream") | ||
| val evens = sql("SELECT * FROM stream WHERE value % 2 = 0") | ||
|
|
||
| testStream(evens)( | ||
| AddData(inputData, 1, 2, 3, 4), | ||
| CheckAnswer(2, 4)) | ||
| } | ||
| } | ||
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.
Can the user reuse a Source? E.g.,
inputData1.toDS().union(inputData1.toDS())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.
Yes, that works already.