You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "HyukjinKwon (via GitHub)" <gi...@apache.org> on 2023/05/02 11:15:32 UTC

[GitHub] [spark] HyukjinKwon commented on a diff in pull request #40586: [SPARK-42939][SS][CONNECT] Core streaming Python API for Spark Connect

HyukjinKwon commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1182411427


##########
python/pyspark/sql/connect/dataframe.py:
##########
@@ -1685,8 +1710,12 @@ def mapInArrow(
 
     mapInArrow.__doc__ = PySparkDataFrame.mapInArrow.__doc__
 
-    def writeStream(self, *args: Any, **kwargs: Any) -> None:
-        raise NotImplementedError("writeStream() is not implemented.")
+    @property
+    def writeStream(self) -> DataStreamWriter:
+        assert self._plan is not None
+        return DataStreamWriter(plan=self._plan, session=self._session)
+
+    writeStream.__doc__ = PySparkDataFrame.writeStream.__doc__

Review Comment:
   Hm, this seems quite flaky:
   
   ```
   File "/__w/spark/spark/python/pyspark/sql/connect/dataframe.py", line ?, in pyspark.sql.connect.dataframe.DataFrame.writeStream
   Failed example:
       with tempfile.TemporaryDirectory() as d:
           # Create a table with Rate source.
           df.writeStream.toTable(
               "my_table", checkpointLocation=d)
   Exception raised:
       Traceback (most recent call last):
         File "/usr/lib/python3.9/doctest.py", line 1336, in __run
           exec(compile(example.source, filename, "single",
         File "<doctest pyspark.sql.connect.dataframe.DataFrame.writeStream[3]>", line 3, in <module>
           df.writeStream.toTable(
         File "/usr/lib/python3.9/tempfile.py", line 965, in __exit__
           self.cleanup()
         File "/usr/lib/python3.9/tempfile.py", line 969, in cleanup
           self._rmtree(self.name)
         File "/usr/lib/python3.9/tempfile.py", line 951, in _rmtree
           _rmtree(name, onerror=onerror)
         File "/usr/lib/python3.9/shutil.py", line 722, in rmtree
           onerror(os.rmdir, path, sys.exc_info())
         File "/usr/lib/python3.9/shutil.py", line 720, in rmtree
           os.rmdir(path)
       OSError: [Errno 39] Directory not empty: '/__w/spark/spark/python/target/b03d461e-84a3-4dfe-bf25-4d813d4da3a3/tmp48azcvu_'
   **********************************************************************
      1 of   4 in pyspark.sql.connect.dataframe.DataFrame.writeStream
   ```
   
   https://github.com/apache/spark/actions/runs/4858380330/jobs/8659766804



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