-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-11663][Streaming]Add Java API for trackStateByKey #9636
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 3 commits
ac6c491
74a9a83
44cf5c2
de4ef2b
7160786
04845c8
f6c45cb
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,27 @@ | ||
| /* | ||
| * 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.api.java.function; | ||
|
|
||
| import java.io.Serializable; | ||
|
|
||
| /** | ||
| * A four-argument function that takes arguments of type T1, T2, T3 and T4 and returns an R. | ||
| */ | ||
| public interface Function4<T1, T2, T3, T4, R> extends Serializable { | ||
| public R call(T1 v1, T2 v2, T3 v3, T4 v4) throws Exception; | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,15 +17,14 @@ | |
|
|
||
| package org.apache.spark.streaming | ||
|
|
||
| import scala.reflect.ClassTag | ||
|
|
||
| import com.google.common.base.Optional | ||
| import org.apache.spark.annotation.Experimental | ||
| import org.apache.spark.api.java.JavaPairRDD | ||
| import org.apache.spark.api.java.{JavaPairRDD, JavaUtils} | ||
| import org.apache.spark.api.java.function.{Function2 => JFunction2, Function4 => JFunction4} | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.util.ClosureCleaner | ||
| import org.apache.spark.{HashPartitioner, Partitioner} | ||
|
|
||
|
|
||
| /** | ||
| * :: Experimental :: | ||
| * Abstract class representing all the specifications of the DStream transformation | ||
|
|
@@ -49,11 +48,11 @@ import org.apache.spark.{HashPartitioner, Partitioner} | |
| * | ||
| * Example in Java: | ||
| * {{{ | ||
| * StateStateSpec[KeyType, ValueType, StateType, EmittedDataType] spec = | ||
| * StateStateSpec.function[KeyType, ValueType, StateType, EmittedDataType](trackingFunction) | ||
| * StateSpec[KeyType, ValueType, StateType, EmittedDataType] spec = | ||
| * StateSpec.function[KeyType, ValueType, StateType, EmittedDataType](trackingFunction) | ||
| * .numPartition(10); | ||
| * | ||
| * JavaDStream[EmittedDataType] emittedRecordDStream = | ||
| * JavaTrackStateDStream[KeyType, ValueType, StateType, EmittedType] emittedRecordDStream = | ||
| * javaPairDStream.trackStateByKey[StateType, EmittedDataType](spec); | ||
| * }}} | ||
| */ | ||
|
|
@@ -92,29 +91,26 @@ sealed abstract class StateSpec[KeyType, ValueType, StateType, EmittedType] exte | |
| /** | ||
| * :: Experimental :: | ||
| * Builder object for creating instances of [[org.apache.spark.streaming.StateSpec StateSpec]] | ||
| * that is used for specifying the parameters of the DStream transformation | ||
| * `trackStateByKey` operation of a | ||
| * [[org.apache.spark.streaming.dstream.PairDStreamFunctions pair DStream]] (Scala) or a | ||
| * [[org.apache.spark.streaming.api.java.JavaPairDStream JavaPairDStream]] (Java). | ||
| * that is used for specifying the parameters of the DStream transformation `trackStateByKey` | ||
| * operation of a [[org.apache.spark.streaming.dstream.PairDStreamFunctions pair DStream]]. | ||
|
Contributor
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. Please add JavaPairDStream back in the docs. |
||
| * | ||
| * Example in Scala: | ||
| * {{{ | ||
| * def trackingFunction(data: Option[ValueType], wrappedState: State[StateType]): EmittedType = { | ||
| * ... | ||
| * } | ||
| * | ||
| * val spec = StateSpec.function(trackingFunction).numPartitions(10) | ||
| * | ||
| * val emittedRecordDStream = keyValueDStream.trackStateByKey[StateType, EmittedDataType](spec) | ||
| * val emittedRecordDStream = keyValueDStream.trackStateByKey[StateType, EmittedDataType]( | ||
| * StateSpec.function(trackingFunction).numPartitions(10)) | ||
| * }}} | ||
| * | ||
| * Example in Java: | ||
| * {{{ | ||
| * StateStateSpec[KeyType, ValueType, StateType, EmittedDataType] spec = | ||
| * StateStateSpec.function[KeyType, ValueType, StateType, EmittedDataType](trackingFunction) | ||
| * StateSpec[KeyType, ValueType, StateType, EmittedDataType] spec = | ||
| * StateSpec.function[KeyType, ValueType, StateType, EmittedDataType](trackingFunction) | ||
| * .numPartition(10); | ||
| * | ||
| * JavaDStream[EmittedDataType] emittedRecordDStream = | ||
| * JavaTrackStateDStream[KeyType, ValueType, StateType, EmittedType] emittedRecordDStream = | ||
| * javaPairDStream.trackStateByKey[StateType, EmittedDataType](spec); | ||
| * }}} | ||
| */ | ||
|
|
@@ -123,8 +119,8 @@ object StateSpec { | |
| /** | ||
| * Create a [[org.apache.spark.streaming.StateSpec StateSpec]] for setting all the specifications | ||
|
Contributor
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.
Member
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. Done |
||
| * `trackStateByKey` operation on a | ||
| * [[org.apache.spark.streaming.dstream.PairDStreamFunctions pair DStream]] (Scala) or a | ||
| * [[org.apache.spark.streaming.api.java.JavaPairDStream JavaPairDStream]] (Java). | ||
| * [[org.apache.spark.streaming.dstream.PairDStreamFunctions pair DStream]]. | ||
|
Contributor
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.
|
||
| * | ||
| * @param trackingFunction The function applied on every data item to manage the associated state | ||
| * and generate the emitted data | ||
| * @tparam KeyType Class of the keys | ||
|
|
@@ -142,8 +138,8 @@ object StateSpec { | |
| /** | ||
| * Create a [[org.apache.spark.streaming.StateSpec StateSpec]] for setting all the specifications | ||
| * `trackStateByKey` operation on a | ||
| * [[org.apache.spark.streaming.dstream.PairDStreamFunctions pair DStream]] (Scala) or a | ||
| * [[org.apache.spark.streaming.api.java.JavaPairDStream JavaPairDStream]] (Java). | ||
| * [[org.apache.spark.streaming.dstream.PairDStreamFunctions pair DStream]]. | ||
| * | ||
| * @param trackingFunction The function applied on every data item to manage the associated state | ||
| * and generate the emitted data | ||
| * @tparam ValueType Class of the values | ||
|
|
@@ -160,6 +156,48 @@ object StateSpec { | |
| } | ||
| new StateSpecImpl(wrappedFunction) | ||
| } | ||
|
|
||
| /** | ||
| * Create a [[org.apache.spark.streaming.StateSpec StateSpec]] for setting all | ||
| * the specifications `trackStateByKey` operation on a | ||
| * [[org.apache.spark.streaming.api.java.JavaPairDStream JavaPairDStream]]. | ||
| * | ||
| * @param javaTrackingFunction The function applied on every data item to manage the associated | ||
| * state and generate the emitted data | ||
| * @tparam KeyType Class of the keys | ||
| * @tparam ValueType Class of the values | ||
| * @tparam StateType Class of the states data | ||
| * @tparam EmittedType Class of the emitted data | ||
| */ | ||
| def function[KeyType, ValueType, StateType, EmittedType](javaTrackingFunction: | ||
| JFunction4[Time, KeyType, Optional[ValueType], State[StateType], Optional[EmittedType]]): | ||
| StateSpec[KeyType, ValueType, StateType, EmittedType] = { | ||
| val trackingFunc = (time: Time, k: KeyType, v: Option[ValueType], s: State[StateType]) => { | ||
| val t = javaTrackingFunction.call(time, k, JavaUtils.optionToOptional(v), s) | ||
| Option(t.orNull) | ||
| } | ||
| StateSpec.function(trackingFunc) | ||
| } | ||
|
|
||
| /** | ||
| * Create a [[org.apache.spark.streaming.StateSpec StateSpec]] for setting all the specifications | ||
| * `trackStateByKey` operation on a | ||
| * [[org.apache.spark.streaming.api.java.JavaPairDStream JavaPairDStream]]. | ||
| * | ||
| * @param javaTrackingFunction The function applied on every data item to manage the associated | ||
| * state and generate the emitted data | ||
| * @tparam ValueType Class of the values | ||
| * @tparam StateType Class of the states data | ||
| * @tparam EmittedType Class of the emitted data | ||
| */ | ||
| def function[KeyType, ValueType, StateType, EmittedType]( | ||
| javaTrackingFunction: JFunction2[Optional[ValueType], State[StateType], EmittedType]): | ||
| StateSpec[KeyType, ValueType, StateType, EmittedType] = { | ||
| val trackingFunc = (v: Option[ValueType], s: State[StateType]) => { | ||
| javaTrackingFunction.call(Optional.fromNullable(v.get), s) | ||
| } | ||
| StateSpec.function(trackingFunc) | ||
| } | ||
| } | ||
|
|
||
|
|
||
|
|
@@ -184,7 +222,6 @@ case class StateSpecImpl[K, V, S, T]( | |
| this | ||
| } | ||
|
|
||
|
|
||
| override def numPartitions(numPartitions: Int): this.type = { | ||
| this.partitioner(new HashPartitioner(numPartitions)) | ||
| this | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -28,8 +28,10 @@ import com.google.common.base.Optional | |
| import org.apache.hadoop.conf.Configuration | ||
| import org.apache.hadoop.mapred.{JobConf, OutputFormat} | ||
| import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} | ||
|
|
||
| import org.apache.spark.Partitioner | ||
| import org.apache.spark.api.java.{JavaPairRDD, JavaUtils} | ||
| import org.apache.spark.annotation.Experimental | ||
| import org.apache.spark.api.java.{JavaPairRDD, JavaSparkContext, JavaUtils} | ||
| import org.apache.spark.api.java.JavaPairRDD._ | ||
| import org.apache.spark.api.java.JavaSparkContext.fakeClassTag | ||
| import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} | ||
|
|
@@ -426,6 +428,48 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( | |
| ) | ||
| } | ||
|
|
||
| /** | ||
| * :: Experimental :: | ||
| * Return a new [[JavaDStream]] of data generated by combining the key-value data in `this` stream | ||
| * with a continuously updated per-key state. The user-provided state tracking function is | ||
| * applied on each keyed data item along with its corresponding state. The function can choose to | ||
| * update/remove the state and return a transformed data, which forms the | ||
| * [[JavaTrackStateDStream]]. | ||
| * | ||
| * The specifications of this transformation is made through the | ||
| * [[org.apache.spark.streaming.StateSpec StateSpec]] class. Besides the tracking function, there | ||
| * are a number of optional parameters - initial state data, number of partitions, timeouts, etc. | ||
| * See the [[org.apache.spark.streaming.StateSpec StateSpec]] for more details. | ||
| * | ||
| * Example of using `trackStateByKey`: | ||
| * {{{ | ||
| * // A tracking function that maintains an integer state and return a String | ||
| * Function2<Optional<Integer>, State<Integer>, Optional<String>> trackStateFunc = | ||
| * new Function2<Optional<Integer>, State<Integer>, Optional<String>>() { | ||
| * | ||
| * @Override | ||
| * public Optional<String> call(Optional<Integer> one, State<Integer> state) { | ||
| * // Check if state exists, accordingly update/remove state and return transformed data | ||
| * } | ||
| * }; | ||
| * | ||
| * JavaTrackStateDStream[Integer, Integer, Integer, String] trackStateDStream = | ||
|
Contributor
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. Shouldnt these be
Contributor
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. Make sure that the code would compile
Member
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. Just went through all examples and fixed them |
||
| * keyValueDStream.trackStateByKey[Int, String]( | ||
| * StateSpec.function(trackingFunction).numPartitions(10)); | ||
| * }}} | ||
| * | ||
| * @param spec Specification of this transformation | ||
| * @tparam StateType Class type of the state | ||
| * @tparam EmittedType Class type of the tranformed data return by the tracking function | ||
| */ | ||
| @Experimental | ||
| def trackStateByKey[StateType, EmittedType](spec: StateSpec[K, V, StateType, EmittedType]): | ||
| JavaTrackStateDStream[K, V, StateType, EmittedType] = { | ||
| new JavaTrackStateDStream(dstream.trackStateByKey(spec)( | ||
| JavaSparkContext.fakeClassTag, | ||
| JavaSparkContext.fakeClassTag)) | ||
| } | ||
|
|
||
| private def convertUpdateStateFunction[S](in: JFunction2[JList[V], Optional[S], Optional[S]]): | ||
| (Seq[V], Option[S]) => Option[S] = { | ||
| val scalaFunc: (Seq[V], Option[S]) => Option[S] = (values, state) => { | ||
|
|
||
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 you make sure you add a test for this in Java8?
they are in extras/java-8-tests/...
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.
Done