-
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
Changes from 1 commit
e238911
d2706b5
7a3590f
c8a9238
dddd192
a0a1e7b
15bed31
89464a9
d133dbb
e3c4c83
90fa6d3
b1c1dc6
9205068
eab186d
cd575db
d233eae
3423dce
b0b20e5
f5d9642
f8911e4
6387781
6242bc2
95fd978
f928595
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
- Loading branch information
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,35 @@ | ||
| /* | ||
| * 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 | ||
|
|
||
| case class LongOffset(offset: Long) extends Offset { | ||
|
Collaborator
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Good to have some scala docs here. |
||
| override def >(other: Offset): Boolean = other match { | ||
| case l: LongOffset => offset > l.offset | ||
| case _ => | ||
| throw new IllegalArgumentException(s"Invalid comparison of $getClass with ${other.getClass}") | ||
| } | ||
|
|
||
| override def <(other: Offset): Boolean = other match { | ||
| case l: LongOffset => offset < l.offset | ||
| case _ => | ||
| throw new IllegalArgumentException(s"Invalid comparison of $getClass with ${other.getClass}") | ||
| } | ||
|
|
||
| def +(increment: Long): LongOffset = new LongOffset(offset + increment) | ||
| def -(decrement: Long): LongOffset = new LongOffset(offset - decrement) | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,8 +17,6 @@ | |
|
|
||
| package org.apache.spark.sql.execution.streaming | ||
|
|
||
|
|
||
|
|
||
| /** | ||
| * A offset is a monotonically increasing metric used to track progress in the computation of a | ||
| * stream. An [[Offset]] must be comparable. | ||
|
Collaborator
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. By comparable do you mean only
Owner
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Now implements a more general
Collaborator
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is better I think. Any benefit in making it extend Java Comparable class?
Owner
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I actually did that at first, I wasn't sure if we wanted to be |
||
|
|
@@ -28,20 +26,3 @@ trait Offset extends Serializable { | |
|
|
||
| def <(other: Offset): Boolean | ||
| } | ||
|
|
||
| case class LongOffset(offset: Long) extends Offset { | ||
| override def >(other: Offset): Boolean = other match { | ||
| case l: LongOffset => offset > l.offset | ||
| case _ => | ||
| throw new IllegalArgumentException(s"Invalid comparison of $getClass with ${other.getClass}") | ||
| } | ||
|
|
||
| override def <(other: Offset): Boolean = other match { | ||
| case l: LongOffset => offset < l.offset | ||
| case _ => | ||
| throw new IllegalArgumentException(s"Invalid comparison of $getClass with ${other.getClass}") | ||
| } | ||
|
|
||
| def +(increment: Long): LongOffset = new LongOffset(offset + increment) | ||
| def -(decrement: Long): LongOffset = new LongOffset(offset - decrement) | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -22,18 +22,34 @@ import org.apache.spark.sql.SQLContext | |
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.types.StructType | ||
|
|
||
|
|
||
| /** | ||
| * A source of continually arriving data for a streaming query. A [[Source]] must have a | ||
| * monotonically increasing notion of progress that can be represented as an [[Offset]]. Spark | ||
| * will regularly query each [[Source]] for is current [[Offset]] in order to determine when new | ||
| * data has arrived. | ||
| */ | ||
| trait Source { | ||
|
|
||
| /** Returns the schema of the data from this source */ | ||
| def schema: StructType | ||
|
|
||
| /** Returns the maximum offset that can be retrieved from the source. */ | ||
| /** | ||
| * Returns the maximum offset that can be retrieved from the source. This function will be called | ||
| * only before attempting to start a new batch, in order to determine if new data is available. | ||
| * Therfore is acceptable to perform potentially expensive work when this function is called that | ||
|
Collaborator
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Therefore, it |
||
| * is required in-order to correctly replay sections of data when `getSlice` is called. For | ||
| * example, a [[Source]] might append to a write-ahead log in order to record what data is present | ||
| * at a given [[Offset]]. | ||
| */ | ||
| def getCurrentOffset: Offset | ||
|
|
||
| /** | ||
| * Returns the data between the `start` and `end` offsets. This function must always return | ||
| * the same set of data for any given pair of offsets. | ||
| * the same set of data for any given pair of offsets in order to guarantee exactly-once semantics | ||
| * in the presence of failures. | ||
| * | ||
| * When `start` is [[None]], the stream should be replayed from the beginning. `getSlice` will | ||
| * never be called with an `end` that is greater than the result of `getCurrentOffset`. | ||
| */ | ||
| def getSlice(sqlContext: SQLContext, start: Option[Offset], end: Offset): RDD[InternalRow] | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,6 +19,11 @@ package org.apache.spark.sql.execution.streaming | |
|
|
||
| import scala.collection.mutable | ||
|
|
||
| /** | ||
| * Tracks the progress of processing data from one or more [[Source]]s that are present in a | ||
| * streaming query. This is similar to simplified, single-instance vector clock that progresses | ||
| * monotonically forward. | ||
| */ | ||
| class StreamProgress extends Serializable { | ||
|
Collaborator
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. How will external sinks create this object if this neither takes a map of offsets in constructor, nor can the update() be called.
Owner
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'll add a TODO for serialization API. |
||
| private val currentOffsets = new mutable.HashMap[Source, Offset] | ||
| with mutable.SynchronizedMap[Source, Offset] | ||
|
|
@@ -42,6 +47,10 @@ class StreamProgress extends Serializable { | |
| updated | ||
| } | ||
|
|
||
| /** | ||
| * Used to create a new copy of this [[StreamProgress]]. While this class is currently mutable, | ||
| * it should be copied before being passed to user code. | ||
| */ | ||
| private[streaming] def copy(): StreamProgress = { | ||
| val copied = new StreamProgress | ||
| currentOffsets.foreach(copied.update) | ||
|
|
@@ -60,4 +69,4 @@ class StreamProgress extends Serializable { | |
| override def hashCode: Int = { | ||
| currentOffsets.toSeq.sortBy(_._1.toString).hashCode() | ||
| } | ||
| } | ||
| } | ||
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.
Why are the basic abstractions of streaming stuff like
Offsetinsql.execution package? From user point of view, its not intuitive to have "execution" in the middle, and painful to import that deep a package name.I think basic abstractions like
Offset,Source,Sinkshould be insql.streaming, and things likeStreamExecutioncan be insql.execution.streamingorsql.streaming.execution.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.
My thought was to by default put everything in
executionwhich is hidden from scala doc / etc. We can move stuff out as we decide to make it public API.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.
I am cool with that for now.