Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
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
11 changes: 11 additions & 0 deletions common/utils/src/main/resources/error/error-conditions.json
Original file line number Diff line number Diff line change
Expand Up @@ -5526,6 +5526,17 @@
"message" : [
"<provider> is not supported"
]
},
"UNSUPPORTED_STATEFUL_OPERATOR" : {
"message" : [
"Unsupported stateful operator <operatorName>. Please use a checkpoint with supported stateful operators."
]
},
"UNSUPPORTED_TRANSFORM_WITH_STATE_VARIABLE_TYPE" : {
"message" : [
"Unsupported transform with state variable type <variableType> (TTL_Enabled: <ttlEnabled>, ColFamilyName: <colFamilyName>).",
"Please use a checkpoint with supported transform with state variable types."
]
}
},
"sqlState" : "55019"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -120,7 +120,8 @@ case class FlatMapGroupsInPandasWithStateExec(
override def requiredChildOrdering: Seq[Seq[SortOrder]] = Seq(
groupingAttributes.map(SortOrder(_, Ascending)))

override def shortName: String = "applyInPandasWithState"
override def shortName: String =
StatefulOperatorsUtils.FLAT_MAP_GROUPS_IN_PANDAS_WITH_STATE_EXEC_OP_NAME
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nice - good to move these to utils


override protected def withNewChildInternal(
newChild: SparkPlan): FlatMapGroupsInPandasWithStateExec = copy(child = newChild)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,82 @@
/*
* 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.operators.stateful

import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOperatorsUtils
import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.FlatMapGroupsWithStatePartitionKeyExtractor
import org.apache.spark.sql.execution.streaming.operators.stateful.join.SymmetricHashJoinStateManager
import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.{TransformWithStatePartitionKeyExtractorFactory, TransformWithStateVariableInfo}
import org.apache.spark.sql.execution.streaming.state.{OfflineStateRepartitionErrors, StatePartitionKeyExtractor, StateStore, StateStoreId}
import org.apache.spark.sql.types.StructType

/**
* Factory for creating state partition key extractor for various streaming stateful operators.
* This is used for offline state repartitioning, when we need to repartition
* the state for a given operator. If an operator isn't included in this factory,
* then offline repartitioning will not be supported for it.
*
* To support offline repartitioning for a new stateful operator, you need to:
* 1. Create a state partition key extractor for the operator state.
* 2. Register the state partition key extractor in this factory.
*/
object StatePartitionKeyExtractorFactory {
import StatefulOperatorsUtils._

/**
* Creates a state partition key extractor for the given operator.
* An operator may have different extractor for different stores/column families.
*
* @param operatorName The name of the operator.
* @param stateKeySchema The schema of the state key.
* @param storeName The name of the store.
* @param colFamilyName The name of the column family.
* @param stateFormatVersion Optional, the version of the state format. Used by operators
* that have different extractors for different state formats.
* @param stateVariableInfo Optional, the state variable info for TransformWithState.
* @return The state partition key extractor.
*/
def create(
operatorName: String,
stateKeySchema: StructType,
storeName: String = StateStoreId.DEFAULT_STORE_NAME,
colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME,
stateFormatVersion: Option[Int] = None,
stateVariableInfo: Option[TransformWithStateVariableInfo] = None
): StatePartitionKeyExtractor = {
operatorName match {
case STATE_STORE_SAVE_EXEC_OP_NAME =>
new StreamingAggregationStatePartitionKeyExtractor(stateKeySchema)
case DEDUPLICATE_EXEC_OP_NAME =>
new StreamingDeduplicateStatePartitionKeyExtractor(stateKeySchema)
case DEDUPLICATE_WITHIN_WATERMARK_EXEC_OP_NAME =>
new StreamingDedupWithinWatermarkStatePartitionKeyExtractor(stateKeySchema)
case SESSION_WINDOW_STATE_STORE_SAVE_EXEC_OP_NAME =>
new StreamingSessionWindowStatePartitionKeyExtractor(stateKeySchema)
case SYMMETRIC_HASH_JOIN_EXEC_OP_NAME =>
SymmetricHashJoinStateManager.createPartitionKeyExtractor(
storeName, colFamilyName, stateKeySchema, stateFormatVersion.get)
case fmg if FLAT_MAP_GROUPS_OP_NAMES.contains(fmg) =>
new FlatMapGroupsWithStatePartitionKeyExtractor(stateKeySchema)
case tws if TRANSFORM_WITH_STATE_OP_NAMES.contains(tws) =>
TransformWithStatePartitionKeyExtractorFactory.create(
storeName, colFamilyName, stateKeySchema, stateVariableInfo.get)
case _ => throw OfflineStateRepartitionErrors
.unsupportedStatefulOperatorError(checkpointLocation = "", operatorName)
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeRow}
import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateUnsafeProjection, GenerateUnsafeRowJoiner}
import org.apache.spark.sql.catalyst.types.DataTypeUtils
import org.apache.spark.sql.execution.streaming.state.{ReadStateStore, StateStore, UnsafeRowPair}
import org.apache.spark.sql.execution.streaming.state.{NoopStatePartitionKeyExtractor, ReadStateStore, StateStore, UnsafeRowPair}
import org.apache.spark.sql.types.StructType

/**
Expand Down Expand Up @@ -205,3 +205,9 @@ class StreamingAggregationStateManagerImplV2(
}
}
}

/**
* For aggregation state v1 and v2, the state key is the partition key i.e. the aggregation key
*/
class StreamingAggregationStatePartitionKeyExtractor(stateKeySchema: StructType)
extends NoopStatePartitionKeyExtractor(stateKeySchema)
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import org.apache.spark.internal.LogKeys._
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Attribute, Literal, UnsafeProjection, UnsafeRow}
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
import org.apache.spark.sql.execution.streaming.state.{ReadStateStore, StateStore, UnsafeRowPair}
import org.apache.spark.sql.execution.streaming.state.{DropLastNFieldsStatePartitionKeyExtractor, ReadStateStore, StateStore, UnsafeRowPair}
import org.apache.spark.sql.types.{StructType, TimestampType}
import org.apache.spark.util.NextIterator

Expand Down Expand Up @@ -280,3 +280,10 @@ class StreamingSessionWindowHelper(sessionExpression: Attribute, inputSchema: Se
(window.getLong(0), window.getLong(1))
}
}

