Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
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
@@ -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 {
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 make sure you add a test for this in Java8?
they are in extras/java-8-tests/...

Copy link
Member Author

Choose a reason for hiding this comment

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

Done

public R call(T1 v1, T2 v2, T3 v3, T4 v4) throws Exception;
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,18 +26,13 @@
import com.google.common.base.Optional;
import com.google.common.collect.Lists;

import org.apache.spark.HashPartitioner;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.*;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.StorageLevels;
import org.apache.spark.api.java.function.FlatMapFunction;
import org.apache.spark.api.java.function.Function2;
import org.apache.spark.api.java.function.PairFunction;
import org.apache.spark.streaming.Durations;
import org.apache.spark.streaming.api.java.JavaDStream;
import org.apache.spark.streaming.api.java.JavaPairDStream;
import org.apache.spark.streaming.api.java.JavaReceiverInputDStream;
import org.apache.spark.streaming.api.java.JavaStreamingContext;
import org.apache.spark.streaming.Time;
import org.apache.spark.streaming.api.java.*;

/**
* Counts words cumulatively in UTF8 encoded, '\n' delimited text received from the network every
Expand All @@ -63,25 +58,12 @@ public static void main(String[] args) {

StreamingExamples.setStreamingLogLevels();

// Update the cumulative count function
final Function2<List<Integer>, Optional<Integer>, Optional<Integer>> updateFunction =
new Function2<List<Integer>, Optional<Integer>, Optional<Integer>>() {
@Override
public Optional<Integer> call(List<Integer> values, Optional<Integer> state) {
Integer newSum = state.or(0);
for (Integer value : values) {
newSum += value;
}
return Optional.of(newSum);
}
};

// Create the context with a 1 second batch size
SparkConf sparkConf = new SparkConf().setAppName("JavaStatefulNetworkWordCount");
JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, Durations.seconds(1));
ssc.checkpoint(".");

// Initial RDD input to updateStateByKey
// Initial RDD input to trackStateByKey
@SuppressWarnings("unchecked")
List<Tuple2<String, Integer>> tuples = Arrays.asList(new Tuple2<String, Integer>("hello", 1),
new Tuple2<String, Integer>("world", 1));
Expand All @@ -105,9 +87,22 @@ public Tuple2<String, Integer> call(String s) {
}
});

// Update the cumulative count function
final Function4<Time, String, Optional<Integer>, JavaState<Integer>, Optional<Tuple2<String, Integer>>> trackStateFunc =
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 use the simple function API in this example?

Copy link
Member Author

Choose a reason for hiding this comment

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

Need the key to emit the word and its count

new Function4<Time, String, Optional<Integer>, JavaState<Integer>, Optional<Tuple2<String, Integer>>>() {

@Override
public Optional<Tuple2<String, Integer>> call(Time time, String word, Optional<Integer> one, JavaState<Integer> state) {
int sum = one.or(0) + state.getOption().or(0);
Tuple2<String, Integer> output = new Tuple2<String, Integer>(word, sum);
state.update(sum);
return Optional.of(output);
}
};

// This will give a Dstream made of state (which is the cumulative count of the words)
JavaPairDStream<String, Integer> stateDstream = wordsDstream.updateStateByKey(updateFunction,
new HashPartitioner(ssc.sparkContext().defaultParallelism()), initialRDD);
JavaTrackStateDStream<String, Integer, Integer, Tuple2<String, Integer>> stateDstream =
wordsDstream.trackStateByKey(JavaStateSpec.function(trackStateFunc).initialState(initialRDD));

stateDstream.print();
ssc.start();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@ object StatefulNetworkWordCount {
val ssc = new StreamingContext(sparkConf, Seconds(1))
ssc.checkpoint(".")

// Initial RDD input to updateStateByKey
// Initial RDD input to trackStateByKey
val initialRDD = ssc.sparkContext.parallelize(List(("hello", 1), ("world", 1)))

// Create a ReceiverInputDStream on target ip:port and count the
Expand Down
10 changes: 2 additions & 8 deletions streaming/src/main/scala/org/apache/spark/streaming/State.scala
Original file line number Diff line number Diff line change
Expand Up @@ -24,10 +24,9 @@ import org.apache.spark.annotation.Experimental
/**
* :: Experimental ::
* Abstract class for getting and updating the tracked state in the `trackStateByKey` operation of
* a [[org.apache.spark.streaming.dstream.PairDStreamFunctions pair DStream]] (Scala) or a
* [[org.apache.spark.streaming.api.java.JavaPairDStream JavaPairDStream]] (Java).
* a [[org.apache.spark.streaming.dstream.PairDStreamFunctions pair DStream]].
*
* Scala example of using `State`:
* Example of using `State`:
* {{{
* // A tracking function that maintains an integer state and return a String
* def trackStateFunc(data: Option[Int], state: State[Int]): Option[String] = {
Expand All @@ -49,11 +48,6 @@ import org.apache.spark.annotation.Experimental
* }
*
* }}}
*
* Java example:
* {{{
* TODO(@zsxwing)
* }}}
*/
@Experimental
sealed abstract class State[S] {
Expand Down
62 changes: 12 additions & 50 deletions streaming/src/main/scala/org/apache/spark/streaming/StateSpec.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,26 +17,20 @@

package org.apache.spark.streaming

import scala.reflect.ClassTag

import org.apache.spark.annotation.Experimental
import org.apache.spark.api.java.JavaPairRDD
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
* `trackStateByKey` operation of a
* [[org.apache.spark.streaming.dstream.PairDStreamFunctions pair DStream]] (Scala) or a
* [[org.apache.spark.streaming.api.java.JavaPairDStream JavaPairDStream]] (Java).
* Use the [[org.apache.spark.streaming.StateSpec StateSpec.apply()]] or
* [[org.apache.spark.streaming.StateSpec StateSpec.create()]] to create instances of
* [[org.apache.spark.streaming.dstream.PairDStreamFunctions pair DStream]].
* Use the [[org.apache.spark.streaming.StateSpec StateSpec.function()]] to create instances of
* this class.
*
* Example in Scala:
* Example:
* {{{
* def trackingFunction(data: Option[ValueType], wrappedState: State[StateType]): EmittedType = {
* ...
Expand All @@ -46,26 +40,13 @@ import org.apache.spark.{HashPartitioner, Partitioner}
*
* val emittedRecordDStream = keyValueDStream.trackStateByKey[StateType, EmittedDataType](spec)
* }}}
*
* Example in Java:
* {{{
* StateStateSpec[KeyType, ValueType, StateType, EmittedDataType] spec =
* StateStateSpec.function[KeyType, ValueType, StateType, EmittedDataType](trackingFunction)
* .numPartition(10);
*
* JavaDStream[EmittedDataType] emittedRecordDStream =
* javaPairDStream.trackStateByKey[StateType, EmittedDataType](spec);
* }}}
*/
@Experimental
sealed abstract class StateSpec[KeyType, ValueType, StateType, EmittedType] extends Serializable {

/** Set the RDD containing the initial states that will be used by `trackStateByKey` */
def initialState(rdd: RDD[(KeyType, StateType)]): this.type

/** Set the RDD containing the initial states that will be used by `trackStateByKey` */
def initialState(javaPairRDD: JavaPairRDD[KeyType, StateType]): this.type

/**
* Set the number of partitions by which the state RDDs generated by `trackStateByKey`
* will be partitioned. Hash partitioning will be used.
Expand All @@ -92,39 +73,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]].
Copy link
Contributor

Choose a reason for hiding this comment

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

Please add JavaPairDStream back in the docs.

*
* Example in Scala:
* Example:
* {{{
* def trackingFunction(data: Option[ValueType], wrappedState: State[StateType]): EmittedType = {
* ...
* }
*
* val spec = StateSpec.function(trackingFunction).numPartitions(10)
*
* val emittedRecordDStream = keyValueDStream.trackStateByKey[StateType, EmittedDataType](spec)
* }}}
*
* Example in Java:
* {{{
* StateStateSpec[KeyType, ValueType, StateType, EmittedDataType] spec =
* StateStateSpec.function[KeyType, ValueType, StateType, EmittedDataType](trackingFunction)
* .numPartition(10);
*
* JavaDStream[EmittedDataType] emittedRecordDStream =
* javaPairDStream.trackStateByKey[StateType, EmittedDataType](spec);
* val emittedRecordDStream = keyValueDStream.trackStateByKey[StateType, EmittedDataType](
* StateSpec.function(trackingFunction).numPartitions(10))
* }}}
*/
@Experimental
object StateSpec {
/**
* Create a [[org.apache.spark.streaming.StateSpec StateSpec]] for setting all the specifications
Copy link
Contributor

Choose a reason for hiding this comment

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

specifications of the ..

Copy link
Member Author

Choose a reason for hiding this comment

The 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]].
Copy link
Contributor

Choose a reason for hiding this comment

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

Add java back in. Nvm. My mistake.

*
* @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
Expand All @@ -142,8 +110,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
Expand Down Expand Up @@ -179,12 +147,6 @@ case class StateSpecImpl[K, V, S, T](
this
}

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


override def numPartitions(numPartitions: Int): this.type = {
this.partitioner(new HashPartitioner(numPartitions))
this
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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}
Expand Down Expand Up @@ -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 [[JavaStateSpec]] class. Besides
* the tracking function, there are a number of optional parameters - initial state data, number
* of partitions, timeouts, etc. See the [[JavaStateSpec]] for more details.
*
* Example of using `trackStateByKey`:
* {{{
* // A tracking function that maintains an integer state and return a String
* Function2<Optional<Integer>, JavaState<Integer>, Optional<String>> trackStateFunc =
* new Function2<Optional<Integer>, JavaState<Integer>, Optional<String>>() {
*
* @Override
* public Optional<String> call(Optional<Integer> one, JavaState<Integer> state) {
* // Check if state exists, accordingly update/remove state and return transformed data
* }
* };
*
* JavaTrackStateDStream[Integer, Integer, Integer, String] trackStateDStream =
Copy link
Contributor

Choose a reason for hiding this comment

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

Shouldnt these be JavaTrackStateDStream<Integer, Integer, Integer, String>??

Copy link
Contributor

Choose a reason for hiding this comment

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

Make sure that the code would compile

Copy link
Member Author

Choose a reason for hiding this comment

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

Just went through all examples and fixed them

* keyValueDStream.trackStateByKey[Int, String](
* JavaStateSpec.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: JavaStateSpec[K, V, StateType, EmittedType]):
JavaTrackStateDStream[K, V, StateType, EmittedType] = {
new JavaTrackStateDStream(
dstream.trackStateByKey(spec.stateSpec)(
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) => {
Expand Down
Loading