Skip to content
Closed
Show file tree
Hide file tree
Changes from 13 commits
Commits
Show all changes
27 commits
Select commit Hold shift + click to select a range
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -44,18 +44,6 @@ object StatefulNetworkWordCount {

StreamingExamples.setStreamingLogLevels()

val updateFunc = (values: Seq[Int], state: Option[Int]) => {
val currentCount = values.sum

val previousCount = state.getOrElse(0)

Some(currentCount + previousCount)
}

val newUpdateFunc = (iterator: Iterator[(String, Seq[Int], Option[Int])]) => {
iterator.flatMap(t => updateFunc(t._2, t._3).map(s => (t._1, s)))
}

val sparkConf = new SparkConf().setAppName("StatefulNetworkWordCount")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since updateStateByKey isn't removed, maybe it's better to keep this example and create a new file for trackStateByKey.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if we want to encourage people to move to the new API, it might be ok to not have the old one.

// Create the context with a 1 second batch size
val ssc = new StreamingContext(sparkConf, Seconds(1))
Expand All @@ -72,8 +60,16 @@ object StatefulNetworkWordCount {

// Update the cumulative count using updateStateByKey
// This will give a Dstream made of state (which is the cumulative count of the words)
val stateDstream = wordDstream.updateStateByKey[Int](newUpdateFunc,
new HashPartitioner (ssc.sparkContext.defaultParallelism), true, initialRDD)

val trackStateFunc = (word: String, one: Option[Int], state: State[Int]) => {
val sum = one.getOrElse(0) + state.getOrElse(0)
val output = (word, sum)
state.update(sum)
Some(output)
}

val stateDstream = wordDstream.trackStateByKey(
TrackStateSpec(trackStateFunc).initialState(initialRDD))
stateDstream.print()
ssc.start()
ssc.awaitTermination()
Expand Down
138 changes: 138 additions & 0 deletions streaming/src/main/scala/org/apache/spark/streaming/State.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,138 @@
/*
* 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.streaming

/**
* Abstract class for getting and updating the tracked state in the `trackStateByKey` operation of
* [[org.apache.spark.streaming.dstream.PairDStreamFunctions pair DStream]] and
* [[org.apache.spark.streaming.api.java.JavaPairDStream]].
* {{{
*
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Todo: Add example in doc.

* }}}
*/
sealed abstract class State[S] {

/** Whether the state already exists */
def exists(): Boolean

/**
* Get the state if it exists, otherwise wise it will throw an exception.
* Check with `exists()` whether the state exists or not before calling `get()`.
*/
def get(): S

/**
* Update the state with a new value. Note that you cannot update the state if the state is
* timing out (that is, `isTimingOut() return true`, or if the state has already been removed by
* `remove()`.
*/
def update(newState: S): Unit

/** Remove the state if it exists. */
def remove(): Unit
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should this be delete or destroy? Not sure if we have used similar terminology elsewhere. Also it would be good to state the semantics of calling this.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

BTW I have no strong feeling here other than that it match existing things, if we have them.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The only semantically similar thing is a HashMap, and there both Scala and Java HashMap uses remove()


/** Is the state going to be timed out by the system after this batch interval */
def isTimingOut(): Boolean

@inline final def getOption(): Option[S] = Option(get())

/** Get the state if it exists, otherwise return the default value */
@inline final def getOrElse[S1 >: S](default: => S1): S1 = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not sure is this "call-by-name" parameter Java friendly? Assuming this State should also be used in Java code :).

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, that is probably a valid concern. If users cannot call it from Java, this could be a Scala-only thing.

if (exists) this.get else default
}

@inline final override def toString() = getOption.map { _.toString }.getOrElse("<state not set>")
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do we need this?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We dont "need" it. But its just convenient for people to automatically convert to an option, so that they can write state.getOrElse(...) in Scala.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We shouldn't have that as part of our API, we don't use implicit conversions in other places in Spark. It will be pretty confusing. Instead they can just do state.getOption.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Cool. Will remove it.


/** Internal implementation of the [[State]] interface */
private[streaming] class StateImpl[S] extends State[S] {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why did we separate this into another class? I don't see any other subclasses of State or great reasons to create them

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I did it so that we can separate the publicly visible interfaces, from the private[streaming] ones. private[streaming] ones are still publicly accessible from Java, so I was trying to avoid that. Other than than I dont really have a strong reason. If that isnt a strong enough reason to have a separate abstract class and concrete implementation, then I can merge them into one.


private var state: S = null.asInstanceOf[S]
private var defined: Boolean = true
private var timingOut: Boolean = false
private var updated: Boolean = false
private var removed: Boolean = false

// ========= Public API =========
def exists(): Boolean = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can you add override modifier to everything that's implementing an interface function

defined
}

def get(): S = {
state
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

State.get says Get the state if it exists, otherwise wise it will throw an exception. but here it won't throw an exception.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good catch. On the todo list to add State unit tests to verify this behavior.

}

def update(newState: S): Unit = {
require(!removed, "Cannot update the state after it has been removed")
require(!timingOut, "Cannot update the state that is timing out")
state = newState
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this required for defensive guard require(!updated, "cannot update the state this is already updated")?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, if the user accidentally tries to update the state that is going to be removed by timeout anyways, the system should throw an error rather than silently allowing him/her to update without actually being updated.

updated = true
}

def isTimingOut(): Boolean = {
timingOut
}

def remove(): Unit = {
require(!timingOut, "Cannot remove the state that is timing out")
removed = true
}

// ========= Internal API =========

/** Whether the state has been marked for removing */
def isRemoved(): Boolean = {
removed
}

/** Whether the state has been been updated */
def isUpdated(): Boolean = {
updated
}

/**
* Internal method to update the state data and reset internal flags in `this`.
* This method allows `this` object to be reused across many state records.
*/
def wrap(optionalState: Option[S]): Unit = {
optionalState match {
case Some(newState) =>
this.state = newState
defined = true

case None =>
this.state = null.asInstanceOf[S]
defined = false
}
timingOut = false
removed = false
updated = false
}

/**
* Internal method to update the state data and reset internal flags in `this`.
* This method allows `this` object to be reused across many state records.
*/
def wrapTiminoutState(newState: S): Unit = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems to contain a typo? It's also not clear how this is different from wrap, from the comment

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good point, will update comment.

this.state = newState
defined = true
timingOut = true
removed = false
updated = false
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,111 @@
/*
* 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.streaming

import scala.reflect.ClassTag

import org.apache.spark.{HashPartitioner, Partitioner}
import org.apache.spark.api.java.JavaPairRDD
import org.apache.spark.rdd.RDD


/**
* Abstract class having all the specifications of DStream.trackStateByKey().
* Use the `TrackStateSpec.create()` or `TrackStateSpec.create()` to create instances of this class.
*
* {{{
* TrackStateSpec(trackingFunction) // in Scala
* TrackStateSpec.create(trackingFunction) // in Java
* }}}
*/
sealed abstract class TrackStateSpec[K: ClassTag, V: ClassTag, S: ClassTag, T: ClassTag]
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would prefer to just call this StateSpec.

extends Serializable {

def initialState(rdd: RDD[(K, S)]): this.type
def initialState(javaPairRDD: JavaPairRDD[K, S]): this.type

def numPartitions(numPartitions: Int): this.type
def partitioner(partitioner: Partitioner): this.type

def timeout(interval: Duration): this.type
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

A doc here to precisely define timeouts would be really helpful.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this scala Duration? Why not just take milliseconds?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Spark Streaming Duration. To keep it consistent with other API. And will add the doc.

}


/** Builder object for creating instances of TrackStateSpec */
object TrackStateSpec {

def apply[K: ClassTag, V: ClassTag, S: ClassTag, T: ClassTag](
trackingFunction: (K, Option[V], State[S]) => Option[T]): TrackStateSpec[K, V, S, T] = {
new TrackStateSpecImpl[K, V, S, T](trackingFunction)
}

def create[K: ClassTag, V: ClassTag, S: ClassTag, T: ClassTag](
trackingFunction: (K, Option[V], State[S]) => Option[T]): TrackStateSpec[K, V, S, T] = {
apply(trackingFunction)
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think here Java friendly constructor is necessary, create might not be directly used in Java code.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, I was planning to add all the Java friendly stuff on in a later PR, and focus on core functionality in this PR.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

An alternative suggestion from community. Just make an implicit conversion to Option. That would make Scala users happy.

}


/** Internal implementation of [[TrackStateSpec]] interface */
private[streaming]
case class TrackStateSpecImpl[K: ClassTag, V: ClassTag, S: ClassTag, T: ClassTag](
function: (K, Option[V], State[S]) => Option[T]) extends TrackStateSpec[K, V, S, T] {

require(function != null)

@volatile private var partitioner: Partitioner = null
@volatile private var initialStateRDD: RDD[(K, S)] = null
@volatile private var timeoutInterval: Duration = null


def initialState(rdd: RDD[(K, S)]): this.type = {
this.initialStateRDD = rdd
this
}

def initialState(javaPairRDD: JavaPairRDD[K, S]): this.type = {
this.initialStateRDD = javaPairRDD.rdd
this
}


def numPartitions(numPartitions: Int): this.type = {
this.partitioner(new HashPartitioner(numPartitions))
this
}

def partitioner(partitioner: Partitioner): this.type = {
this.partitioner = partitioner
this
}

def timeout(interval: Duration): this.type = {
this.timeoutInterval = interval
this
}

// ================= Private Methods =================

private[streaming] def getFunction(): (K, Option[V], State[S]) => Option[T] = function

private[streaming] def getInitialStateRDD(): Option[RDD[(K, S)]] = Option(initialStateRDD)

private[streaming] def getPartitioner(): Option[Partitioner] = Option(partitioner)

private[streaming] def getTimeoutInterval(): Option[Duration] = Option(timeoutInterval)
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,19 +24,18 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.mapred.{JobConf, OutputFormat}
import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat}

import org.apache.spark.{HashPartitioner, Partitioner}
import org.apache.spark.rdd.RDD
import org.apache.spark.streaming.{Duration, Time}
import org.apache.spark.streaming.StreamingContext.rddToFileName
import org.apache.spark.streaming.{Duration, Time, TrackStateSpec, TrackStateSpecImpl}
import org.apache.spark.util.{SerializableConfiguration, SerializableJobConf}
import org.apache.spark.{HashPartitioner, Partitioner}

/**
* Extra functions available on DStream of (key, value) pairs through an implicit conversion.
*/
class PairDStreamFunctions[K, V](self: DStream[(K, V)])
(implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K])
extends Serializable
{
extends Serializable {
private[streaming] def ssc = self.ssc

private[streaming] def sparkContext = self.context.sparkContext
Expand Down Expand Up @@ -350,6 +349,16 @@ class PairDStreamFunctions[K, V](self: DStream[(K, V)])
)
}

def trackStateByKey[S: ClassTag, T: ClassTag](spec: TrackStateSpec[K, V, S, T]): DStream[T] = {
new TrackeStateDStream[K, V, S, T](
self,
spec.asInstanceOf[TrackStateSpecImpl[K, V, S, T]]
).mapPartitions { partitionIter =>
partitionIter.flatMap { _.emittedRecords }
}
}


/**
* Return a new "state" DStream where the state for each key is updated by applying
* the given function on the previous state of the key and the new values of each key.
Expand Down
Loading