ericm-db commented on code in PR #45991:
URL: https://github.com/apache/spark/pull/45991#discussion_r1571061801


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImplWithTTL.scala:
##########
@@ -0,0 +1,265 @@
+/*
+ * 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.sql.execution.streaming
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.Encoder
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import 
org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchema.{COMPOSITE_KEY_ROW_SCHEMA,
 VALUE_ROW_SCHEMA_WITH_TTL}
+import 
org.apache.spark.sql.execution.streaming.state.{PrefixKeyScanStateEncoderSpec, 
StateStore, StateStoreErrors}
+import org.apache.spark.sql.streaming.{MapState, TTLConfig}
+import org.apache.spark.util.NextIterator
+
+
+class MapStateImplWithTTL[K, V](
+  store: StateStore,
+  stateName: String,
+  keyExprEnc: ExpressionEncoder[Any],
+  userKeyEnc: Encoder[K],
+  valEncoder: Encoder[V],
+  ttlConfig: TTLConfig,
+  batchTimestampMs: Long) extends CompositeKeyTTLStateImpl(stateName, store, 
batchTimestampMs)
+  with MapState[K, V] with Logging {
+
+  private val keySerializer = keyExprEnc.createSerializer()
+  private val stateTypesEncoder = new CompositeKeyStateEncoder(
+    keySerializer, userKeyEnc, valEncoder, COMPOSITE_KEY_ROW_SCHEMA, 
stateName, hasTtl = true)
+
+  private val ttlExpirationMs =
+    StateTTL.calculateExpirationTimeForDuration(ttlConfig.ttlDuration, 
batchTimestampMs)
+
+  initialize()
+
+  private def initialize(): Unit = {
+    store.createColFamilyIfAbsent(stateName, COMPOSITE_KEY_ROW_SCHEMA, 
VALUE_ROW_SCHEMA_WITH_TTL,
+      PrefixKeyScanStateEncoderSpec(COMPOSITE_KEY_ROW_SCHEMA, 1))
+  }
+
+  /** Whether state exists or not. */
+  override def exists(): Boolean = {
+    iterator().nonEmpty
+  }
+
+  /** Get the state value if it exists */
+  override def getValue(key: K): V = {
+    StateStoreErrors.requireNonNullStateValue(key, stateName)
+    val encodedCompositeKey = stateTypesEncoder.encodeCompositeKey(key)
+    val retRow = store.get(encodedCompositeKey, stateName)
+
+    if (retRow != null) {
+      val resState = stateTypesEncoder.decodeValue(retRow)
+
+      if (!stateTypesEncoder.isExpired(retRow, batchTimestampMs)) {
+        resState
+      } else {
+        null.asInstanceOf[V]
+      }
+    } else {
+      null.asInstanceOf[V]
+    }
+  }
+
+  /** Check if the user key is contained in the map */
+  override def containsKey(key: K): Boolean = {
+    StateStoreErrors.requireNonNullStateValue(key, stateName)
+    getValue(key) != null
+  }
+
+  /** Update value for given user key */
+  override def updateValue(key: K, value: V): Unit = {
+    StateStoreErrors.requireNonNullStateValue(key, stateName)
+    StateStoreErrors.requireNonNullStateValue(value, stateName)
+    val encodedValue = stateTypesEncoder.encodeValue(value, ttlExpirationMs)
+    val encodedCompositeKey = stateTypesEncoder.encodeCompositeKey(key)
+    store.put(encodedCompositeKey, encodedValue, stateName)
+    val serializedGroupingKey = stateTypesEncoder.serializeGroupingKey()
+    val serializedUserKey = stateTypesEncoder.serializeUserKey(key)
+    upsertTTLForStateKey(ttlExpirationMs, serializedGroupingKey, 
serializedUserKey)
+  }
+
+  /** Get the map associated with grouping key */
+  override def iterator(): Iterator[(K, V)] = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val unsafeRowPairIterator = store.prefixScan(encodedGroupingKey, stateName)
+    new NextIterator[(K, V)] {
+      override protected def getNext(): (K, V) = {
+        val iter = unsafeRowPairIterator.dropWhile { rowPair =>
+          stateTypesEncoder.isExpired(rowPair.value, batchTimestampMs)
+        }
+        if (iter.hasNext) {
+          val currentRowPair = iter.next()
+          val key = stateTypesEncoder.decodeCompositeKey(currentRowPair.key)
+          val value = stateTypesEncoder.decodeValue(currentRowPair.value)
+          (key, value)
+        } else {
+          finished = true
+          null.asInstanceOf[(K, V)]
+        }
+      }
+
+      override protected def close(): Unit = {}
+    }
+  }
+
+  /** Get the list of keys present in map associated with grouping key */
+  override def keys(): Iterator[K] = {
+    iterator().map(_._1)
+  }
+
+  /** Get the list of values present in map associated with grouping key */
+  override def values(): Iterator[V] = {
+    iterator().map(_._2)
+  }
+
+  /** Remove user key from map state */
+  override def removeKey(key: K): Unit = {
+    StateStoreErrors.requireNonNullStateValue(key, stateName)
+    val compositeKey = stateTypesEncoder.encodeCompositeKey(key)
+    store.remove(compositeKey, stateName)
+  }
+
+  /** Remove this state. */
+  override def clear(): Unit = {
+    store.removeColFamilyIfExists(stateName)
+    initialize()

Review Comment:
   Sorry, I mean dropping and re-creating the columnFamily. I just used 
initialize because it creates the columnFamily but it is probably not the best 
in terms of readability. Will change this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to