/**
* The State key is the session key (i.e. partition key) and the sessionStartTime.
* Drop the last field (sessionStartTime) to get the partition key.
*/
class StreamingSessionWindowStatePartitionKeyExtractor(stateKeySchema: StructType)
extends DropLastNFieldsStatePartitionKeyExtractor(stateKeySchema, numLastColsToDrop = 1)
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.plans.physical.Distribution
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.metric.SQLMetric
import org.apache.spark.sql.execution.streaming.operators.stateful.{StatefulOperatorPartitioning, StatefulOperatorStateInfo, StateStoreWriter, WatermarkSupport}
import org.apache.spark.sql.execution.streaming.operators.stateful.{StatefulOperatorPartitioning, StatefulOperatorStateInfo, StatefulOperatorsUtils, StateStoreWriter, WatermarkSupport}
import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinHelper._
import org.apache.spark.sql.execution.streaming.state._
import org.apache.spark.sql.streaming.{GroupStateTimeout, OutputMode}
Expand Down Expand Up @@ -95,7 +95,7 @@ trait FlatMapGroupsWithStateExecBase

override def keyExpressions: Seq[Attribute] = groupingAttributes

override def shortName: String = "flatMapGroupsWithState"
override def shortName: String = StatefulOperatorsUtils.FLAT_MAP_GROUPS_WITH_STATE_EXEC_OP_NAME

override def shouldRunAnotherBatch(newInputWatermark: Long): Boolean = {
timeoutConf match {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes
import org.apache.spark.sql.execution.ObjectOperator
import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.GroupStateImpl.NO_TIMESTAMP
import org.apache.spark.sql.execution.streaming.state.StateStore
import org.apache.spark.sql.execution.streaming.state.{NoopStatePartitionKeyExtractor, StateStore}
import org.apache.spark.sql.types._


Expand Down Expand Up @@ -246,3 +246,10 @@ object FlatMapGroupsWithStateExecHelper {
}
}
}

