HeartSaVioR commented on code in PR #47508:
URL: https://github.com/apache/spark/pull/47508#discussion_r1696512836
##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreErrors.scala:
##########
@@ -173,8 +173,52 @@ object StateStoreErrors {
StateStoreProviderDoesNotSupportFineGrainedReplay = {
new StateStoreProviderDoesNotSupportFineGrainedReplay(inputClass)
}
+
Review Comment:
nit: remove one empty line
##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateVariableUtils.scala:
##########
@@ -0,0 +1,148 @@
+/*
+ * 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
+import org.json4s.JsonAST._
+import org.json4s.JsonDSL._
+import org.json4s.jackson.JsonMethods
+import org.json4s.jackson.JsonMethods.{compact, render}
+
+import org.apache.spark.internal.Logging
+import
org.apache.spark.sql.execution.streaming.StateVariableType.StateVariableType
+import org.apache.spark.sql.execution.streaming.state.StateStoreErrors
+
+/**
+ * This file contains utility classes and functions for managing state
variables in
+ * the operatorProperties field of the OperatorStateMetadata for
TransformWithState.
+ * We use these utils to read and write state variable information for
validation purposes
+ */
+// Enum of possible State Variable types
+object StateVariableType extends Enumeration {
+ type StateVariableType = Value
+ val ValueState, ListState, MapState = Value
+}
+
+case class TransformWithStateVariableInfo(
+ stateName: String,
+ stateVariableType: StateVariableType,
+ ttlEnabled: Boolean) {
+ def jsonValue: JValue = {
+ ("stateName" -> JString(stateName)) ~
+ ("stateVariableType" -> JString(stateVariableType.toString)) ~
+ ("ttlEnabled" -> JBool(ttlEnabled))
+ }
+
+ def json: String = {
+ compact(render(jsonValue))
+ }
+}
+
+object TransformWithStateVariableInfo {
+
+ def fromJson(json: String): TransformWithStateVariableInfo = {
+ implicit val formats: DefaultFormats.type = DefaultFormats
+ val parsed = JsonMethods.parse(json).extract[Map[String, Any]]
+ fromMap(parsed)
+ }
+
+ def fromMap(map: Map[String, Any]): TransformWithStateVariableInfo = {
+ val stateName = map("stateName").asInstanceOf[String]
+ val stateVariableType = StateVariableType.withName(
+ map("stateVariableType").asInstanceOf[String])
+ val ttlEnabled = map("ttlEnabled").asInstanceOf[Boolean]
+ TransformWithStateVariableInfo(stateName, stateVariableType, ttlEnabled)
+ }
+}
+object TransformWithStateVariableUtils {
Review Comment:
nit: shall we have empty line in above?
##########
common/utils/src/main/resources/error/error-conditions.json:
##########
@@ -3852,12 +3858,24 @@
],
"sqlState" : "42802"
},
+ "STATE_STORE_INVALID_CONFIG_AFTER_RESTART" : {
+ "message" : [
+ "<configName>=<oldConfig> is not equal to <newConfig>. Please set
<configName> to <oldConfig>, or restart with a new checkpoint directory."
Review Comment:
nit: probably reword as similar as below? `Cannot change <configName> from
<oldConfig> to <newConfig>`.
##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala:
##########
@@ -213,6 +213,27 @@ class IncrementalExecution(
statefulOp match {
case ssw: StateStoreWriter =>
val metadata = ssw.operatorStateMetadata(stateSchemaPaths)
+ // validate metadata
+ if (isFirstBatch && currentBatchId != 0) {
+ // If we are restarting from a different checkpoint directory
+ // there may be a mismatch between the stateful operators in the
+ // physical plan and the metadata.
+ val oldMetadata = try {
+ OperatorStateMetadataReader.createReader(
+ new Path(checkpointLocation,
ssw.getStateInfo.operatorId.toString),
+ hadoopConf, ssw.operatorStateMetadataVersion).read()
+ } catch {
+ case e: Exception =>
+ logWarning(s"Error reading metadata path for stateful
operator. This " +
Review Comment:
Looks like we got enforced to use structured logging framework. Linter error
seems to come from here.
##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateVariableUtils.scala:
##########
@@ -0,0 +1,148 @@
+/*
+ * 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
+import org.json4s.JsonAST._
+import org.json4s.JsonDSL._
+import org.json4s.jackson.JsonMethods
+import org.json4s.jackson.JsonMethods.{compact, render}
+
+import org.apache.spark.internal.Logging
+import
org.apache.spark.sql.execution.streaming.StateVariableType.StateVariableType
+import org.apache.spark.sql.execution.streaming.state.StateStoreErrors
+
+/**
+ * This file contains utility classes and functions for managing state
variables in
+ * the operatorProperties field of the OperatorStateMetadata for
TransformWithState.
+ * We use these utils to read and write state variable information for
validation purposes
Review Comment:
nit: while we are here, let's add `.` at the end.
##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala:
##########
@@ -983,6 +1045,163 @@ class TransformWithStateSuite extends
StateStoreMetricsTest
}
}
}
+
+ test("test that different outputMode after query restart fails") {
+ withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+ classOf[RocksDBStateStoreProvider].getName,
+ SQLConf.SHUFFLE_PARTITIONS.key ->
+ TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS.toString) {
+ withTempDir { checkpointDir =>
+ val inputData = MemoryStream[String]
+ val result1 = inputData.toDS()
+ .groupByKey(x => x)
+ .transformWithState(new RunningCountStatefulProcessor(),
+ TimeMode.None(),
+ OutputMode.Update())
+
+ testStream(result1, OutputMode.Update())(
+ StartStream(checkpointLocation = checkpointDir.getCanonicalPath),
+ AddData(inputData, "a"),
+ CheckNewAnswer(("a", "1")),
+ StopStream
+ )
+ val result2 = inputData.toDS()
+ .groupByKey(x => x)
+ .transformWithState(new RunningCountStatefulProcessor(),
+ TimeMode.None(),
+ OutputMode.Append())
+ testStream(result2, OutputMode.Update())(
+ StartStream(checkpointLocation = checkpointDir.getCanonicalPath),
+ AddData(inputData, "a"),
+ ExpectFailure[StateStoreInvalidConfigAfterRestart] { t =>
+ assert(t.getMessage.contains("outputMode"))
+ assert(t.getMessage.contains("is not equal"))
+ }
+ )
+ }
+ }
+ }
+
+ test("test that changing between different state variable types fails") {
+ withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+ classOf[RocksDBStateStoreProvider].getName,
+ SQLConf.SHUFFLE_PARTITIONS.key ->
+ TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS.toString) {
+ withTempDir { checkpointDir =>
+ val inputData = MemoryStream[String]
+ val result = inputData.toDS()
+ .groupByKey(x => x)
+ .transformWithState(new RunningCountStatefulProcessor(),
+ TimeMode.None(),
+ OutputMode.Update())
+
+ testStream(result, OutputMode.Update())(
+ StartStream(checkpointLocation = checkpointDir.getCanonicalPath),
+ AddData(inputData, "a"),
+ CheckNewAnswer(("a", "1")),
+ StopStream
+ )
+ val result2 = inputData.toDS()
+ .groupByKey(x => x)
+ .transformWithState(new RunningCountListStatefulProcessor(),
+ TimeMode.None(),
+ OutputMode.Update())
+ testStream(result2, OutputMode.Update())(
+ StartStream(checkpointLocation = checkpointDir.getCanonicalPath),
+ AddData(inputData, "a"),
+ ExpectFailure[StateStoreInvalidVariableTypeChange] { t =>
+ assert(t.getMessage.contains("Cannot change countState"))
Review Comment:
ditto
##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala:
##########
@@ -983,6 +1045,163 @@ class TransformWithStateSuite extends
StateStoreMetricsTest
}
}
}
+
+ test("test that different outputMode after query restart fails") {
+ withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+ classOf[RocksDBStateStoreProvider].getName,
+ SQLConf.SHUFFLE_PARTITIONS.key ->
+ TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS.toString) {
+ withTempDir { checkpointDir =>
+ val inputData = MemoryStream[String]
+ val result1 = inputData.toDS()
+ .groupByKey(x => x)
+ .transformWithState(new RunningCountStatefulProcessor(),
+ TimeMode.None(),
+ OutputMode.Update())
+
+ testStream(result1, OutputMode.Update())(
+ StartStream(checkpointLocation = checkpointDir.getCanonicalPath),
+ AddData(inputData, "a"),
+ CheckNewAnswer(("a", "1")),
+ StopStream
+ )
+ val result2 = inputData.toDS()
+ .groupByKey(x => x)
+ .transformWithState(new RunningCountStatefulProcessor(),
+ TimeMode.None(),
+ OutputMode.Append())
+ testStream(result2, OutputMode.Update())(
+ StartStream(checkpointLocation = checkpointDir.getCanonicalPath),
+ AddData(inputData, "a"),
+ ExpectFailure[StateStoreInvalidConfigAfterRestart] { t =>
+ assert(t.getMessage.contains("outputMode"))
Review Comment:
Is it possible to use checkError to validate arguments? Exception with error
class is encouraged to be validated with checkError.
##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateVariableUtils.scala:
##########
@@ -0,0 +1,148 @@
+/*
+ * 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
+import org.json4s.JsonAST._
+import org.json4s.JsonDSL._
+import org.json4s.jackson.JsonMethods
+import org.json4s.jackson.JsonMethods.{compact, render}
+
+import org.apache.spark.internal.Logging
+import
org.apache.spark.sql.execution.streaming.StateVariableType.StateVariableType
+import org.apache.spark.sql.execution.streaming.state.StateStoreErrors
+
+/**
+ * This file contains utility classes and functions for managing state
variables in
+ * the operatorProperties field of the OperatorStateMetadata for
TransformWithState.
+ * We use these utils to read and write state variable information for
validation purposes
+ */
+// Enum of possible State Variable types
+object StateVariableType extends Enumeration {
+ type StateVariableType = Value
+ val ValueState, ListState, MapState = Value
+}
+
+case class TransformWithStateVariableInfo(
+ stateName: String,
+ stateVariableType: StateVariableType,
+ ttlEnabled: Boolean) {
+ def jsonValue: JValue = {
+ ("stateName" -> JString(stateName)) ~
+ ("stateVariableType" -> JString(stateVariableType.toString)) ~
+ ("ttlEnabled" -> JBool(ttlEnabled))
+ }
+
+ def json: String = {
+ compact(render(jsonValue))
+ }
+}
+
+object TransformWithStateVariableInfo {
+
+ def fromJson(json: String): TransformWithStateVariableInfo = {
+ implicit val formats: DefaultFormats.type = DefaultFormats
+ val parsed = JsonMethods.parse(json).extract[Map[String, Any]]
+ fromMap(parsed)
+ }
+
+ def fromMap(map: Map[String, Any]): TransformWithStateVariableInfo = {
+ val stateName = map("stateName").asInstanceOf[String]
+ val stateVariableType = StateVariableType.withName(
+ map("stateVariableType").asInstanceOf[String])
+ val ttlEnabled = map("ttlEnabled").asInstanceOf[Boolean]
+ TransformWithStateVariableInfo(stateName, stateVariableType, ttlEnabled)
+ }
+}
+object TransformWithStateVariableUtils {
+ def getValueState(stateName: String, ttlEnabled: Boolean):
TransformWithStateVariableInfo = {
+ TransformWithStateVariableInfo(stateName, StateVariableType.ValueState,
ttlEnabled)
+ }
+
+ def getListState(stateName: String, ttlEnabled: Boolean):
TransformWithStateVariableInfo = {
+ TransformWithStateVariableInfo(stateName, StateVariableType.ListState,
ttlEnabled)
+ }
+
+ def getMapState(stateName: String, ttlEnabled: Boolean):
TransformWithStateVariableInfo = {
+ TransformWithStateVariableInfo(stateName, StateVariableType.MapState,
ttlEnabled)
+ }
+}
+
+case class TransformWithStateOperatorProperties(
+ val timeMode: String,
+ val outputMode: String,
+ val stateVariables: List[TransformWithStateVariableInfo]) {
+
+ def json: String = {
+ val stateVariablesJson = stateVariables.map(_.jsonValue)
+ val json =
+ ("timeMode" -> timeMode) ~
+ ("outputMode" -> outputMode) ~
+ ("stateVariables" -> stateVariablesJson)
+ compact(render(json))
+ }
+}
+
+object TransformWithStateOperatorProperties extends Logging {
+ def fromJson(json: String): TransformWithStateOperatorProperties = {
+ implicit val formats: DefaultFormats.type = DefaultFormats
+ val jsonMap = JsonMethods.parse(json).extract[Map[String, Any]]
+ TransformWithStateOperatorProperties(
+ jsonMap("timeMode").asInstanceOf[String],
+ jsonMap("outputMode").asInstanceOf[String],
+ jsonMap("stateVariables").asInstanceOf[List[Map[String, Any]]].map {
stateVarMap =>
+ TransformWithStateVariableInfo.fromMap(stateVarMap)
+ }
+ )
+ }
+
+ // This function is to confirm that the operator properties and state
variables have
+ // only changed in an acceptable way after query restart. If the properties
have changed
+ // in an unacceptable way, this function will throw an exception.
+ def validateOperatorProperties(
+ oldOperatorProperties: TransformWithStateOperatorProperties,
+ newOperatorProperties: TransformWithStateOperatorProperties): Unit = {
+ if (oldOperatorProperties.timeMode != newOperatorProperties.timeMode) {
+ throw StateStoreErrors.invalidConfigChangedAfterRestart(
+ "timeMode", oldOperatorProperties.timeMode,
newOperatorProperties.timeMode)
+ }
+
+ if (oldOperatorProperties.outputMode != newOperatorProperties.outputMode) {
+ throw StateStoreErrors.invalidConfigChangedAfterRestart(
+ "outputMode", oldOperatorProperties.outputMode,
newOperatorProperties.outputMode)
+ }
+
+ val oldStateVariableInfos = oldOperatorProperties.stateVariables
Review Comment:
Just wanted to recall (sorry I've been context-switched a lot), adding /
removing state variables is supported during restart, hence this way of check,
do I understand correctly?
##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala:
##########
@@ -983,6 +1045,163 @@ class TransformWithStateSuite extends
StateStoreMetricsTest
}
}
}
+
+ test("test that different outputMode after query restart fails") {
+ withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+ classOf[RocksDBStateStoreProvider].getName,
+ SQLConf.SHUFFLE_PARTITIONS.key ->
+ TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS.toString) {
+ withTempDir { checkpointDir =>
+ val inputData = MemoryStream[String]
+ val result1 = inputData.toDS()
+ .groupByKey(x => x)
+ .transformWithState(new RunningCountStatefulProcessor(),
+ TimeMode.None(),
+ OutputMode.Update())
+
+ testStream(result1, OutputMode.Update())(
+ StartStream(checkpointLocation = checkpointDir.getCanonicalPath),
+ AddData(inputData, "a"),
+ CheckNewAnswer(("a", "1")),
+ StopStream
+ )
+ val result2 = inputData.toDS()
+ .groupByKey(x => x)
+ .transformWithState(new RunningCountStatefulProcessor(),
+ TimeMode.None(),
+ OutputMode.Append())
+ testStream(result2, OutputMode.Update())(
+ StartStream(checkpointLocation = checkpointDir.getCanonicalPath),
+ AddData(inputData, "a"),
+ ExpectFailure[StateStoreInvalidConfigAfterRestart] { t =>
+ assert(t.getMessage.contains("outputMode"))
+ assert(t.getMessage.contains("is not equal"))
+ }
+ )
+ }
+ }
+ }
+
+ test("test that changing between different state variable types fails") {
+ withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+ classOf[RocksDBStateStoreProvider].getName,
+ SQLConf.SHUFFLE_PARTITIONS.key ->
+ TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS.toString) {
+ withTempDir { checkpointDir =>
+ val inputData = MemoryStream[String]
+ val result = inputData.toDS()
+ .groupByKey(x => x)
+ .transformWithState(new RunningCountStatefulProcessor(),
+ TimeMode.None(),
+ OutputMode.Update())
+
+ testStream(result, OutputMode.Update())(
+ StartStream(checkpointLocation = checkpointDir.getCanonicalPath),
+ AddData(inputData, "a"),
+ CheckNewAnswer(("a", "1")),
+ StopStream
+ )
+ val result2 = inputData.toDS()
+ .groupByKey(x => x)
+ .transformWithState(new RunningCountListStatefulProcessor(),
+ TimeMode.None(),
+ OutputMode.Update())
+ testStream(result2, OutputMode.Update())(
+ StartStream(checkpointLocation = checkpointDir.getCanonicalPath),
+ AddData(inputData, "a"),
+ ExpectFailure[StateStoreInvalidVariableTypeChange] { t =>
+ assert(t.getMessage.contains("Cannot change countState"))
+ }
+ )
+ }
+ }
+ }
+
+ test("test that different timeMode after query restart fails") {
+ withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+ classOf[RocksDBStateStoreProvider].getName,
+ SQLConf.SHUFFLE_PARTITIONS.key ->
+ TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS.toString) {
+ withTempDir { checkpointDir =>
+ val clock = new StreamManualClock
+ val inputData = MemoryStream[String]
+ val result1 = inputData.toDS()
+ .groupByKey(x => x)
+ .transformWithState(new RunningCountStatefulProcessor(),
+ TimeMode.None(),
+ OutputMode.Update())
+
+ testStream(result1, OutputMode.Update())(
+ StartStream(checkpointLocation = checkpointDir.getCanonicalPath),
+ AddData(inputData, "a"),
+ CheckNewAnswer(("a", "1")),
+ StopStream
+ )
+ val result2 = inputData.toDS()
+ .groupByKey(x => x)
+ .transformWithState(new RunningCountStatefulProcessor(),
+ TimeMode.ProcessingTime(),
+ OutputMode.Update())
+ testStream(result2, OutputMode.Update())(
+ StartStream(
+ checkpointLocation = checkpointDir.getCanonicalPath,
+ trigger = Trigger.ProcessingTime("1 second"),
+ triggerClock = clock),
+ AddData(inputData, "a"),
+ AdvanceManualClock(1 * 1000),
+ ExpectFailure[StateStoreInvalidConfigAfterRestart] { t =>
+ assert(t.getMessage.contains("timeMode"))
+ assert(t.getMessage.contains("is not equal"))
+ }
+ )
+ }
+ }
+ }
+
+ test("test that introducing TTL after restart fails query") {
+ withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+ classOf[RocksDBStateStoreProvider].getName,
+ SQLConf.SHUFFLE_PARTITIONS.key ->
+ TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS.toString) {
+ withTempDir { checkpointDir =>
+ val inputData = MemoryStream[String]
+ val clock = new StreamManualClock
+ val result = inputData.toDS()
+ .groupByKey(x => x)
+ .transformWithState(new RunningCountStatefulProcessor(),
+ TimeMode.ProcessingTime(),
+ OutputMode.Update())
+
+ testStream(result, OutputMode.Update())(
+ StartStream(
+ trigger = Trigger.ProcessingTime("1 second"),
+ checkpointLocation = checkpointDir.getCanonicalPath,
+ triggerClock = clock),
+ AddData(inputData, "a"),
+ AdvanceManualClock(1 * 1000),
+ CheckNewAnswer(("a", "1")),
+ AdvanceManualClock(1 * 1000),
+ StopStream
+ )
+ val result2 = inputData.toDS()
+ .groupByKey(x => x)
+ .transformWithState(new RunningCountStatefulProcessorWithTTL(),
+ TimeMode.ProcessingTime(),
+ OutputMode.Update())
+ testStream(result2, OutputMode.Update())(
+ StartStream(
+ trigger = Trigger.ProcessingTime("1 second"),
+ checkpointLocation = checkpointDir.getCanonicalPath,
+ triggerClock = clock),
+ AddData(inputData, "a"),
+ AdvanceManualClock(1 * 1000),
+ ExpectFailure[StateStoreValueSchemaNotCompatible] { t =>
+
assert(t.getMessage.contains("StructField(ttlExpirationMs,LongType,true)"))
Review Comment:
ditto
##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateVariableUtils.scala:
##########
@@ -0,0 +1,148 @@
+/*
+ * 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
+import org.json4s.JsonAST._
+import org.json4s.JsonDSL._
+import org.json4s.jackson.JsonMethods
+import org.json4s.jackson.JsonMethods.{compact, render}
+
+import org.apache.spark.internal.Logging
+import
org.apache.spark.sql.execution.streaming.StateVariableType.StateVariableType
+import org.apache.spark.sql.execution.streaming.state.StateStoreErrors
+
+/**
+ * This file contains utility classes and functions for managing state
variables in
+ * the operatorProperties field of the OperatorStateMetadata for
TransformWithState.
+ * We use these utils to read and write state variable information for
validation purposes
Review Comment:
Also you'd want to check that this comment is not bound to the class doc of
object StateVariableType. I guess it doesn't as we have another line comment,
but wanted to double confirm.
##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateVariableUtils.scala:
##########
@@ -0,0 +1,148 @@
+/*
+ * 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
+import org.json4s.JsonAST._
+import org.json4s.JsonDSL._
+import org.json4s.jackson.JsonMethods
+import org.json4s.jackson.JsonMethods.{compact, render}
+
+import org.apache.spark.internal.Logging
+import
org.apache.spark.sql.execution.streaming.StateVariableType.StateVariableType
+import org.apache.spark.sql.execution.streaming.state.StateStoreErrors
+
+/**
+ * This file contains utility classes and functions for managing state
variables in
+ * the operatorProperties field of the OperatorStateMetadata for
TransformWithState.
+ * We use these utils to read and write state variable information for
validation purposes
+ */
+// Enum of possible State Variable types
+object StateVariableType extends Enumeration {
+ type StateVariableType = Value
+ val ValueState, ListState, MapState = Value
+}
+
+case class TransformWithStateVariableInfo(
+ stateName: String,
+ stateVariableType: StateVariableType,
+ ttlEnabled: Boolean) {
+ def jsonValue: JValue = {
+ ("stateName" -> JString(stateName)) ~
+ ("stateVariableType" -> JString(stateVariableType.toString)) ~
+ ("ttlEnabled" -> JBool(ttlEnabled))
+ }
+
+ def json: String = {
+ compact(render(jsonValue))
+ }
+}
+
+object TransformWithStateVariableInfo {
+
+ def fromJson(json: String): TransformWithStateVariableInfo = {
+ implicit val formats: DefaultFormats.type = DefaultFormats
+ val parsed = JsonMethods.parse(json).extract[Map[String, Any]]
+ fromMap(parsed)
+ }
+
+ def fromMap(map: Map[String, Any]): TransformWithStateVariableInfo = {
+ val stateName = map("stateName").asInstanceOf[String]
+ val stateVariableType = StateVariableType.withName(
+ map("stateVariableType").asInstanceOf[String])
+ val ttlEnabled = map("ttlEnabled").asInstanceOf[Boolean]
+ TransformWithStateVariableInfo(stateName, stateVariableType, ttlEnabled)
+ }
+}
+object TransformWithStateVariableUtils {
+ def getValueState(stateName: String, ttlEnabled: Boolean):
TransformWithStateVariableInfo = {
+ TransformWithStateVariableInfo(stateName, StateVariableType.ValueState,
ttlEnabled)
+ }
+
+ def getListState(stateName: String, ttlEnabled: Boolean):
TransformWithStateVariableInfo = {
+ TransformWithStateVariableInfo(stateName, StateVariableType.ListState,
ttlEnabled)
+ }
+
+ def getMapState(stateName: String, ttlEnabled: Boolean):
TransformWithStateVariableInfo = {
+ TransformWithStateVariableInfo(stateName, StateVariableType.MapState,
ttlEnabled)
+ }
+}
+
+case class TransformWithStateOperatorProperties(
+ val timeMode: String,
Review Comment:
Do you have specific reason to define this as val? This is already defined
as case class.
##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala:
##########
@@ -983,6 +1045,163 @@ class TransformWithStateSuite extends
StateStoreMetricsTest
}
}
}
+
+ test("test that different outputMode after query restart fails") {
+ withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+ classOf[RocksDBStateStoreProvider].getName,
+ SQLConf.SHUFFLE_PARTITIONS.key ->
+ TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS.toString) {
+ withTempDir { checkpointDir =>
+ val inputData = MemoryStream[String]
+ val result1 = inputData.toDS()
+ .groupByKey(x => x)
+ .transformWithState(new RunningCountStatefulProcessor(),
+ TimeMode.None(),
+ OutputMode.Update())
+
+ testStream(result1, OutputMode.Update())(
+ StartStream(checkpointLocation = checkpointDir.getCanonicalPath),
+ AddData(inputData, "a"),
+ CheckNewAnswer(("a", "1")),
+ StopStream
+ )
+ val result2 = inputData.toDS()
+ .groupByKey(x => x)
+ .transformWithState(new RunningCountStatefulProcessor(),
+ TimeMode.None(),
+ OutputMode.Append())
+ testStream(result2, OutputMode.Update())(
+ StartStream(checkpointLocation = checkpointDir.getCanonicalPath),
+ AddData(inputData, "a"),
+ ExpectFailure[StateStoreInvalidConfigAfterRestart] { t =>
+ assert(t.getMessage.contains("outputMode"))
+ assert(t.getMessage.contains("is not equal"))
+ }
+ )
+ }
+ }
+ }
+
+ test("test that changing between different state variable types fails") {
+ withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+ classOf[RocksDBStateStoreProvider].getName,
+ SQLConf.SHUFFLE_PARTITIONS.key ->
+ TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS.toString) {
+ withTempDir { checkpointDir =>
+ val inputData = MemoryStream[String]
+ val result = inputData.toDS()
+ .groupByKey(x => x)
+ .transformWithState(new RunningCountStatefulProcessor(),
+ TimeMode.None(),
+ OutputMode.Update())
+
+ testStream(result, OutputMode.Update())(
+ StartStream(checkpointLocation = checkpointDir.getCanonicalPath),
+ AddData(inputData, "a"),
+ CheckNewAnswer(("a", "1")),
+ StopStream
+ )
+ val result2 = inputData.toDS()
+ .groupByKey(x => x)
+ .transformWithState(new RunningCountListStatefulProcessor(),
+ TimeMode.None(),
+ OutputMode.Update())
+ testStream(result2, OutputMode.Update())(
+ StartStream(checkpointLocation = checkpointDir.getCanonicalPath),
+ AddData(inputData, "a"),
+ ExpectFailure[StateStoreInvalidVariableTypeChange] { t =>
+ assert(t.getMessage.contains("Cannot change countState"))
+ }
+ )
+ }
+ }
+ }
+
+ test("test that different timeMode after query restart fails") {
+ withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+ classOf[RocksDBStateStoreProvider].getName,
+ SQLConf.SHUFFLE_PARTITIONS.key ->
+ TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS.toString) {
+ withTempDir { checkpointDir =>
+ val clock = new StreamManualClock
+ val inputData = MemoryStream[String]
+ val result1 = inputData.toDS()
+ .groupByKey(x => x)
+ .transformWithState(new RunningCountStatefulProcessor(),
+ TimeMode.None(),
+ OutputMode.Update())
+
+ testStream(result1, OutputMode.Update())(
+ StartStream(checkpointLocation = checkpointDir.getCanonicalPath),
+ AddData(inputData, "a"),
+ CheckNewAnswer(("a", "1")),
+ StopStream
+ )
+ val result2 = inputData.toDS()
+ .groupByKey(x => x)
+ .transformWithState(new RunningCountStatefulProcessor(),
+ TimeMode.ProcessingTime(),
+ OutputMode.Update())
+ testStream(result2, OutputMode.Update())(
+ StartStream(
+ checkpointLocation = checkpointDir.getCanonicalPath,
+ trigger = Trigger.ProcessingTime("1 second"),
+ triggerClock = clock),
+ AddData(inputData, "a"),
+ AdvanceManualClock(1 * 1000),
+ ExpectFailure[StateStoreInvalidConfigAfterRestart] { t =>
+ assert(t.getMessage.contains("timeMode"))
Review Comment:
ditto
--
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]