tdas commented on a change in pull request #33093:
URL: https://github.com/apache/spark/pull/33093#discussion_r661569762



##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala
##########
@@ -178,6 +257,47 @@ case class FlatMapGroupsWithStateExec(
       }
     }
 
+    /**
+     * Process the new data iterator along with the initial state. The initial 
state is applied
+     * before processing the new data for every key. The user defined function 
is called only
+     * once on the data.
+     */
+    def processNewDataWithInitState(
+        childDataIter: Iterator[InternalRow],
+        initStateIter: Iterator[InternalRow]
+      ): Iterator[InternalRow] = {
+
+      if (!childDataIter.hasNext && !initStateIter.hasNext) return 
Iterator.empty
+
+      val groupedChildDataIter = GroupedIterator(childDataIter, 
groupingAttributes, child.output)
+      val groupedInitStateIter =
+        GroupedIterator(initStateIter, initStateGroupAttrs, 
initialState.output)
+
+      val keyOrderingComparator = GenerateOrdering.generate(
+        groupingAttributes.map(SortOrder(_, Ascending)), groupingAttributes)

Review comment:
       this object `groupingAttributes.map(SortOrder(_, Ascending))` must be 
reused between here and the `requiredChildOrdering`.  the code has to ensure 
that the child ordering done by the SparkPlan and the comparator being used for 
this merging ... is based on the same sorting strategy.




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