/**
* For FlatMapGroupsWithStateExec and FlatMapGroupsInPandasWithStateExec (v1 & v2),
* the state key is the partition key i.e. the grouping key
*/
class FlatMapGroupsWithStatePartitionKeyExtractor(stateKeySchema: StructType)
extends NoopStatePartitionKeyExtractor(stateKeySchema)
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ import org.apache.spark.sql.execution.metric.SQLMetric
import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOperatorStateInfo
import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOpStateStoreCheckpointInfo
import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinHelper._
import org.apache.spark.sql.execution.streaming.state.{KeyStateEncoderSpec, NoPrefixKeyStateEncoderSpec, StateSchemaBroadcast, StateStore, StateStoreCheckpointInfo, StateStoreColFamilySchema, StateStoreConf, StateStoreErrors, StateStoreId, StateStoreMetrics, StateStoreProvider, StateStoreProviderId, SupportsFineGrainedReplay}
import org.apache.spark.sql.execution.streaming.state.{DropLastNFieldsStatePartitionKeyExtractor, KeyStateEncoderSpec, NoopStatePartitionKeyExtractor, NoPrefixKeyStateEncoderSpec, StatePartitionKeyExtractor, StateSchemaBroadcast, StateStore, StateStoreCheckpointInfo, StateStoreColFamilySchema, StateStoreConf, StateStoreErrors, StateStoreId, StateStoreMetrics, StateStoreProvider, StateStoreProviderId, SupportsFineGrainedReplay}
import org.apache.spark.sql.types.{BooleanType, LongType, StructField, StructType}
import org.apache.spark.util.NextIterator

