Github user tdas commented on a diff in the pull request:
https://github.com/apache/spark/pull/9256#discussion_r44207353
--- Diff:
streaming/src/main/scala/org/apache/spark/streaming/StateSpec.scala ---
@@ -0,0 +1,196 @@
+/*
+ * 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.annotation.Experimental
+import org.apache.spark.api.java.JavaPairRDD
+import org.apache.spark.rdd.RDD
+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
+ * this class.
+ *
+ * Example in Scala:
+ * {{{
+ * val spec = StateSpec(trackingFunction).numPartitions(10)
+ *
+ * val emittedRecordDStream =
keyValueDStream.trackStateByKey[StateType, EmittedDataType](spec)
+ * }}}
+ *
+ * Example in Java:
+ * {{{
+ * StateStateSpec[StateType, EmittedDataType] spec =
+ * StateStateSpec.create[StateType,
EmittedDataType](trackingFunction).numPartition(10);
+ *
+ * JavaDStream[EmittedDataType] emittedRecordDStream =
+ * javaPairDStream.trackStateByKey[StateType, EmittedDataType](spec);
+ * }}}
+ */
+@Experimental
+sealed abstract class StateSpec[K, V, S, T] extends Serializable {
+
+ /** Set the RDD containing the initial states that will be used by
`trackStateByKey`*/
+ def initialState(rdd: RDD[(K, S)]): this.type
+
+ /** Set the RDD containing the initial states that will be used by
`trackStateByKey`*/
+ def initialState(javaPairRDD: JavaPairRDD[K, S]): this.type
+
+ /**
+ * Set the number of partitions by which the state RDDs generated by
`trackStateByKey`
+ * will be partitioned. Hash partitioning will be used on the
+ */
+ def numPartitions(numPartitions: Int): this.type
+
+ /**
+ * Set the partitioner by which the state RDDs generated by
`trackStateByKey` will be
+ * be partitioned.
+ */
+ def partitioner(partitioner: Partitioner): this.type
+
+ /**
+ * Set the duration after which the state of an idle key will be
removed. A key and its state is
+ * considered idle if it has not received any data for at least the
given duration. The state
+ * tracking function will be called one final time on the idle states
that are going to be
+ * removed; [[org.apache.spark.streaming.State State.isTimingOut()]] set
+ * to `true` in that call.
+ */
+ def timeout(idleDuration: Duration): this.type
+}
+
+
+/**
+ * :: 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).
+ *
+ * Example in Scala:
+ * {{{
+ * val spec = StateSpec(trackingFunction).numPartitions(10)
+ *
+ * val emittedRecordDStream =
keyValueDStream.trackStateByKey[StateType, EmittedDataType](spec)
+ * }}}
+ *
+ * Example in Java:
+ * {{{
+ * StateStateSpec[StateType, EmittedDataType] spec =
+ * StateStateSpec.create[StateType,
EmittedDataType](trackingFunction).numPartition(10);
+ *
+ * JavaDStream[EmittedDataType] emittedRecordDStream =
+ * javaPairDStream.trackStateByKey[StateType, EmittedDataType](spec);
+ * }}}
+ */
+@Experimental
+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).
+ * @param trackingFunction The function applied on every data item to
manage the associated state
+ * and generate the emitted data and
+ * @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 apply[KeyType, ValueType, StateType, EmittedType](
--- End diff --
@rxin @pwendell There are three questions that I would like to discuss
regarding this.
1. Value for a key being presented in function as Seq or Option. Seq allows
multiple values for a key to be presented together so that the user can sort
them in the correct order before updating state. But it means we have to do the
grouping. Potentially additional cost. My thought is that there are user-level
ways to get around it (like groupByKey) and we can add this later if we need.
Batch time in the function signature. This is based on some good feedback
on the doc. The function would like to know what is the current batch time in
which it is being processed and accordingly do its own expiry (if the simple
timeout logic is not sufficient). That would make the function signature more
complex (key: Key, value: Option[Value], state: State[S], batchTime: Time) =>
Option[T] Currently there is no good workaround other than the user using
DStream.transform to embed the batch time in every record. Very inefficient.
Even if we dont include time the signature (K, Option[V], State[S]) =>
Option[T] is a daunting signature. It might be worthwhilte to include multiple
methods - a simplified one like (Option[V], State[S]) => T and a full one.
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]