Github user huitseeker commented on a diff in the pull request:
https://github.com/apache/spark/pull/9256#discussion_r44190238
--- Diff:
streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala ---
@@ -0,0 +1,304 @@
+/*
+ * 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.util
+
+import java.io.{ObjectInputStream, ObjectOutputStream}
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.SparkConf
+import org.apache.spark.streaming.util.OpenHashMapBasedStateMap._
+import org.apache.spark.util.collection.OpenHashMap
+
+/** Internal interface for defining the map that keeps track of sessions.
*/
+private[streaming] abstract class StateMap[K: ClassTag, S: ClassTag]
extends Serializable {
+
+ /** Get the state for a key if it exists */
+ def get(key: K): Option[S]
+
+ /** Get all the keys and states whose updated time is older than the
given threshold time */
+ def getByTime(threshUpdatedTime: Long): Iterator[(K, S, Long)]
+
+ /** Get all the keys and states in this map. */
+ def getAll(): Iterator[(K, S, Long)]
+
+ /** Add or update state */
+ def put(key: K, state: S, updatedTime: Long): Unit
+
+ /** Remove a key */
+ def remove(key: K): Unit
+
+ /**
+ * Shallow copy `this` map to create a new state map.
+ * Updates to the new map should not mutate `this` map.
+ */
+ def copy(): StateMap[K, S]
+
+ def toDebugString(): String = toString()
+}
+
+/** Companion object for [[StateMap]], with utility methods */
+private[streaming] object StateMap {
+ def empty[K: ClassTag, S: ClassTag]: StateMap[K, S] = new
EmptyStateMap[K, S]
+
+ def create[K: ClassTag, S: ClassTag](conf: SparkConf): StateMap[K, S] = {
+ val deltaChainThreshold =
conf.getInt("spark.streaming.sessionByKey.deltaChainThreshold",
+ DELTA_CHAIN_LENGTH_THRESHOLD)
+ new OpenHashMapBasedStateMap[K, S](64, deltaChainThreshold)
+ }
+}
+
+/** Specific implementation of SessionStore interface representing an
empty map */
+private[streaming] class EmptyStateMap[K: ClassTag, S: ClassTag] extends
StateMap[K, S] {
+ override def put(key: K, session: S, updateTime: Long): Unit = {
+ throw new NotImplementedError("put() should not be called on an
EmptyStateMap")
+ }
+ override def get(key: K): Option[S] = None
+ override def getByTime(threshUpdatedTime: Long): Iterator[(K, S, Long)]
= Iterator.empty
+ override def getAll(): Iterator[(K, S, Long)] = Iterator.empty
+ override def copy(): StateMap[K, S] = new EmptyStateMap[K, S]
+ override def remove(key: K): Unit = { }
+ override def toDebugString(): String = ""
+}
+
+
+
+/** Implementation of StateMap based on Spark's OpenHashMap */
+private[streaming] class OpenHashMapBasedStateMap[K: ClassTag, S:
ClassTag](
+ @transient @volatile var parentStateMap: StateMap[K, S],
+ initialCapacity: Int = 64,
+ deltaChainThreshold: Int = DELTA_CHAIN_LENGTH_THRESHOLD
+ ) extends StateMap[K, S] { self =>
+
+ def this(initialCapacity: Int, deltaChainThreshold: Int) = this(
+ new EmptyStateMap[K, S],
+ initialCapacity = initialCapacity,
+ deltaChainThreshold = deltaChainThreshold)
+
+ def this(deltaChainThreshold: Int) = this(
+ initialCapacity = 64, deltaChainThreshold = deltaChainThreshold)
+
+ def this() = this(DELTA_CHAIN_LENGTH_THRESHOLD)
+
+ @transient @volatile private var deltaMap =
+ new OpenHashMap[K, StateInfo[S]](initialCapacity)
+
+ /** Get the session data if it exists */
+ override def get(key: K): Option[S] = {
+ val stateInfo = deltaMap(key)
+ if (stateInfo != null) {
+ if (!stateInfo.deleted) {
+ Some(stateInfo.data)
+ } else {
+ None
+ }
+ } else {
+ parentStateMap.get(key)
+ }
+ }
+
+ /** Get all the keys and states whose updated time is older than the
give threshold time */
+ override def getByTime(threshUpdatedTime: Long): Iterator[(K, S, Long)]
= {
+ val oldStates = parentStateMap.getByTime(threshUpdatedTime).filter {
case (key, value, _) =>
+ !deltaMap.contains(key)
+ }
+
+ val updatedStates = deltaMap.iterator.flatMap { case (key, stateInfo)
=>
+ if (! stateInfo.deleted && stateInfo.updateTime < threshUpdatedTime)
{
+ Some((key, stateInfo.data, stateInfo.updateTime))
+ } else None
+ }
+ oldStates ++ updatedStates
+ }
+
+ /** Get all the keys and states in this map. */
+ override def getAll(): Iterator[(K, S, Long)] = {
+
+ val oldStates = parentStateMap.getAll().filter { case (key, _, _) =>
+ !deltaMap.contains(key)
+ }
+
+ val updatedStates = deltaMap.iterator.filter { ! _._2.deleted }.map {
case (key, stateInfo) =>
+ (key, stateInfo.data, stateInfo.updateTime)
+ }
+ oldStates ++ updatedStates
+ }
+
+ /** Add or update state */
+ override def put(key: K, state: S, updateTime: Long): Unit = {
+ val stateInfo = deltaMap(key)
+ if (stateInfo != null) {
+ stateInfo.update(state, updateTime)
+ } else {
+ deltaMap.update(key, new StateInfo(state, updateTime))
+ }
+ }
+
+ /** Remove a state */
+ override def remove(key: K): Unit = {
+ val stateInfo = deltaMap(key)
+ if (stateInfo != null) {
+ stateInfo.markDeleted()
+ } else {
+ val newInfo = new StateInfo[S](deleted = true)
+ deltaMap.update(key, newInfo)
+ }
+ }
+
+ /**
+ * Shallow copy the map to create a new session store. Updates to the
new map
+ * should not mutate `this` map.
+ */
+ override def copy(): StateMap[K, S] = {
+ new OpenHashMapBasedStateMap[K, S](this, deltaChainThreshold =
deltaChainThreshold)
+ }
+
+ def shouldCompact: Boolean = {
+ deltaChainLength >= deltaChainThreshold
+ }
+
+ def deltaChainLength: Int = parentStateMap match {
+ case map: OpenHashMapBasedStateMap[_, _] => map.deltaChainLength + 1
+ case _ => 0
+ }
+
+ def approxSize: Int = deltaMap.size + {
--- End diff --
Perhaps scaladoc that this is an overestimation ?
---
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]