Expand Down Expand Up @@ -101,14 +101,14 @@ abstract class SymmetricHashJoinStateManager(
joinStoreGenerator: JoinStateManagerStoreGenerator) extends Logging {
import SymmetricHashJoinStateManager._

protected val keySchema = StructType(
private[streaming] val keySchema = StructType(
joinKeys.zipWithIndex.map { case (k, i) => StructField(s"field$i", k.dataType, k.nullable) })
protected val keyAttributes = toAttributes(keySchema)

protected val keyToNumValues = new KeyToNumValuesStore(
private[streaming] val keyToNumValues = new KeyToNumValuesStore(
stateFormatVersion,
snapshotOptions.map(_.getKeyToNumValuesHandlerOpts()))
protected val keyWithIndexToValue = new KeyWithIndexToValueStore(
private[streaming] val keyWithIndexToValue = new KeyWithIndexToValueStore(
stateFormatVersion,
snapshotOptions.map(_.getKeyWithIndexToValueHandlerOpts()))

Expand Down Expand Up @@ -1254,21 +1254,53 @@ object SymmetricHashJoinStateManager {
}
}

private[join] sealed trait StateStoreType
private[streaming] sealed trait StateStoreType

private[join] case object KeyToNumValuesType extends StateStoreType {
private[streaming] case object KeyToNumValuesType extends StateStoreType {
override def toString(): String = "keyToNumValues"
}

private[join] case object KeyWithIndexToValueType extends StateStoreType {
private[streaming] case object KeyWithIndexToValueType extends StateStoreType {
override def toString(): String = "keyWithIndexToValue"
}

private[join] def getStateStoreName(
private[streaming] def getStateStoreName(
joinSide: JoinSide, storeType: StateStoreType): String = {
s"$joinSide-$storeType"
}

private[join] def getStoreType(storeName: String): StateStoreType = {
if (storeName == getStateStoreName(LeftSide, KeyToNumValuesType) ||
storeName == getStateStoreName(RightSide, KeyToNumValuesType)) {
KeyToNumValuesType
} else if (storeName == getStateStoreName(LeftSide, KeyWithIndexToValueType) ||
storeName == getStateStoreName(RightSide, KeyWithIndexToValueType)) {
KeyWithIndexToValueType
} else {
throw new IllegalArgumentException(s"Unknown join store name: $storeName")
}
}

/**
* Returns the partition key extractor for the given join store and column family name.
*/
def createPartitionKeyExtractor(
storeName: String,
colFamilyName: String,
stateKeySchema: StructType,
stateFormatVersion: Int): StatePartitionKeyExtractor = {
assert(stateFormatVersion <= 3, "State format version must be less than or equal to 3")
val name = if (stateFormatVersion == 3) colFamilyName else storeName
if (getStoreType(name) == KeyWithIndexToValueType) {
// For KeyWithIndex, the index is added to the join (i.e. partition) key.
// Drop the last field (index) to get the partition key
new DropLastNFieldsStatePartitionKeyExtractor(stateKeySchema, numLastColsToDrop = 1)
} else {
// State key is the partition key
new NoopStatePartitionKeyExtractor(stateKeySchema)
}
}

/** Helper class for representing data (value, matched). */
case class ValueAndMatchPair(value: UnsafeRow, matched: Boolean)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -946,7 +946,7 @@ case class StateStoreSaveExec(
}
}

override def shortName: String = "stateStoreSave"
override def shortName: String = StatefulOperatorsUtils.STATE_STORE_SAVE_EXEC_OP_NAME

override def shouldRunAnotherBatch(newInputWatermark: Long): Boolean = {
(outputMode.contains(Append) || outputMode.contains(Update)) &&
Expand Down Expand Up @@ -1074,7 +1074,8 @@ case class SessionWindowStateStoreSaveExec(

override def keyExpressions: Seq[Attribute] = keyWithoutSessionExpressions

override def shortName: String = "sessionWindowStateStoreSaveExec"
override def shortName: String =
StatefulOperatorsUtils.SESSION_WINDOW_STATE_STORE_SAVE_EXEC_OP_NAME

private val stateManager = StreamingSessionWindowStateManager.createStateManager(
keyWithoutSessionExpressions, sessionExpression, child.output, stateFormatVersion)
Expand Down Expand Up @@ -1395,7 +1396,7 @@ case class StreamingDeduplicateExec(
removeKeysOlderThanWatermark(store)
}

override def shortName: String = "dedupe"
override def shortName: String = StatefulOperatorsUtils.DEDUPLICATE_EXEC_OP_NAME

override protected def withNewChildInternal(newChild: SparkPlan): StreamingDeduplicateExec =
copy(child = newChild)
Expand All @@ -1416,6 +1417,12 @@ object StreamingDeduplicateExec {
UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
}

/**
* For Deduplicate, the state key is the partition key i.e. the dedup key
*/
class StreamingDeduplicateStatePartitionKeyExtractor(stateKeySchema: StructType)
extends NoopStatePartitionKeyExtractor(stateKeySchema)

case class StreamingDeduplicateWithinWatermarkExec(
keyExpressions: Seq[Attribute],
child: SparkPlan,
Expand Down Expand Up @@ -1478,7 +1485,7 @@ case class StreamingDeduplicateWithinWatermarkExec(
}
}

override def shortName: String = "dedupeWithinWatermark"
override def shortName: String = StatefulOperatorsUtils.DEDUPLICATE_WITHIN_WATERMARK_EXEC_OP_NAME

override def validateAndMaybeEvolveStateSchema(
hadoopConf: Configuration, batchId: Long, stateSchemaVersion: Int):
Expand All @@ -1494,6 +1501,12 @@ case class StreamingDeduplicateWithinWatermarkExec(
newChild: SparkPlan): StreamingDeduplicateWithinWatermarkExec = copy(child = newChild)
}

/**
* For DeduplicateWithinWatermark, the state key is the partition key i.e. the dedup key
*/
class StreamingDedupWithinWatermarkStatePartitionKeyExtractor(stateKeySchema: StructType)
extends NoopStatePartitionKeyExtractor(stateKeySchema)

trait SchemaValidationUtils extends Logging {

// Determines whether the operator should be able to evolve their schema
Expand Down Expand Up @@ -1561,4 +1574,14 @@ object StatefulOperatorsUtils {
TRANSFORM_WITH_STATE_IN_PYSPARK_EXEC_OP_NAME
)
val SYMMETRIC_HASH_JOIN_EXEC_OP_NAME = "symmetricHashJoin"
val STATE_STORE_SAVE_EXEC_OP_NAME = "stateStoreSave"
val DEDUPLICATE_EXEC_OP_NAME = "dedupe"
val DEDUPLICATE_WITHIN_WATERMARK_EXEC_OP_NAME = "dedupeWithinWatermark"
val SESSION_WINDOW_STATE_STORE_SAVE_EXEC_OP_NAME = "sessionWindowStateStoreSaveExec"
val FLAT_MAP_GROUPS_WITH_STATE_EXEC_OP_NAME = "flatMapGroupsWithState"
val FLAT_MAP_GROUPS_IN_PANDAS_WITH_STATE_EXEC_OP_NAME = "applyInPandasWithState"
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: should we rename as APPLY_IN_PANDAS_WITH_STATE_EXEC ?

val FLAT_MAP_GROUPS_OP_NAMES: Seq[String] = Seq(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah TWS node names are already here - ok nice

FLAT_MAP_GROUPS_WITH_STATE_EXEC_OP_NAME,
FLAT_MAP_GROUPS_IN_PANDAS_WITH_STATE_EXEC_OP_NAME
)
}
Loading