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 compiles lol
  • Loading branch information
ericm-db committed Jul 16, 2024
commit 948b2f75f2afaf783cd1f3692044757987b09cd5
6 changes: 6 additions & 0 deletions common/utils/src/main/resources/error/error-conditions.json
Original file line number Diff line number Diff line change
Expand Up @@ -3785,6 +3785,12 @@
],
"sqlState" : "42802"
},
"STATE_STORE_DUPLICATE_STATE_VARIABLE_DEFINED" : {
"message" : [
"State variable with name <stateName> has already been defined in the StatefulProcessor."
],
"sqlState" : "42802"
},
"STATE_STORE_HANDLE_NOT_INITIALIZED" : {
"message" : [
"The handle has not been initialized for this StatefulProcessor.",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -316,10 +316,17 @@ class DriverStatefulProcessorHandleImpl(timeMode: TimeMode, keyExprEnc: Expressi

def getStateVariableInfos: Map[String, TransformWithStateVariableInfo] = stateVariableInfos.toMap

def checkIfDuplicateVariableDefined(stateName: String): Unit = {
if (columnFamilySchemas.contains(stateName)) {
throw StateStoreErrors.duplicateStateVariableDefined(stateName)
}
}

override def getValueState[T](stateName: String, valEncoder: Encoder[T]): ValueState[T] = {
verifyStateVarOperations("get_value_state", PRE_INIT)
val colFamilySchema = columnFamilySchemaUtils.
getValueStateSchema(stateName, keyExprEnc, valEncoder, false)
checkIfDuplicateVariableDefined(stateName)
columnFamilySchemas.put(stateName, colFamilySchema)
val stateVariableInfo = stateVariableUtils.
getValueState(stateName, ttlEnabled = false)
Expand All @@ -334,6 +341,7 @@ class DriverStatefulProcessorHandleImpl(timeMode: TimeMode, keyExprEnc: Expressi
verifyStateVarOperations("get_value_state", PRE_INIT)
val colFamilySchema = columnFamilySchemaUtils.
getValueStateSchema(stateName, keyExprEnc, valEncoder, true)
checkIfDuplicateVariableDefined(stateName)
columnFamilySchemas.put(stateName, colFamilySchema)
val stateVariableInfo = stateVariableUtils.
getValueState(stateName, ttlEnabled = true)
Expand All @@ -345,6 +353,7 @@ class DriverStatefulProcessorHandleImpl(timeMode: TimeMode, keyExprEnc: Expressi
verifyStateVarOperations("get_list_state", PRE_INIT)
val colFamilySchema = columnFamilySchemaUtils.
getListStateSchema(stateName, keyExprEnc, valEncoder, false)
checkIfDuplicateVariableDefined(stateName)
columnFamilySchemas.put(stateName, colFamilySchema)
val stateVariableInfo = stateVariableUtils.
getListState(stateName, ttlEnabled = false)
Expand All @@ -359,6 +368,7 @@ class DriverStatefulProcessorHandleImpl(timeMode: TimeMode, keyExprEnc: Expressi
verifyStateVarOperations("get_list_state", PRE_INIT)
val colFamilySchema = columnFamilySchemaUtils.
getListStateSchema(stateName, keyExprEnc, valEncoder, true)
checkIfDuplicateVariableDefined(stateName)
columnFamilySchemas.put(stateName, colFamilySchema)
val stateVariableInfo = stateVariableUtils.
getListState(stateName, ttlEnabled = true)
Expand All @@ -373,6 +383,7 @@ class DriverStatefulProcessorHandleImpl(timeMode: TimeMode, keyExprEnc: Expressi
verifyStateVarOperations("get_map_state", PRE_INIT)
val colFamilySchema = columnFamilySchemaUtils.
getMapStateSchema(stateName, keyExprEnc, userKeyEnc, valEncoder, false)
checkIfDuplicateVariableDefined(stateName)
columnFamilySchemas.put(stateName, colFamilySchema)
val stateVariableInfo = stateVariableUtils.
getMapState(stateName, ttlEnabled = false)
Expand All @@ -388,6 +399,7 @@ class DriverStatefulProcessorHandleImpl(timeMode: TimeMode, keyExprEnc: Expressi
verifyStateVarOperations("get_map_state", PRE_INIT)
val colFamilySchema = columnFamilySchemaUtils.
getMapStateSchema(stateName, keyExprEnc, valEncoder, userKeyEnc, true)
checkIfDuplicateVariableDefined(stateName)
columnFamilySchemas.put(stateName, colFamilySchema)
val stateVariableInfo = stateVariableUtils.
getMapState(stateName, ttlEnabled = true)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -179,11 +179,23 @@ object StateStoreErrors {
new StateStoreInvalidConfigAfterRestart(configName, oldConfig, newConfig)
}

def duplicateStateVariableDefined(stateName: String):
StateStoreDuplicateStateVariableDefined = {
new StateStoreDuplicateStateVariableDefined(stateName)
}

def invalidVariableTypeChange(stateName: String, oldType: String, newType: String):
StateStoreInvalidVariableTypeChange = {
new StateStoreInvalidVariableTypeChange(stateName, oldType, newType)
}
}
class StateStoreDuplicateStateVariableDefined(stateName: String)
extends SparkRuntimeException(
errorClass = "STATE_STORE_DUPLICATE_STATE_VARIABLE_DEFINED",
messageParameters = Map(
"stateName" -> stateName
)
)

class StateStoreInvalidConfigAfterRestart(configName: String, oldConfig: String, newConfig: String)
extends SparkUnsupportedOperationException(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ package org.apache.spark.sql.streaming

import java.io.File
import java.util.UUID

import org.apache.hadoop.fs.Path

import org.apache.spark.SparkRuntimeException
Expand All @@ -27,7 +28,7 @@ import org.apache.spark.sql.{Dataset, Encoders, Row}
import org.apache.spark.sql.catalyst.util.stringToFile
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchema.{COMPOSITE_KEY_ROW_SCHEMA, KEY_ROW_SCHEMA}
import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, ColumnFamilySchema, ColumnFamilySchemaV1, NoPrefixKeyStateEncoderSpec, OperatorInfoV1, OperatorStateMetadataV2, POJOTestClass, PrefixKeyScanStateEncoderSpec, RocksDBStateStoreProvider, StateSchemaV3File, StateStoreInvalidConfigAfterRestart, StateStoreInvalidVariableTypeChange, StateStoreMetadataV2, StateStoreMultipleColumnFamiliesNotSupportedException, StateStoreValueSchemaNotCompatible, StatefulProcessorCannotPerformOperationWithInvalidHandleState, TestClass}
import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, ColumnFamilySchema, ColumnFamilySchemaV1, NoPrefixKeyStateEncoderSpec, OperatorInfoV1, OperatorStateMetadataV2, POJOTestClass, PrefixKeyScanStateEncoderSpec, RocksDBStateStoreProvider, StatefulProcessorCannotPerformOperationWithInvalidHandleState, StateSchemaV3File, StateStoreInvalidConfigAfterRestart, StateStoreInvalidVariableTypeChange, StateStoreMetadataV2, StateStoreMultipleColumnFamiliesNotSupportedException, StateStoreValueSchemaNotCompatible, TestClass}
import org.apache.spark.sql.functions.timestamp_seconds
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.streaming.util.StreamManualClock
Expand Down