Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
it writes operatorStateMetadata out
  • Loading branch information
ericm-db committed Jun 7, 2024
commit d24677b3e7d49d45e858437c5893581394e75ef1
Original file line number Diff line number Diff line change
@@ -0,0 +1,71 @@
/*
* 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.json4s.{DefaultFormats, Formats, JValue}
import org.json4s.JsonAST.{JBool, JString}
import org.json4s.JsonDSL._

// Enum to store the types of state variables we support
sealed trait StateVariableType

case object ValueState extends StateVariableType
case object ListState extends StateVariableType
case object MapState extends StateVariableType

// This object is used to convert the state type from string to the corresponding enum
object StateVariableType {
def withName(name: String): StateVariableType = name match {
case "ValueState" => ValueState
case "ListState" => ListState
case "MapState" => MapState
case _ => throw new IllegalArgumentException(s"Unknown state type: $name")
}
}

// This class is used to store the information about a state variable.
// It is stored in operatorProperties for the TransformWithStateExec operator
// to be able to validate that the State Variables are the same across restarts.
class StateVariableInfo(
val stateName: String,
val stateType: StateVariableType,
val isTtlEnabled: Boolean
) {
def jsonValue: JValue = {
("stateName" -> JString(stateName)) ~
("stateType" -> JString(stateType.toString)) ~
("isTtlEnabled" -> JBool(isTtlEnabled))
}
}

// This object is used to convert the state variable information
// from JSON to a list of StateVariableInfo
object StateVariableInfo {
implicit val formats: Formats = DefaultFormats
def fromJson(json: Any): List[StateVariableInfo] = {
assert(json.isInstanceOf[List[_]], s"Expected List but got ${json.getClass}")
val stateVariables = json.asInstanceOf[List[Map[String, Any]]]
// Extract each JValue to StateVariableInfo
stateVariables.map { stateVariable =>
new StateVariableInfo(
stateVariable("stateName").asInstanceOf[String],
StateVariableType.withName(stateVariable("stateType").asInstanceOf[String]),
stateVariable("isTtlEnabled").asInstanceOf[Boolean]
)
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -94,6 +94,9 @@ class StatefulProcessorHandleImpl(
*/
private[sql] val ttlStates: util.List[TTLState] = new util.ArrayList[TTLState]()

private[sql] val stateVariables: util.List[StateVariableInfo] =
new util.ArrayList[StateVariableInfo]()

private val BATCH_QUERY_ID = "00000000-0000-0000-0000-000000000000"

private def buildQueryInfo(): QueryInfo = {
Expand Down Expand Up @@ -131,6 +134,7 @@ class StatefulProcessorHandleImpl(
verifyStateVarOperations("get_value_state")
incrementMetric("numValueStateVars")
val resultState = new ValueStateImpl[T](store, stateName, keyEncoder, valEncoder)
stateVariables.add(new StateVariableInfo(stateName, ValueState, false))
resultState
}

Expand All @@ -146,6 +150,7 @@ class StatefulProcessorHandleImpl(
keyEncoder, valEncoder, ttlConfig, batchTimestampMs.get)
incrementMetric("numValueStateWithTTLVars")
ttlStates.add(valueStateWithTTL)
stateVariables.add(new StateVariableInfo(stateName, ValueState, true))
valueStateWithTTL
}

Expand Down Expand Up @@ -242,6 +247,7 @@ class StatefulProcessorHandleImpl(
verifyStateVarOperations("get_list_state")
incrementMetric("numListStateVars")
val resultState = new ListStateImpl[T](store, stateName, keyEncoder, valEncoder)
stateVariables.add(new StateVariableInfo(stateName, ListState, false))
resultState
}

Expand Down Expand Up @@ -273,7 +279,7 @@ class StatefulProcessorHandleImpl(
keyEncoder, valEncoder, ttlConfig, batchTimestampMs.get)
incrementMetric("numListStateWithTTLVars")
ttlStates.add(listStateWithTTL)

stateVariables.add(new StateVariableInfo(stateName, ListState, true))
listStateWithTTL
}

Expand All @@ -284,6 +290,7 @@ class StatefulProcessorHandleImpl(
verifyStateVarOperations("get_map_state")
incrementMetric("numMapStateVars")
val resultState = new MapStateImpl[K, V](store, stateName, keyEncoder, userKeyEnc, valEncoder)
stateVariables.add(new StateVariableInfo(stateName, MapState, false))
resultState
}

Expand All @@ -300,7 +307,7 @@ class StatefulProcessorHandleImpl(
valEncoder, ttlConfig, batchTimestampMs.get)
incrementMetric("numMapStateWithTTLVars")
ttlStates.add(mapStateWithTTL)

stateVariables.add(new StateVariableInfo(stateName, MapState, true))
mapStateWithTTL
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,14 @@ package org.apache.spark.sql.execution.streaming
import java.util.UUID
import java.util.concurrent.TimeUnit.NANOSECONDS

import scala.jdk.CollectionConverters.CollectionHasAsScala

import org.json4s.{DefaultFormats, JArray, JString}
import org.json4s.JsonAST.JValue
import org.json4s.JsonDSL._
import org.json4s.jackson.JsonMethods
import org.json4s.jackson.JsonMethods.{compact, render}

