zifeif2 commented on code in PR #53355:
URL: https://github.com/apache/spark/pull/53355#discussion_r2600199798


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala:
##########
@@ -587,6 +596,102 @@ class StreamingDeduplicationSuite extends 
StateStoreMetricsTest {
     }
     assert(ex.getMessage.contains("State store encoding format as avro is not 
supported"))
   }
+
+  test("Partition key extraction - Dedupe") {
+    val df = (input: Dataset[(String, Int)]) => 
input.dropDuplicates("_1").toDF()
+
+    testPartitionKeyExtraction(
+      StatefulOperatorsUtils.DEDUPLICATE_EXEC_OP_NAME,
+      inputData = Seq(("a", 1), ("b", 2), ("c", 3)),
+      dedupeDF = df,
+      // The key schema for dedup is just the _1 column (String)
+      keySchema = new StructType().add("_1", StringType),
+      // Empty value schema for dedup
+      valueSchema = new StructType(),
+      sqlConf = spark.sessionState.conf
+    )
+  }
+}
+
+trait StreamingDeduplicationSuiteBase { self: StreamTest =>
+  import testImplicits._
+
+  protected def testPartitionKeyExtraction(
+      operatorName: String,
+      inputData: Seq[(String, Int)],
+      dedupeDF: Dataset[(String, Int)] => DataFrame,
+      keySchema: StructType,
+      valueSchema: StructType,
+      sqlConf: SQLConf): Unit = {
+    withTempDir { checkpointDir =>
+      // 1 partition to make verification easier
+      val conf = Map(SQLConf.SHUFFLE_PARTITIONS.key -> "1")
+
+      val inputStream = MemoryStream[(String, Int)]
+
+      // Run streaming query to populate state
+      testStream(dedupeDF(inputStream.toDS()), Append)(
+        StartStream(checkpointLocation = checkpointDir.getAbsolutePath, 
additionalConfs = conf),
+        AddData(inputStream, inputData: _*),
+        ProcessAllAvailable(),
+        StopStream
+      )
+
+      // Now access the state store to verify partition key extraction
+      val storeConf = new StateStoreConf(sqlConf)
+      val storeId = StateStoreId(checkpointDir.getAbsolutePath + "/state", 0, 
0)
+      val storeProviderId = StateStoreProviderId(storeId, UUID.randomUUID())
+
+      val keyProjection = UnsafeProjection.create(keySchema)
+      def createExpectedKeyRow(inputKey: String) = {
+        val row = new 
GenericInternalRow(Array[Any](UTF8String.fromString(inputKey)))
+        keyProjection.apply(row).copy()
+      }
+
+      val store = StateStore.getReadOnly(
+        storeProviderId,
+        keySchema,
+        valueSchema,
+        NoPrefixKeyStateEncoderSpec(keySchema),
+        version = 1,
+        stateStoreCkptId = None,
+        stateSchemaBroadcast = None,
+        useColumnFamilies = false,
+        storeConf,
+        new Configuration
+      )
+
+      try {
+        val extractor = StatePartitionKeyExtractorFactory.create(
+          operatorName,
+          keySchema
+        )
+
+        // Verify partition key schema matches the key schema
+        assert(extractor.partitionKeySchema === keySchema,
+          "Partition key schema should match the dedup key schema")
+
+        // Get all state keys written by the dedup query
+        val stateKeys = store.iterator().map(_.key.copy()).toList
+        assert(stateKeys.length === inputData.length,
+          s"Should have ${inputData.length} unique keys, found 
${stateKeys.length}")
+
+        // Extract partition keys
+        val partitionKeys = stateKeys.map(extractor.partitionKey(_).copy())
+        // Verify each partition key equals its corresponding state key (for 
dedup)
+        assert(partitionKeys === stateKeys,
+          "Partition keys should match state keys")
+
+        // Expected keys
+        inputData.foreach { case (key, _) =>
+          val keyRow = createExpectedKeyRow(key)
+          assert(partitionKeys.count(_ === keyRow) == 1, s"Should have 1 
partition key for $key")
+        }

Review Comment:
   nit: do we still need this check after checking partitionKeys == stateKeys? 



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StatePartitionKeyExtractorFactory.scala:
##########
@@ -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)

Review Comment:
   nit: do we want to add a check to require stateVariableInfo is not null when 
operatorName is tws?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StatePartitionKeyExtractor.scala:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.state
+
+import scala.collection.immutable.ArraySeq
+
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, 
UnsafeProjection, UnsafeRow}
+import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Trait for extracting partition keys from state key rows.
+ * The partition key is the key used by shuffle.
+ * This is used for offline state repartitioning.
+ */
+trait StatePartitionKeyExtractor {
+  /**
+   * Returns the schema of the partition key.
+   */
+  def partitionKeySchema: StructType
+
+  /**
+   * Extracts the partition key row from the given state key row.
+   *
+   * @note Depending on the implementation, it might not be safe to buffer the
+   *       returned UnsafeRow across multiple calls of this method, due to 
UnsafeRow re-use.
+   *       If you are holding on to the row between multiple calls, you should 
copy the row.
+   */
+  def partitionKey(stateKeyRow: UnsafeRow): UnsafeRow
+}
+
+/**
+ * No-op state partition key extractor that returns the state key row as the 
partition key row.
+ * This is used by operators that use the partition key as the state key.
+ *
+ * @param stateKeySchema The schema of the state key row
+ */
+class NoopStatePartitionKeyExtractor(stateKeySchema: StructType)
+  extends StatePartitionKeyExtractor {
+  override lazy val partitionKeySchema: StructType = stateKeySchema
+
+  override def partitionKey(stateKeyRow: UnsafeRow): UnsafeRow = stateKeyRow
+}
+
+/**
+ * State partition key extractor that returns the field at the specified index
+ * of the state key row as the partition key row.
+ *
+ * @param stateKeySchema The schema of the state key row
+ * @param partitionKeyIndex The index of the field to extract as the partition 
key
+ */
+class IndexBasedStatePartitionKeyExtractor(stateKeySchema: StructType, 
partitionKeyIndex: Int)
+  extends StatePartitionKeyExtractor {
+  override lazy val partitionKeySchema: StructType =
+    stateKeySchema.fields(partitionKeyIndex).dataType.asInstanceOf[StructType]
+
+  override def partitionKey(stateKeyRow: UnsafeRow): UnsafeRow = {
+    stateKeyRow.getStruct(partitionKeyIndex, partitionKeySchema.length)
+  }
+}
+
+/**
+ * State partition key extractor that drops the last N fields of the state key 
row
+ * and returns the remaining fields as the partition key row.
+ *
+ * @param stateKeySchema The schema of the state key row
+ * @param numLastColsToDrop The number of last columns to drop in the state key
+ */
+class DropLastNFieldsStatePartitionKeyExtractor(stateKeySchema: StructType, 
numLastColsToDrop: Int)
+  extends StatePartitionKeyExtractor {
+  override lazy val partitionKeySchema: StructType =
+    StructType(stateKeySchema.dropRight(numLastColsToDrop))

Review Comment:
   Do we want to check if numLastColsToDrop < stateKeySchema.length here? o.w. 
it might return empty schema without throwing any error.



-- 
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]

Reply via email to