You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "anishshri-db (via GitHub)" <gi...@apache.org> on 2024/02/01 01:47:51 UTC

Re: [PR] [SPARK-46865][SS] Add Batch Support for TransformWithState Operator [spark]

anishshri-db commented on code in PR #44884:
URL: https://github.com/apache/spark/pull/44884#discussion_r1473701291


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala:
##########
@@ -152,22 +159,104 @@ case class TransformWithStateExec(
   override protected def doExecute(): RDD[InternalRow] = {
     metrics // force lazy init at driver
 
-    child.execute().mapPartitionsWithStateStore[InternalRow](
-      getStateInfo,
-      schemaForKeyRow,
-      schemaForValueRow,
-      numColsPrefixKey = 0,
-      session.sqlContext.sessionState,
-      Some(session.sqlContext.streams.stateStoreCoordinator),
-      useColumnFamilies = true
-    ) {
-      case (store: StateStore, singleIterator: Iterator[InternalRow]) =>
-        val processorHandle = new StatefulProcessorHandleImpl(store, getStateInfo.queryRunId)
-        assert(processorHandle.getHandleState == StatefulProcessorHandleState.CREATED)
-        statefulProcessor.init(processorHandle, outputMode)
-        processorHandle.setHandleState(StatefulProcessorHandleState.INITIALIZED)
-        val result = processDataWithPartition(singleIterator, store, processorHandle)
-        result
+    if (isStreaming) {
+      child.execute().mapPartitionsWithStateStore[InternalRow](
+        getStateInfo,
+        schemaForKeyRow,
+        schemaForValueRow,
+        numColsPrefixKey = 0,
+        session.sqlContext.sessionState,
+        Some(session.sqlContext.streams.stateStoreCoordinator),
+        useColumnFamilies = true
+      ) {
+        case (store: StateStore, singleIterator: Iterator[InternalRow]) =>
+          processData(store, singleIterator)
+      }
+    } else {
+      // If the query is running in batch mode, we need to create a new StateStore and instantiate
+      // a temp directory on the executors in mapPartitionsWithIndex.
+      child.execute().mapPartitionsWithIndex[InternalRow](
+        (i, iter) => {
+          val providerId = new StateStoreProviderId(
+            StateStoreId(Utils.createTempDir().getAbsolutePath,

Review Comment:
   Yea in this case - the tmp dir creation has to be within the `mapPartitions` block. @ericm-db - did you try passing an empty checkpoint loc ? does that work ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org