HyukjinKwon commented on code in PR #42035:
URL: https://github.com/apache/spark/pull/42035#discussion_r1267397837
##########
python/pyspark/sql/connect/session.py:
##########
@@ -654,6 +655,14 @@ def copyFromLocalToFs(self, local_path: str, dest_path:
str) -> None:
copyFromLocalToFs.__doc__ = PySparkSession.copyFromLocalToFs.__doc__
+ def _createRemoteDataFrame(self, remote_id: str) -> "DataFrame":
Review Comment:
```suggestion
def _create_remote_dataframe(self, remote_id: str) -> "DataFrame":
```
##########
python/pyspark/streaming_worker.py:
##########
@@ -0,0 +1,78 @@
+#
+# 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.
+#
+
+"""
+A worker for streaming foreachBatch and query listener in Spark Connect.
+"""
+import os
+
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+ write_int,
+ read_long,
+ UTF8Deserializer,
+ CPickleSerializer,
+)
+from pyspark import worker
+from pyspark.sql import SparkSession
+
+pickleSer = CPickleSerializer()
+utf8_deserializer = UTF8Deserializer()
+
+
+def main(infile, outfile): # type: ignore[no-untyped-def]
+ log_name = "Streaming ForeachBatch worker"
+ connect_url = os.environ["SPARK_CONNECT_LOCAL_URL"]
+ sessionId = utf8_deserializer.loads(infile)
+
+ print(f"{log_name} is starting with url {connect_url} and sessionId
{sessionId}.")
+
+ sparkConnectSession =
SparkSession.builder.remote(connect_url).getOrCreate()
Review Comment:
```suggestion
spark_connect_session =
SparkSession.builder.remote(connect_url).getOrCreate()
```
##########
core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.api.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream,
DataOutputStream}
+import java.net.Socket
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.BUFFER_SIZE
+import org.apache.spark.internal.config.Python.{PYTHON_AUTH_SOCKET_TIMEOUT,
PYTHON_USE_DAEMON}
+
+
+private[spark] object StreamingPythonRunner {
+ def apply(func: PythonFunction, connectUrl: String): StreamingPythonRunner =
{
+ new StreamingPythonRunner(func, connectUrl)
+ }
+}
+
+private[spark] class StreamingPythonRunner(func: PythonFunction, connectUrl:
String)
+ extends Logging {
+ private val conf = SparkEnv.get.conf
+ protected val bufferSize: Int = conf.get(BUFFER_SIZE)
+ protected val authSocketTimeout = conf.get(PYTHON_AUTH_SOCKET_TIMEOUT)
+
+ private val envVars: java.util.Map[String, String] = func.envVars
+ private val pythonExec: String = func.pythonExec
+ protected val pythonVer: String = func.pythonVer
+
+ /**
+ * Initializes the Python worker for streaming functions. Sets up Spark
Connect session
+ * to be used with the functions.
+ */
+ def init(sessionId: String): (DataOutputStream, DataInputStream) = {
+ log.info(s"Initializing Python runner (session: $sessionId ,pythonExec:
$pythonExec")
Review Comment:
```suggestion
logInfo(s"Initializing Python runner (session: $sessionId ,pythonExec:
$pythonExec")
```
##########
python/pyspark/streaming_worker.py:
##########
@@ -0,0 +1,78 @@
+#
+# 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.
+#
+
+"""
+A worker for streaming foreachBatch and query listener in Spark Connect.
+"""
+import os
+
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+ write_int,
+ read_long,
+ UTF8Deserializer,
+ CPickleSerializer,
+)
+from pyspark import worker
+from pyspark.sql import SparkSession
+
+pickleSer = CPickleSerializer()
+utf8_deserializer = UTF8Deserializer()
+
+
+def main(infile, outfile): # type: ignore[no-untyped-def]
+ log_name = "Streaming ForeachBatch worker"
+ connect_url = os.environ["SPARK_CONNECT_LOCAL_URL"]
+ sessionId = utf8_deserializer.loads(infile)
+
+ print(f"{log_name} is starting with url {connect_url} and sessionId
{sessionId}.")
+
+ sparkConnectSession =
SparkSession.builder.remote(connect_url).getOrCreate()
+ sparkConnectSession._client._session_id = sessionId
+
+ # TODO(SPARK-44460): Pass credentials.
+ # TODO(SPARK-44461): Enable Process Isolation
+
+ func = worker.read_command(pickleSer, infile)
Review Comment:
```suggestion
func = worker.read_command(pickle_ser, infile)
```
##########
python/pyspark/streaming_worker.py:
##########
@@ -0,0 +1,78 @@
+#
+# 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.
+#
+
+"""
+A worker for streaming foreachBatch and query listener in Spark Connect.
+"""
+import os
+
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+ write_int,
+ read_long,
+ UTF8Deserializer,
+ CPickleSerializer,
+)
+from pyspark import worker
+from pyspark.sql import SparkSession
+
+pickleSer = CPickleSerializer()
Review Comment:
```suggestion
pickle_ser = CPickleSerializer()
```
##########
python/pyspark/streaming_worker.py:
##########
@@ -0,0 +1,78 @@
+#
+# 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.
+#
+
+"""
+A worker for streaming foreachBatch and query listener in Spark Connect.
+"""
+import os
+
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+ write_int,
+ read_long,
+ UTF8Deserializer,
+ CPickleSerializer,
+)
+from pyspark import worker
+from pyspark.sql import SparkSession
+
+pickleSer = CPickleSerializer()
+utf8_deserializer = UTF8Deserializer()
+
+
+def main(infile, outfile): # type: ignore[no-untyped-def]
+ log_name = "Streaming ForeachBatch worker"
+ connect_url = os.environ["SPARK_CONNECT_LOCAL_URL"]
+ sessionId = utf8_deserializer.loads(infile)
Review Comment:
```suggestion
session_id = utf8_deserializer.loads(infile)
```
##########
python/pyspark/streaming_worker.py:
##########
@@ -0,0 +1,78 @@
+#
+# 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.
+#
+
+"""
+A worker for streaming foreachBatch and query listener in Spark Connect.
+"""
+import os
+
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+ write_int,
+ read_long,
+ UTF8Deserializer,
+ CPickleSerializer,
+)
+from pyspark import worker
+from pyspark.sql import SparkSession
+
+pickleSer = CPickleSerializer()
+utf8_deserializer = UTF8Deserializer()
+
+
+def main(infile, outfile): # type: ignore[no-untyped-def]
+ log_name = "Streaming ForeachBatch worker"
+ connect_url = os.environ["SPARK_CONNECT_LOCAL_URL"]
+ sessionId = utf8_deserializer.loads(infile)
+
+ print(f"{log_name} is starting with url {connect_url} and sessionId
{sessionId}.")
+
+ sparkConnectSession =
SparkSession.builder.remote(connect_url).getOrCreate()
+ sparkConnectSession._client._session_id = sessionId
+
+ # TODO(SPARK-44460): Pass credentials.
+ # TODO(SPARK-44461): Enable Process Isolation
+
+ func = worker.read_command(pickleSer, infile)
+ write_int(0, outfile) # Indicate successful initialization
+
+ outfile.flush()
+
+ def process(dfId, batchId): # type: ignore[no-untyped-def]
+ print(f"{log_name} Started batch {batchId} with DF id {dfId}")
+ batchDf = sparkConnectSession._createRemoteDataFrame(dfId)
+ func(batchDf, batchId)
Review Comment:
```suggestion
func(batch_df, batchId)
```
##########
python/pyspark/sql/connect/streaming/readwriter.py:
##########
@@ -495,14 +495,14 @@ def foreach(self, f: Union[Callable[[Row], None],
"SupportsProcess"]) -> "DataSt
foreach.__doc__ = PySparkDataStreamWriter.foreach.__doc__
- # TODO (SPARK-42944): Implement and uncomment the doc
def foreachBatch(self, func: Callable[["DataFrame", int], None]) ->
"DataStreamWriter":
- raise PySparkNotImplementedError(
- error_class="NOT_IMPLEMENTED",
- message_parameters={"feature": "foreachBatch()"},
+ self._write_proto.foreach_batch.python_function.command =
CloudPickleSerializer().dumps(
+ func
)
+ self._write_proto.foreach_batch.python_function.python_ver = "%d.%d" %
sys.version_info[:2]
+ return self
- # foreachBatch.__doc__ = PySparkDataStreamWriter.foreachBatch.__doc__
+ foreachBatch.__doc__ = PySparkDataStreamWriter.foreachBatch.__doc__
Review Comment:
We should fix non-Spark Connect foreachBatch docs and add a directive like:
``` .. versionchanged:: 3.4.0
Supports Spark Connect.
```
##########
python/pyspark/streaming_worker.py:
##########
@@ -0,0 +1,78 @@
+#
+# 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.
+#
+
+"""
+A worker for streaming foreachBatch and query listener in Spark Connect.
+"""
+import os
+
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+ write_int,
+ read_long,
+ UTF8Deserializer,
+ CPickleSerializer,
+)
+from pyspark import worker
+from pyspark.sql import SparkSession
+
+pickleSer = CPickleSerializer()
+utf8_deserializer = UTF8Deserializer()
+
+
+def main(infile, outfile): # type: ignore[no-untyped-def]
+ log_name = "Streaming ForeachBatch worker"
+ connect_url = os.environ["SPARK_CONNECT_LOCAL_URL"]
+ sessionId = utf8_deserializer.loads(infile)
+
+ print(f"{log_name} is starting with url {connect_url} and sessionId
{sessionId}.")
+
+ sparkConnectSession =
SparkSession.builder.remote(connect_url).getOrCreate()
+ sparkConnectSession._client._session_id = sessionId
Review Comment:
```suggestion
sparkConnectSession._client._session_id = session_id
```
##########
python/pyspark/streaming_worker.py:
##########
@@ -0,0 +1,78 @@
+#
+# 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.
+#
+
+"""
+A worker for streaming foreachBatch and query listener in Spark Connect.
+"""
+import os
+
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+ write_int,
+ read_long,
+ UTF8Deserializer,
+ CPickleSerializer,
+)
+from pyspark import worker
+from pyspark.sql import SparkSession
+
+pickleSer = CPickleSerializer()
+utf8_deserializer = UTF8Deserializer()
+
+
+def main(infile, outfile): # type: ignore[no-untyped-def]
+ log_name = "Streaming ForeachBatch worker"
+ connect_url = os.environ["SPARK_CONNECT_LOCAL_URL"]
+ sessionId = utf8_deserializer.loads(infile)
+
+ print(f"{log_name} is starting with url {connect_url} and sessionId
{sessionId}.")
+
+ sparkConnectSession =
SparkSession.builder.remote(connect_url).getOrCreate()
+ sparkConnectSession._client._session_id = sessionId
+
+ # TODO(SPARK-44460): Pass credentials.
+ # TODO(SPARK-44461): Enable Process Isolation
+
+ func = worker.read_command(pickleSer, infile)
+ write_int(0, outfile) # Indicate successful initialization
+
+ outfile.flush()
+
+ def process(dfId, batchId): # type: ignore[no-untyped-def]
+ print(f"{log_name} Started batch {batchId} with DF id {dfId}")
+ batchDf = sparkConnectSession._createRemoteDataFrame(dfId)
Review Comment:
```suggestion
batch_df = spark_connect_session._create_remote_dataframe(dfId)
```
##########
python/pyspark/sql/tests/connect/streaming/test_parity_streaming_foreachBatch.py:
##########
@@ -0,0 +1,44 @@
+#
Review Comment:
should add this into `dev/sparktestsupport/modules.py` so the tests actually
run
--
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]