rangadi commented on code in PR #41129:
URL: https://github.com/apache/spark/pull/41129#discussion_r1196819312


##########
connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/foreachWriterPacket.scala:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.connect.common
+
+import com.google.protobuf.ByteString
+import java.io.{InputStream, ObjectInputStream, ObjectOutputStream, 
OutputStream}
+
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoder
+
+/**
+ * A wrapper class around the foreachWriter and it's Input/Output 
[[AgnosticEncoder]](s).
+ *
+ * This class is shared between the client and the server to allow for 
serialization and
+ * deserialization of the JVM object.
+ *
+ * @param foreachWriter
+ *   The actual foreachWriter from client
+ * @param rowEncoder
+ *   An [[AgnosticEncoder]] for the input row
+ */
+@SerialVersionUID(3882541391565582579L)
+case class foreachWriterPacket(foreachWriter: AnyRef, rowEncoder: 
AgnosticEncoder[_])

Review Comment:
   Why is this required? It is not not doing anything other than default 
serialization. 



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -2386,10 +2393,26 @@ class SparkConnectPlanner(val session: SparkSession) {
     }
 
     if (writeOp.hasForeachWriter) {
-      val foreach = writeOp.getForeachWriter.getPythonWriter
-      val pythonFcn = transformPythonFunction(foreach)
-      writer.foreachImplementation(
-        new PythonForeachWriter(pythonFcn, 
dataset.schema).asInstanceOf[ForeachWriter[Any]])
+      if (writeOp.getForeachWriter.hasPythonWriter) {
+        val foreach = writeOp.getForeachWriter.getPythonWriter
+        val pythonFcn = transformPythonFunction(foreach)
+        writer.foreachImplementation(
+          new PythonForeachWriter(pythonFcn, 
dataset.schema).asInstanceOf[ForeachWriter[Any]])
+      } else {
+        val foreachWriterPkt = 
unpackForeachWriter(writeOp.getForeachWriter.getScalaWriter)
+        val clientWriter = foreachWriterPkt.foreachWriter
+        if (foreachWriterPkt.rowEncoder == null) {
+          // rowEncoder is null means the client-side writer has type 
parameter Row,
+          // Since server-side dataset is always dataframe, here just use 
foreach directly.
+          writer.foreach(clientWriter.asInstanceOf[ForeachWriter[Row]])
+        } else {
+          val encoder = ExpressionEncoder(
+            foreachWriterPkt.rowEncoder.asInstanceOf[AgnosticEncoder[Any]])

Review Comment:
   @hvanhovell is this safe? We are deserializing arbitrary Encoder class from 
user. Would it need user's jars? Otherwise, it could be a security issue.



##########
connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala:
##########
@@ -140,10 +140,88 @@ class StreamingQuerySuite extends RemoteSparkSession with 
SQLHelper {
       assert(!terminated)
 
       q.stop()
-      // TODO (SPARK-43032): uncomment below
-      // eventually(timeout(1.minute)) {
-      // q.awaitTermination()
-      // }
+      eventually(timeout(1.minute)) {
+        q.awaitTermination()
+      }
+    }
+  }
+
+  test("foreach Row") {
+    withTempPath { f =>
+      val path = f.getCanonicalPath + "/output"
+      val writer = new ForeachWriter[Row] {
+        var fileWriter: FileWriter = _
+
+        def open(partitionId: Long, version: Long): Boolean = {
+          fileWriter = new FileWriter(path, true)
+          true
+        }
+
+        def process(row: Row): Unit = {
+          fileWriter.write(row.mkString(", "))
+          fileWriter.write("\n")
+        }
+
+        def close(errorOrNull: Throwable): Unit = {
+          fileWriter.close()
+        }
+      }

Review Comment:
   Scala improvement:
   Define this implementation as `MyForeachWriter[T]` and use it both tests. 
`MyForeachWriter[Row]` and `MyForeachWriter[Int]`. 



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -2386,10 +2393,26 @@ class SparkConnectPlanner(val session: SparkSession) {
     }
 
     if (writeOp.hasForeachWriter) {
-      val foreach = writeOp.getForeachWriter.getPythonWriter
-      val pythonFcn = transformPythonFunction(foreach)
-      writer.foreachImplementation(
-        new PythonForeachWriter(pythonFcn, 
dataset.schema).asInstanceOf[ForeachWriter[Any]])
+      if (writeOp.getForeachWriter.hasPythonWriter) {
+        val foreach = writeOp.getForeachWriter.getPythonWriter
+        val pythonFcn = transformPythonFunction(foreach)
+        writer.foreachImplementation(
+          new PythonForeachWriter(pythonFcn, 
dataset.schema).asInstanceOf[ForeachWriter[Any]])
+      } else {
+        val foreachWriterPkt = 
unpackForeachWriter(writeOp.getForeachWriter.getScalaWriter)
+        val clientWriter = foreachWriterPkt.foreachWriter
+        if (foreachWriterPkt.rowEncoder == null) {
+          // rowEncoder is null means the client-side writer has type 
parameter Row,
+          // Since server-side dataset is always dataframe, here just use 
foreach directly.
+          writer.foreach(clientWriter.asInstanceOf[ForeachWriter[Row]])
+        } else {
+          val encoder = ExpressionEncoder(
+            foreachWriterPkt.rowEncoder.asInstanceOf[AgnosticEncoder[Any]])

Review Comment:
   Should we instead always convert it to row encoder based on schema? 



##########
connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala:
##########
@@ -140,10 +140,88 @@ class StreamingQuerySuite extends RemoteSparkSession with 
SQLHelper {
       assert(!terminated)
 
       q.stop()
-      // TODO (SPARK-43032): uncomment below
-      // eventually(timeout(1.minute)) {
-      // q.awaitTermination()
-      // }
+      eventually(timeout(1.minute)) {
+        q.awaitTermination()
+      }
+    }
+  }
+
+  test("foreach Row") {

Review Comment:
   Could you add one more test with custom type? Say `case class Person(name: 
String, id: Int)`. 



-- 
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-unsubscr...@spark.apache.org

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


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

Reply via email to