import org.apache.spark.broadcast.Broadcast
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
Expand Down Expand Up @@ -75,8 +83,32 @@ case class TransformWithStateExec(
initialState: SparkPlan)
extends BinaryExecNode with StateStoreWriter with WatermarkSupport with ObjectProducerExec {

val operatorProperties: OperatorProperties =
OperatorProperties.create(
sparkContext,
"colFamilyMetadata"
)

override def operatorStateMetadataVersion: Int = 2

override def shortName: String = "transformWithStateExec"


/** Metadata of this stateful operator and its states stores. */
override def operatorStateMetadata(): OperatorStateMetadata = {
val info = getStateInfo
val operatorInfo = OperatorInfoV1(info.operatorId, shortName)
val stateStoreInfo =
Array(StateStoreMetadataV1(StateStoreId.DEFAULT_STORE_NAME, 0, info.numPartitions))

val operatorPropertiesJson: JValue = ("timeMode" -> JString(timeMode.toString)) ~
("outputMode" -> JString(outputMode.toString)) ~
("stateVariables" -> operatorProperties.value.get("stateVariables"))

val json = compact(render(operatorPropertiesJson))
OperatorStateMetadataV2(operatorInfo, stateStoreInfo, json)
}

override def shouldRunAnotherBatch(newInputWatermark: Long): Boolean = {
if (timeMode == ProcessingTime) {
// TODO: check if we can return true only if actual timers are registered, or there is
Expand Down Expand Up @@ -306,6 +338,9 @@ case class TransformWithStateExec(
store.abort()
}
}
operatorProperties.add(Map
("stateVariables" -> JArray(processorHandle.stateVariables.
asScala.map(_.jsonValue).toList)))
setStoreMetrics(store)
setOperatorMetrics()
statefulProcessor.close()
Expand Down Expand Up @@ -561,6 +596,14 @@ object TransformWithStateExec {
initialStateDeserializer,
initialState)
}

def deserializeOperatorProperties(json: String): Map[String, Any] = {
val parsedJson = JsonMethods.parse(json)

implicit val formats = DefaultFormats
val deserializedMap: Map[String, Any] = parsedJson.extract[Map[String, Any]]
deserializedMap
}
}
// scalastyle:on argcount

Original file line number Diff line number Diff line change
Expand Up @@ -140,6 +140,8 @@ trait StateStoreWriter extends StatefulOperator with PythonSQLMetrics { self: Sp
*/
def produceOutputWatermark(inputWatermarkMs: Long): Option[Long] = Some(inputWatermarkMs)

def operatorStateMetadataVersion: Int = 1

override lazy val metrics = statefulOperatorCustomMetrics ++ Map(
"numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"),
"numRowsDroppedByWatermark" -> SQLMetrics.createMetric(sparkContext,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import org.apache.spark.SparkRuntimeException
import org.apache.spark.internal.Logging
import org.apache.spark.sql.{Dataset, Encoders}
import org.apache.spark.sql.catalyst.util.stringToFile
import org.apache.spark.sql.execution.datasources.v2.state.StateSourceOptions
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, RocksDBStateStoreProvider, StatefulProcessorCannotPerformOperationWithInvalidHandleState, StateStoreMultipleColumnFamiliesNotSupportedException}
import org.apache.spark.sql.functions.timestamp_seconds
Expand Down Expand Up @@ -448,6 +449,75 @@ class TransformWithStateSuite extends StateStoreMetricsTest
}
}

test("verify that operatorProperties contain all stateVariables") {
withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
classOf[RocksDBStateStoreProvider].getName) {
withTempDir { chkptDir =>
val clock = new StreamManualClock

val inputData = MemoryStream[String]
val result = inputData.toDS()
.groupByKey(x => x)
.transformWithState(new RunningCountStatefulProcessorWithProcTimeTimer(),
TimeMode.ProcessingTime(),
OutputMode.Update())

testStream(result, OutputMode.Update())(
StartStream(
Trigger.ProcessingTime("1 second"),
triggerClock = clock,
checkpointLocation = chkptDir.getCanonicalPath
),
AddData(inputData, "a"),
AdvanceManualClock(1 * 1000),
CheckNewAnswer(("a", "1")),
Execute { q =>
assert(q.lastProgress.stateOperators(0).customMetrics.get("numValueStateVars") > 0)
assert(q.lastProgress.stateOperators(0).customMetrics.get("numRegisteredTimers") === 1)
},
AddData(inputData, "b"),
AdvanceManualClock(1 * 1000),
CheckNewAnswer(("b", "1")),

AddData(inputData, "b"),
AdvanceManualClock(10 * 1000),
CheckNewAnswer(("a", "-1"), ("b", "2")),

AddData(inputData, "b"),
AddData(inputData, "c"),
AdvanceManualClock(1 * 1000),
CheckNewAnswer(("c", "1")), // should remove 'b' as count reaches 3

AddData(inputData, "d"),
AdvanceManualClock(10 * 1000),
CheckNewAnswer(("c", "-1"), ("d", "1")),
StopStream
)

val df = spark.read
.format("state-metadata")
.option(StateSourceOptions.PATH, chkptDir.getAbsolutePath)
.load()

val propsString = df.select("operatorProperties").
collect().head.getString(0)

val map = TransformWithStateExec.
deserializeOperatorProperties(propsString)
assert(map("timeMode") === "ProcessingTime")
assert(map("outputMode") === "Update")

val stateVariableInfos = StateVariableInfo.fromJson(
map("stateVariables"))
assert(stateVariableInfos.size === 1)
val stateVariableInfo = stateVariableInfos.head
assert(stateVariableInfo.stateName === "countState")
assert(stateVariableInfo.isTtlEnabled === false)
assert(stateVariableInfo.stateType === ValueState)
}
}
}

test("transformWithState - streaming with rocksdb and processing time timer " +
"and multiple timers should succeed") {
withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
Expand Down