sahnib commented on code in PR #45932: URL: https://github.com/apache/spark/pull/45932#discussion_r1562685312
########## sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala: ########## @@ -0,0 +1,235 @@ +/* + * 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 + +/** + * Provides concrete implementation for list of values associated with a state variable + * 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 keyEnc - Spark SQL encoder for key + * @param valEncoder - Spark SQL encoder for value + * @tparam S - data type of object that will be stored in the list + */ +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 encodedKey = stateTypesEncoder.encodeGroupingKey() + 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() + } + + /** Append an entry to the list. */ + override def appendValue(newState: S): Unit = { + StateStoreErrors.requireNonNullStateValue(newState, stateName) + store.merge(stateTypesEncoder.encodeGroupingKey(), + stateTypesEncoder.encodeValue(newState, ttlExpirationMs), stateName) + upsertTTLForStateKey() + } + + /** Append an entire list to the existing value. */ + override def appendList(newState: Array[S]): Unit = { + validateNewState(newState) + + val encodedKey = stateTypesEncoder.encodeGroupingKey() + newState.foreach { v => + val encodedValue = stateTypesEncoder.encodeValue(v, ttlExpirationMs) + store.merge(encodedKey, encodedValue, stateName) + } + upsertTTLForStateKey() + } + + /** Remove this state. */ + override def clear(): Unit = { + store.remove(stateTypesEncoder.encodeGroupingKey(), stateName) + } + + private def validateNewState(newState: Array[S]): Unit = { + StateStoreErrors.requireNonNullStateValue(newState, stateName) + StateStoreErrors.requireNonEmptyListStateValue(newState, stateName) + + newState.foreach { v => + StateStoreErrors.requireNonNullStateValue(v, stateName) + } + } + + /** + * + * Loops through all the values associated with the grouping key, and removes + * the expired elements from the list. + * @param groupingKey grouping key for which cleanup should be performed. + */ + override def clearIfExpired(groupingKey: Array[Byte]): Unit = { + val encodedGroupingKey = stateTypesEncoder.encodeSerializedGroupingKey(groupingKey) + val unsafeRowValuesIterator = store.valuesIterator(encodedGroupingKey, stateName) + // We clear the list, and use the iterator to put back all of the non-expired values + store.remove(encodedGroupingKey, stateName) + var isFirst = true + unsafeRowValuesIterator.foreach { encodedValue => + if (!stateTypesEncoder.isExpired(encodedValue, batchTimestampMs)) { + if (isFirst) { + store.put(encodedGroupingKey, encodedValue, stateName) + isFirst = false + } else { + store.merge(encodedGroupingKey, encodedValue, stateName) + } + } + } + } + + private def upsertTTLForStateKey(): Unit = { + val serializedGroupingKey = stateTypesEncoder.serializeGroupingKey() Review Comment: Can we avoid this serialization happening 2 times? We will always serialize the key to store it in the state variable column family, and then re-serialize it here to store it in ttlState. We can do this only once and that should be good enough. ########## sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala: ########## @@ -0,0 +1,235 @@ +/* + * 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 + +/** + * Provides concrete implementation for list of values associated with a state variable + * 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 keyEnc - Spark SQL encoder for key + * @param valEncoder - Spark SQL encoder for value + * @tparam S - data type of object that will be stored in the list + */ +class ListStateImplWithTTL[S]( Review Comment: @anishshri-db added some metrics for `TransformWithState` as part of this PR - https://github.com/apache/spark/pull/45937. I think we need to calculate expired values in this class to keep the metrics up to date. ########## sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StatefulProcessorHandleSuite.scala: ########## @@ -237,6 +237,24 @@ class StatefulProcessorHandleSuite extends StateVariableSuiteBase { } } + test(s"ttl States are populated for listState and ttlMode=ProcessingTime") { Review Comment: Can you also add listState instantiation to the test `ttl States are not populated for ttlMode=NoTTL` below, and ensure creating a ListState does not add anything to ttlStates when ttlMode=NoTTL. ########## sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala: ########## @@ -141,14 +135,15 @@ class ValueStateImplWithTTL[S]( /** * Read the ttl value associated with the grouping key. */ - private[sql] def getTTLValue(): Option[Long] = { + private[sql] def getTTLValue(): Option[(S, Long)] = { val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey() val retRow = store.get(encodedGroupingKey, stateName) - if (retRow != null) { - stateTypesEncoder.decodeTtlExpirationMs(retRow) - } else { - None + // if the returned row is not null, we want to return the value associated with the + // ttlExpiration + Option(retRow).flatMap { row => + val ttlExpiration = stateTypesEncoder.decodeTtlExpirationMs(row) + ttlExpiration.map(expiration => (stateTypesEncoder.decodeValue(row), expiration)) Review Comment: We can simply do ``` if (retRow != null) { (stateTypesEncoder.decodeValue(row), stateTypesEncoder.decodeTtlExpirationMs(retRow)) } else { None } ``` or ``` Option(retRow).flatMap { row => val ttlExpiration = stateTypesEncoder.decodeTtlExpirationMs(row) (stateTypesEncoder.decodeValue(row), expiration), ttlExpiration) } ``` ########## sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala: ########## @@ -333,7 +333,8 @@ class ValueStateSuite extends StateVariableSuiteBase { val ttlExpirationMs = timestampMs + 60000 var ttlValue = testState.getTTLValue() assert(ttlValue.isDefined) - assert(ttlValue.get === ttlExpirationMs) + val ttlExpiration1 = ttlValue.get._2 Review Comment: [nit] Can be renamed to `ttlExpirationMsValueInState`. ########## sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala: ########## @@ -0,0 +1,235 @@ +/* + * 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 + +/** + * Provides concrete implementation for list of values associated with a state variable + * 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 keyEnc - Spark SQL encoder for key + * @param valEncoder - Spark SQL encoder for value + * @tparam S - data type of object that will be stored in the list + */ +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 encodedKey = stateTypesEncoder.encodeGroupingKey() + 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() + } + + /** Append an entry to the list. */ + override def appendValue(newState: S): Unit = { + StateStoreErrors.requireNonNullStateValue(newState, stateName) + store.merge(stateTypesEncoder.encodeGroupingKey(), + stateTypesEncoder.encodeValue(newState, ttlExpirationMs), stateName) + upsertTTLForStateKey() + } + + /** Append an entire list to the existing value. */ + override def appendList(newState: Array[S]): Unit = { + validateNewState(newState) + + val encodedKey = stateTypesEncoder.encodeGroupingKey() + newState.foreach { v => + val encodedValue = stateTypesEncoder.encodeValue(v, ttlExpirationMs) + store.merge(encodedKey, encodedValue, stateName) + } + upsertTTLForStateKey() + } + + /** Remove this state. */ + override def clear(): Unit = { + store.remove(stateTypesEncoder.encodeGroupingKey(), stateName) + } + + private def validateNewState(newState: Array[S]): Unit = { + StateStoreErrors.requireNonNullStateValue(newState, stateName) + StateStoreErrors.requireNonEmptyListStateValue(newState, stateName) + + newState.foreach { v => + StateStoreErrors.requireNonNullStateValue(v, stateName) + } + } + + /** + * + * Loops through all the values associated with the grouping key, and removes + * the expired elements from the list. + * @param groupingKey grouping key for which cleanup should be performed. + */ + override def clearIfExpired(groupingKey: Array[Byte]): Unit = { + val encodedGroupingKey = stateTypesEncoder.encodeSerializedGroupingKey(groupingKey) + val unsafeRowValuesIterator = store.valuesIterator(encodedGroupingKey, stateName) + // We clear the list, and use the iterator to put back all of the non-expired values + store.remove(encodedGroupingKey, stateName) + var isFirst = true + unsafeRowValuesIterator.foreach { encodedValue => + if (!stateTypesEncoder.isExpired(encodedValue, batchTimestampMs)) { + if (isFirst) { + store.put(encodedGroupingKey, encodedValue, stateName) + isFirst = false + } else { + store.merge(encodedGroupingKey, encodedValue, stateName) + } + } + } + } + + private def upsertTTLForStateKey(): Unit = { + val serializedGroupingKey = stateTypesEncoder.serializeGroupingKey() + upsertTTLForStateKey(ttlExpirationMs, serializedGroupingKey) + } + + /* + * Internal methods to probe state for testing. The below methods exist for unit tests + * to read the state ttl values, and ensure that values are persisted correctly in + * the underlying state store. + */ + + /** + * Retrieves the value from State even if its expired. This method is used + * in tests to read the state store value, and ensure if its cleaned up at the + * end of the micro-batch. + */ + private[sql] def getWithoutEnforcingTTL(): Iterator[S] = { + val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey() + val unsafeRowValuesIterator = store.valuesIterator(encodedGroupingKey, stateName) + unsafeRowValuesIterator.map{ + valueUnsafeRow => + stateTypesEncoder.decodeValue(valueUnsafeRow) + } + } + + /** + * Read the ttl value associated with the grouping key. + */ + private[sql] def getTTLValues(): Iterator[(S, Long)] = { + val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey() + val unsafeRowValuesIterator = store.valuesIterator(encodedGroupingKey, stateName) + unsafeRowValuesIterator.map{ + valueUnsafeRow => + (stateTypesEncoder.decodeValue(valueUnsafeRow), + stateTypesEncoder.decodeTtlExpirationMs(valueUnsafeRow).get) + } + } + /** + * Get all ttl values stored in ttl state for current implicit + * grouping key. + */ + private[sql] def getValuesInTTLState(): Iterator[Long] = { Review Comment: As this function is exactly the same for `ValueStateImplWithTTL` and `ListStateImplWithTTL`, maybe we should move it to the common base class - `SingleKeyTTLStateImpl`. Now that I think about it more, its more apt for this method to be in class `SingleKeyTTLStateImpl`. ########## sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithValueStateTTLSuite.scala: ########## @@ -27,22 +26,10 @@ import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.StreamManualClock -case class InputEvent( - key: String, - action: String, - value: Int, - eventTime: Timestamp = null) - -case class OutputEvent( - key: String, - value: Int, - isTTLValue: Boolean, - ttlValue: Long) - object TTLInputProcessFunction { def processRow( - row: InputEvent, - valueState: ValueStateImplWithTTL[Int]): Iterator[OutputEvent] = { + row: InputEvent, + valueState: ValueStateImplWithTTL[Int]): Iterator[OutputEvent] = { Review Comment: Thanks for fixing the indent here. ########## sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala: ########## @@ -0,0 +1,302 @@ +/* + * 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.streaming + +import java.time.Duration + +import org.apache.spark.sql.Encoders +import org.apache.spark.sql.execution.streaming.{ListStateImplWithTTL, MemoryStream} +import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.streaming.util.StreamManualClock + +class ListStateTTLProcessor(ttlConfig: TTLConfig) + extends StatefulProcessor[String, InputEvent, OutputEvent] { + + @transient private var _listState: ListStateImplWithTTL[Int] = _ + + override def init( + outputMode: OutputMode, + timeoutMode: TimeoutMode, + ttlMode: TTLMode): Unit = { + _listState = getHandle + .getListState("listState", Encoders.scalaInt, ttlConfig) + .asInstanceOf[ListStateImplWithTTL[Int]] + } + + override def handleInputRows( + key: String, + inputRows: Iterator[InputEvent], + timerValues: TimerValues, + expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = { + var results = List[OutputEvent]() + + inputRows.foreach { row => + val resultIter = processRow(row, _listState) + resultIter.foreach { r => + results = r :: results + } + } + + results.iterator + } + + def processRow( + row: InputEvent, + listState: ListStateImplWithTTL[Int]): Iterator[OutputEvent] = { + + var results = List[OutputEvent]() + val key = row.key + if (row.action == "get") { + val currState = listState.get() + currState.foreach { v => + results = OutputEvent(key, v, isTTLValue = false, -1) :: results + } + } else if (row.action == "get_without_enforcing_ttl") { + val currState = listState.getWithoutEnforcingTTL() + currState.foreach { v => + results = OutputEvent(key, v, isTTLValue = false, -1) :: results + } + } else if (row.action == "get_ttl_value_from_state") { + val ttlValues = listState.getTTLValues() + ttlValues.foreach { ttlValue => + results = OutputEvent(key, ttlValue._1, isTTLValue = true, ttlValue._2) :: results + } + } else if (row.action == "put") { + listState.put(Array(row.value)) + } else if (row.action == "append") { + listState.appendValue(row.value) + } else if (row.action == "get_values_in_ttl_state") { + val ttlValues = listState.getValuesInTTLState() + ttlValues.foreach { v => + results = OutputEvent(key, -1, isTTLValue = true, ttlValue = v) :: results + } + } + + results.iterator + } +} + +/** + * Test suite for testing list state with TTL. + * We use the base TTL suite with a list state processor. + */ +class TransformWithListStateTTLSuite extends TransformWithStateTTLTest { + + import testImplicits._ + override def getProcessor(ttlConfig: TTLConfig): + StatefulProcessor[String, InputEvent, OutputEvent] = { + new ListStateTTLProcessor(ttlConfig) + } + + test("verify iterator works with expired values in middle of list") { + withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> + classOf[RocksDBStateStoreProvider].getName, + SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + withTempDir { checkpointLocation => + val ttlConfig1 = TTLConfig(ttlDuration = Duration.ofMinutes(3)) + val inputStream = MemoryStream[InputEvent] + val result1 = inputStream.toDS() + .groupByKey(x => x.key) + .transformWithState( + getProcessor(ttlConfig1), + TimeoutMode.NoTimeouts(), + TTLMode.ProcessingTimeTTL(), + OutputMode.Append()) + + val clock = new StreamManualClock + // add 3 elements with a duration of a minute + testStream(result1)( + StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock, + checkpointLocation = checkpointLocation.getAbsolutePath), + AddData(inputStream, InputEvent("k1", "put", 1)), + AdvanceManualClock(1 * 1000), + AddData(inputStream, InputEvent("k1", "append", 2)), + AddData(inputStream, InputEvent("k1", "append", 3)), + // advance clock to trigger processing + AdvanceManualClock(1 * 1000), + CheckNewAnswer(), + // get ttl values + AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)), + AdvanceManualClock(1 * 1000), + CheckNewAnswer( + OutputEvent("k1", 1, isTTLValue = true, 181000), + OutputEvent("k1", 2, isTTLValue = true, 182000), + OutputEvent("k1", 3, isTTLValue = true, 182000) + ), + AddData(inputStream, InputEvent("k1", "get", -1, null)), + AdvanceManualClock(1 * 1000), + CheckNewAnswer( + OutputEvent("k1", 1, isTTLValue = false, -1), + OutputEvent("k1", 2, isTTLValue = false, -1), + OutputEvent("k1", 3, isTTLValue = false, -1) + ), + StopStream + ) + + val ttlConfig2 = TTLConfig(ttlDuration = Duration.ofSeconds(15)) + val result2 = inputStream.toDS() + .groupByKey(x => x.key) + .transformWithState( + getProcessor(ttlConfig2), + TimeoutMode.NoTimeouts(), + TTLMode.ProcessingTimeTTL(), + OutputMode.Append()) + // add 3 elements with a duration of 15 seconds + testStream(result2)( + StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock, + checkpointLocation = checkpointLocation.getAbsolutePath), + AddData(inputStream, InputEvent("k1", "append", 4)), + AddData(inputStream, InputEvent("k1", "append", 5)), + AddData(inputStream, InputEvent("k1", "append", 6)), + // advance clock to trigger processing + AdvanceManualClock(1 * 1000), + CheckNewAnswer(), + // get all elements without enforcing ttl + AddData(inputStream, InputEvent("k1", "get_without_enforcing_ttl", -1, null)), + AdvanceManualClock(1 * 1000), + CheckNewAnswer( + OutputEvent("k1", 1, isTTLValue = false, -1), + OutputEvent("k1", 2, isTTLValue = false, -1), + OutputEvent("k1", 3, isTTLValue = false, -1), + OutputEvent("k1", 4, isTTLValue = false, -1), + OutputEvent("k1", 5, isTTLValue = false, -1), + OutputEvent("k1", 6, isTTLValue = false, -1) + ), + StopStream + ) + // add 3 more elements with a duration of a minute + testStream(result1)( + StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock, + checkpointLocation = checkpointLocation.getAbsolutePath), + AddData(inputStream, InputEvent("k1", "append", 7)), + AddData(inputStream, InputEvent("k1", "append", 8)), + AddData(inputStream, InputEvent("k1", "append", 9)), + // advance clock to trigger processing + AdvanceManualClock(1 * 1000), + CheckNewAnswer(), + // advance clock to expire the middle three elements + AdvanceManualClock(45 * 1000), + // Get all elements in the list + AddData(inputStream, InputEvent("k1", "get", -1, null)), + AdvanceManualClock(1 * 1000), + // validate that the expired elements are not returned + CheckNewAnswer( + OutputEvent("k1", 1, isTTLValue = false, -1), + OutputEvent("k1", 2, isTTLValue = false, -1), + OutputEvent("k1", 3, isTTLValue = false, -1), + OutputEvent("k1", 7, isTTLValue = false, -1), + OutputEvent("k1", 8, isTTLValue = false, -1), + OutputEvent("k1", 9, isTTLValue = false, -1) + ), + StopStream Review Comment: Can we also get these elements withoutEnforcingTTL and ensure they are actually deleted from the state Store. Similarly, lets also verify the timer value for middle 3 elements is also deleted. ########## sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithValueStateTTLSuite.scala: ########## @@ -399,7 +205,7 @@ class TransformWithValueStateTTLSuite AddData(inputStream, InputEvent(ttlKey, "get_ttl_value_from_state", -1)), AddData(inputStream, InputEvent(noTtlKey, "get_ttl_value_from_state", -1)), AdvanceManualClock(1 * 1000), - CheckNewAnswer(OutputEvent(ttlKey, -1, isTTLValue = true, 61000)), + CheckNewAnswer(OutputEvent(ttlKey, 1, isTTLValue = true, 61000)), Review Comment: Was this testcase incorrect? Why do we need to change this value to 1 instead? ########## sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala: ########## @@ -0,0 +1,302 @@ +/* + * 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.streaming + +import java.time.Duration + +import org.apache.spark.sql.Encoders +import org.apache.spark.sql.execution.streaming.{ListStateImplWithTTL, MemoryStream} +import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.streaming.util.StreamManualClock + +class ListStateTTLProcessor(ttlConfig: TTLConfig) + extends StatefulProcessor[String, InputEvent, OutputEvent] { + + @transient private var _listState: ListStateImplWithTTL[Int] = _ + + override def init( + outputMode: OutputMode, + timeoutMode: TimeoutMode, + ttlMode: TTLMode): Unit = { + _listState = getHandle + .getListState("listState", Encoders.scalaInt, ttlConfig) + .asInstanceOf[ListStateImplWithTTL[Int]] + } + + override def handleInputRows( + key: String, + inputRows: Iterator[InputEvent], + timerValues: TimerValues, + expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = { + var results = List[OutputEvent]() + + inputRows.foreach { row => + val resultIter = processRow(row, _listState) + resultIter.foreach { r => + results = r :: results + } + } + + results.iterator + } + + def processRow( + row: InputEvent, + listState: ListStateImplWithTTL[Int]): Iterator[OutputEvent] = { + + var results = List[OutputEvent]() + val key = row.key + if (row.action == "get") { + val currState = listState.get() + currState.foreach { v => + results = OutputEvent(key, v, isTTLValue = false, -1) :: results + } + } else if (row.action == "get_without_enforcing_ttl") { + val currState = listState.getWithoutEnforcingTTL() + currState.foreach { v => + results = OutputEvent(key, v, isTTLValue = false, -1) :: results + } + } else if (row.action == "get_ttl_value_from_state") { + val ttlValues = listState.getTTLValues() + ttlValues.foreach { ttlValue => + results = OutputEvent(key, ttlValue._1, isTTLValue = true, ttlValue._2) :: results + } + } else if (row.action == "put") { + listState.put(Array(row.value)) + } else if (row.action == "append") { + listState.appendValue(row.value) + } else if (row.action == "get_values_in_ttl_state") { + val ttlValues = listState.getValuesInTTLState() + ttlValues.foreach { v => + results = OutputEvent(key, -1, isTTLValue = true, ttlValue = v) :: results + } + } + + results.iterator + } +} + +/** + * Test suite for testing list state with TTL. + * We use the base TTL suite with a list state processor. + */ +class TransformWithListStateTTLSuite extends TransformWithStateTTLTest { + + import testImplicits._ + override def getProcessor(ttlConfig: TTLConfig): + StatefulProcessor[String, InputEvent, OutputEvent] = { + new ListStateTTLProcessor(ttlConfig) + } + + test("verify iterator works with expired values in middle of list") { + withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> + classOf[RocksDBStateStoreProvider].getName, + SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + withTempDir { checkpointLocation => + val ttlConfig1 = TTLConfig(ttlDuration = Duration.ofMinutes(3)) + val inputStream = MemoryStream[InputEvent] + val result1 = inputStream.toDS() + .groupByKey(x => x.key) + .transformWithState( + getProcessor(ttlConfig1), + TimeoutMode.NoTimeouts(), + TTLMode.ProcessingTimeTTL(), + OutputMode.Append()) + + val clock = new StreamManualClock + // add 3 elements with a duration of a minute + testStream(result1)( + StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock, + checkpointLocation = checkpointLocation.getAbsolutePath), + AddData(inputStream, InputEvent("k1", "put", 1)), + AdvanceManualClock(1 * 1000), + AddData(inputStream, InputEvent("k1", "append", 2)), + AddData(inputStream, InputEvent("k1", "append", 3)), + // advance clock to trigger processing + AdvanceManualClock(1 * 1000), + CheckNewAnswer(), + // get ttl values + AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)), + AdvanceManualClock(1 * 1000), + CheckNewAnswer( + OutputEvent("k1", 1, isTTLValue = true, 181000), + OutputEvent("k1", 2, isTTLValue = true, 182000), + OutputEvent("k1", 3, isTTLValue = true, 182000) + ), + AddData(inputStream, InputEvent("k1", "get", -1, null)), + AdvanceManualClock(1 * 1000), + CheckNewAnswer( + OutputEvent("k1", 1, isTTLValue = false, -1), + OutputEvent("k1", 2, isTTLValue = false, -1), + OutputEvent("k1", 3, isTTLValue = false, -1) + ), + StopStream + ) + + val ttlConfig2 = TTLConfig(ttlDuration = Duration.ofSeconds(15)) + val result2 = inputStream.toDS() + .groupByKey(x => x.key) + .transformWithState( + getProcessor(ttlConfig2), + TimeoutMode.NoTimeouts(), + TTLMode.ProcessingTimeTTL(), + OutputMode.Append()) + // add 3 elements with a duration of 15 seconds + testStream(result2)( + StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock, + checkpointLocation = checkpointLocation.getAbsolutePath), + AddData(inputStream, InputEvent("k1", "append", 4)), + AddData(inputStream, InputEvent("k1", "append", 5)), + AddData(inputStream, InputEvent("k1", "append", 6)), + // advance clock to trigger processing + AdvanceManualClock(1 * 1000), + CheckNewAnswer(), + // get all elements without enforcing ttl + AddData(inputStream, InputEvent("k1", "get_without_enforcing_ttl", -1, null)), + AdvanceManualClock(1 * 1000), + CheckNewAnswer( + OutputEvent("k1", 1, isTTLValue = false, -1), + OutputEvent("k1", 2, isTTLValue = false, -1), + OutputEvent("k1", 3, isTTLValue = false, -1), + OutputEvent("k1", 4, isTTLValue = false, -1), + OutputEvent("k1", 5, isTTLValue = false, -1), + OutputEvent("k1", 6, isTTLValue = false, -1) + ), + StopStream + ) + // add 3 more elements with a duration of a minute + testStream(result1)( + StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock, + checkpointLocation = checkpointLocation.getAbsolutePath), + AddData(inputStream, InputEvent("k1", "append", 7)), + AddData(inputStream, InputEvent("k1", "append", 8)), + AddData(inputStream, InputEvent("k1", "append", 9)), + // advance clock to trigger processing + AdvanceManualClock(1 * 1000), + CheckNewAnswer(), + // advance clock to expire the middle three elements + AdvanceManualClock(45 * 1000), + // Get all elements in the list + AddData(inputStream, InputEvent("k1", "get", -1, null)), + AdvanceManualClock(1 * 1000), + // validate that the expired elements are not returned + CheckNewAnswer( + OutputEvent("k1", 1, isTTLValue = false, -1), + OutputEvent("k1", 2, isTTLValue = false, -1), + OutputEvent("k1", 3, isTTLValue = false, -1), + OutputEvent("k1", 7, isTTLValue = false, -1), + OutputEvent("k1", 8, isTTLValue = false, -1), + OutputEvent("k1", 9, isTTLValue = false, -1) + ), + StopStream + ) + } + } + } + + test("verify iterator works with expired values in beginning of list") { + withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> + classOf[RocksDBStateStoreProvider].getName, + SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + withTempDir { checkpointLocation => + val inputStream = MemoryStream[InputEvent] + val ttlConfig1 = TTLConfig(ttlDuration = Duration.ofMinutes(1)) + val result1 = inputStream.toDS() + .groupByKey(x => x.key) + .transformWithState( + getProcessor(ttlConfig1), + TimeoutMode.NoTimeouts(), + TTLMode.ProcessingTimeTTL(), + OutputMode.Append()) + + val ttlConfig2 = TTLConfig(ttlDuration = Duration.ofMinutes(2)) + val result2 = inputStream.toDS() + .groupByKey(x => x.key) + .transformWithState( + getProcessor(ttlConfig2), + TimeoutMode.NoTimeouts(), + TTLMode.ProcessingTimeTTL(), + OutputMode.Append()) + + val clock = new StreamManualClock + // add 3 elements with a duration of a minute + testStream(result1)( + StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock, + checkpointLocation = checkpointLocation.getAbsolutePath), + AddData(inputStream, InputEvent("k1", "put", 1)), + AdvanceManualClock(1 * 1000), + AddData(inputStream, InputEvent("k1", "append", 2)), + AddData(inputStream, InputEvent("k1", "append", 3)), + // advance clock to trigger processing + AdvanceManualClock(1 * 1000), + CheckNewAnswer(), + // get ttl values + AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)), + AdvanceManualClock(1 * 1000), + CheckNewAnswer( + OutputEvent("k1", 1, isTTLValue = true, 61000), + OutputEvent("k1", 2, isTTLValue = true, 62000), + OutputEvent("k1", 3, isTTLValue = true, 62000) + ), + AddData(inputStream, InputEvent("k1", "get", -1, null)), + AdvanceManualClock(1 * 1000), + CheckNewAnswer( + OutputEvent("k1", 1, isTTLValue = false, -1), + OutputEvent("k1", 2, isTTLValue = false, -1), + OutputEvent("k1", 3, isTTLValue = false, -1) + ), + StopStream + ) + + // add 3 elements with a duration of two minutes + testStream(result2)( + StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock, + checkpointLocation = checkpointLocation.getAbsolutePath), + AddData(inputStream, InputEvent("k1", "append", 4)), + AddData(inputStream, InputEvent("k1", "append", 5)), + AddData(inputStream, InputEvent("k1", "append", 6)), + // advance clock to trigger processing + AdvanceManualClock(1 * 1000), + CheckNewAnswer(), + // get ttl values + AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)), + AdvanceManualClock(1 * 1000), + CheckNewAnswer( + OutputEvent("k1", 1, isTTLValue = true, 61000), + OutputEvent("k1", 2, isTTLValue = true, 62000), + OutputEvent("k1", 3, isTTLValue = true, 62000), + OutputEvent("k1", 4, isTTLValue = true, 125000), + OutputEvent("k1", 5, isTTLValue = true, 125000), + OutputEvent("k1", 6, isTTLValue = true, 125000) + ), + // expire beginning values + AdvanceManualClock(60 * 1000), + AddData(inputStream, InputEvent("k1", "get", -1, null)), + AdvanceManualClock(1 * 1000), + CheckNewAnswer( + OutputEvent("k1", 4, isTTLValue = false, -1), + OutputEvent("k1", 5, isTTLValue = false, -1), + OutputEvent("k1", 6, isTTLValue = false, -1) + ), Review Comment: Same as above. Can we also get these elements withoutEnforcingTTL and ensure they are actually deleted from the state Store. Similarly, lets also verify the timer value for middle 3 elements is also deleted. -- 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