ericm-db commented on code in PR #45932: URL: https://github.com/apache/spark/pull/45932#discussion_r1566193244
########## sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala: ########## @@ -0,0 +1,221 @@ +/* + * 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.sql.Encoder +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchema.{KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA_WITH_TTL} +import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateStore, StateStoreErrors} +import org.apache.spark.sql.streaming.{ListState, TTLConfig} +import org.apache.spark.util.NextIterator + +/** + * Class that provides a concrete implementation for a list state state associated with state + * variables (with ttl expiration support) used in the streaming transformWithState operator. + * + * @param store - reference to the StateStore instance to be used for storing state + * @param stateName - name of logical state partition + * @param keyExprEnc - Spark SQL encoder for key + * @param valEncoder - Spark SQL encoder for value + * @param ttlConfig - TTL configuration for values stored in this state + * @param batchTimestampMs - current batch processing timestamp. + * @tparam S - data type of object that will be stored + */ +class ListStateImplWithTTL[S]( + store: StateStore, + stateName: String, + keyExprEnc: ExpressionEncoder[Any], + valEncoder: Encoder[S], + ttlConfig: TTLConfig, + batchTimestampMs: Long) + extends SingleKeyTTLStateImpl(stateName, store, batchTimestampMs) with ListState[S] { + + private lazy val keySerializer = keyExprEnc.createSerializer() + + private lazy val stateTypesEncoder = StateTypesEncoder( + keySerializer, valEncoder, stateName, hasTtl = true) + + private lazy val ttlExpirationMs = + StateTTL.calculateExpirationTimeForDuration(ttlConfig.ttlDuration, batchTimestampMs) + + initialize() + + private def initialize(): Unit = { + store.createColFamilyIfAbsent(stateName, KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA_WITH_TTL, + NoPrefixKeyStateEncoderSpec(KEY_ROW_SCHEMA), useMultipleValuesPerKey = true) + } + + /** Whether state exists or not. */ + override def exists(): Boolean = { + get().nonEmpty + } + + /** + * Get the state value if it exists. If the state does not exist in state store, an + * empty iterator is returned. + */ + override def get(): Iterator[S] = { + val encodedKey = stateTypesEncoder.encodeGroupingKey() + val unsafeRowValuesIterator = store.valuesIterator(encodedKey, stateName) + + new NextIterator[S] { + + override protected def getNext(): S = { + val iter = unsafeRowValuesIterator.dropWhile { row => + stateTypesEncoder.isExpired(row, batchTimestampMs) + } + + if (iter.hasNext) { + val currentRow = iter.next() + stateTypesEncoder.decodeValue(currentRow) + } else { + finished = true + null.asInstanceOf[S] + } + } + + override protected def close(): Unit = {} + } + } + + /** Update the value of the list. */ + override def put(newState: Array[S]): Unit = { + validateNewState(newState) + + val serializedGroupingKey = stateTypesEncoder.serializeGroupingKey() + val encodedKey = stateTypesEncoder.encodeSerializedGroupingKey(serializedGroupingKey) + var isFirst = true + + newState.foreach { v => + val encodedValue = stateTypesEncoder.encodeValue(v, ttlExpirationMs) + if (isFirst) { + store.put(encodedKey, encodedValue, stateName) + isFirst = false + } else { + store.merge(encodedKey, encodedValue, stateName) + } + } + upsertTTLForStateKey(serializedGroupingKey) Review Comment: Yes, essentially. ########## sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ListStateSuite.scala: ########## @@ -160,4 +161,60 @@ class ListStateSuite extends StateVariableSuiteBase { assert(listState1.get().toSeq === Seq.empty[Long]) } } + + test(s"test List state TTL") { + tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider => + val store = provider.getStore(0) + val timestampMs = 10 + val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(), + Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]], + TimeMode.ProcessingTime(), batchTimestampMs = Some(timestampMs)) + + val ttlConfig = TTLConfig(ttlDuration = Duration.ofMinutes(1)) + val testState: ListStateImplWithTTL[String] = handle.getListState[String]("testState", + Encoders.STRING, ttlConfig).asInstanceOf[ListStateImplWithTTL[String]] + ImplicitGroupingKeyTracker.setImplicitKey("test_key") + testState.put(Array("v1", "v2", "v3")) + assert(testState.get().toSeq === Seq("v1", "v2", "v3")) + assert(testState.getWithoutEnforcingTTL().toSeq === Seq("v1", "v2", "v3")) + + val ttlExpirationMs = timestampMs + 60000 + var ttlValues = testState.getTTLValues() + assert(ttlValues.nonEmpty) + assert(ttlValues.forall(_._2 === ttlExpirationMs)) + var ttlStateValueIterator = testState.getValuesInTTLState() + assert(ttlStateValueIterator.hasNext) + + // increment batchProcessingTime, or watermark and ensure expired value is not returned + val nextBatchHandle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(), + Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]], + TimeMode.ProcessingTime(), batchTimestampMs = Some(ttlExpirationMs)) + + val nextBatchTestState: ListStateImplWithTTL[String] = + nextBatchHandle.getListState[String]("testState", Encoders.STRING, ttlConfig) + .asInstanceOf[ListStateImplWithTTL[String]] + + ImplicitGroupingKeyTracker.setImplicitKey("test_key") + + // ensure get does not return the expired value + assert(!nextBatchTestState.exists()) + assert(nextBatchTestState.get().isEmpty) + + // ttl value should still exist in state + ttlValues = nextBatchTestState.getTTLValues() + assert(ttlValues.nonEmpty) + assert(ttlValues.forall(_._2 === ttlExpirationMs)) + ttlStateValueIterator = nextBatchTestState.getValuesInTTLState() + assert(ttlStateValueIterator.hasNext) + assert(ttlStateValueIterator.next() === ttlExpirationMs) + assert(ttlStateValueIterator.isEmpty) + + // getWithoutTTL should still return the expired value + assert(nextBatchTestState.getWithoutEnforcingTTL().toSeq === Seq("v1", "v2", "v3")) + + nextBatchTestState.clear() + assert(!nextBatchTestState.exists()) + assert(nextBatchTestState.get().isEmpty) + } + } Review Comment: Done -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
