zhenlineo commented on code in PR #41129:
URL: https://github.com/apache/spark/pull/41129#discussion_r1196867820
##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -216,6 +216,7 @@ message WriteStreamOperationStart {
message StreamingForeachWriter {
oneof writer {
PythonUDF python_writer = 1;
+ ScalarScalaUDF scala_writer = 2;
Review Comment:
It looks like you only needed bytes, would it be enough to only use bytes?
e.g.
```
oneof writer {
bytes python_writer = 1;
bytes scala_writer = 2;
}
```
##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala:
##########
@@ -202,6 +208,28 @@ final class DataStreamWriter[T] private[sql] (ds:
Dataset[T]) extends Logging {
this
}
+ /**
+ * Sets the output of the streaming query to be processed using the provided
writer object.
+ * object. See [[org.apache.spark.sql.ForeachWriter]] for more details on
the lifecycle and
+ * semantics.
+ * @since 3.5.0
+ */
+ def foreach(writer: ForeachWriter[T]): DataStreamWriter[T] = {
+ // ds.encoder equal to UnboundRowEncoder means type parameter T is Row,
+ // which is not able to be serialized. Server will detect this and use
default encoder.
+ val rowEncoder = if (ds.encoder != UnboundRowEncoder) {
Review Comment:
Can you add a TODO and a ticket for us
[here](https://issues.apache.org/jira/browse/SPARK-42554)
##########
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:
If you could write the logic to work based on the schema of the dataset, pls
do. The user's writer func would always expect a T, we need cast between row
and T. It would be the best if we could not unpack the scala bytes (writer +
encoder) before the execution, as the unpacking should be isolated.
However we know this issue with encoder and current impl and udfs. It is not
necessary this PR's problem to address this issue.
##########
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:
Suggest rename rowEncoder -> datasetEncoder.
DataStreamWriter SQL impl needed a dataset as input, then the dataset mostly
is used to provide the exprEncoder. This makes sense that we directly send the
encoder with the writer.
You probably do not need the other methods as long as you define this class
as `Serializable`.
--
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]