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

[GitHub] [spark] rangadi opened a new pull request, #40586: [SPARK-42939] Core streaming Python API for Spark Connect

rangadi opened a new pull request, #40586:
URL: https://github.com/apache/spark/pull/40586

   ### What changes were proposed in this pull request?
   This adds core streaming API support for Spark Connect. With this, we can run majority of streaming queries. All the sources and syncs are supported. Most of the aggregations are supported. 
   Examples of features that are not yet supported: APIs that run user codes like streaming listener, `foreachBatch()` API etc. 
   
   The remaining missing APIs will be added soon. 
   
   How to try it in local mode (`./bin/pyspark --remote "local[*]"`):
   
   ```
   >>> 
   >>> query = ( 
   ...   spark
   ...     .readStream
   ...     .format("rate")
   ...     .option("numPartitions", "1")
   ...     .load()
   ...     .withWatermark("timestamp", "1 minute")
   ...     .groupBy(window("timestamp", "10 seconds"))
   ...     .count() # count for each 10 sedonds.
   ...     .writeStream
   ...     .format("memory")
   ...     .queryName("rate_table")
   ...     .trigger(processingTime="10 seconds")
   ...     .start()
   ... )
   >>>
   >>> query.isActive
   True
   >>> 
   >>> >>> spark.sql("select window.start, count from rate_table").show()
   +-------------------+-----+
   |              start|count|
   +-------------------+-----+
   |2023-03-11 22:45:40|    6|
   |2023-03-11 22:45:50|   10|
   +-------------------+-----+
   ```
   
   
   
   ### Does this PR introduce _any_ user-facing change?
   
   This is needed to run streaming queries over Spark Connect. 
   
   ### How was this patch tested?
    - Manually tested. 
    - We will enable most of streaming python tests in follow up PRs.
    - 


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


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

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1152436270


##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,97 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+message WriteStreamOperation {

Review Comment:
   What is missing? '(Optional)' or documentation? 



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


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

Posted by "amaliujia (via GitHub)" <gi...@apache.org>.
amaliujia commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1152441819


##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,97 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+message WriteStreamOperation {

Review Comment:
   Both? I saw many of the proto fields has no comment at all?



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


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

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1157597665


##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,126 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+// Starts write stream operation as streaming query. Query ID and Run ID of the streaming
+// query are returned.
+message WriteStreamOperationStart {
+
+  // (Required) The output of the `input` streaming relation will be written.
+  Relation input = 1;
+
+  // The following fields directly map to API for DataStreamWriter().
+  // Consult API documentation unless explicitly documented here.
+
+  string format = 2;
+  map<string, string> options = 3;
+  repeated string partitioning_column_names = 4;
+
+  oneof trigger {
+    string processing_time_interval = 5;
+    bool available_now = 6;
+    bool once = 7;
+    string continuous_checkpoint_interval = 8;
+  }
+
+  string output_mode = 9;
+  string query_name = 10;
+
+  // The destination is optional. When set, it can be a path or a table name.
+  oneof sink_destination {
+    string path = 11;
+    string table_name = 12;
+  }
+}
+
+message WriteStreamOperationStartResult {
+
+  // (Required) Query instance. See `StreamingQueryInstanceId`.
+  StreamingQueryInstanceId query_id = 1;
+
+  // An optional query name.
+  string name = 2;
+
+  // TODO: How do we indicate errors?
+  // TODO: Consider adding status, last progress etc here.
+}
+
+// A tuple that uniquely identifies an instance of streaming query run. It consists of `id` that
+// persists across the streaming runs and `run_id` that changes between each run of the
+// streaming query that resumes from the checkpoint.
+message StreamingQueryInstanceId {
+

Review Comment:
   FYI: @grundprinzip @amaliujia : Added StreamingQueryInstanceId that uniquely identifies an instance of streaming query run. Added full description of the fields (ported from Scala doc).



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


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

Posted by "WweiL (via GitHub)" <gi...@apache.org>.
WweiL commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1152323338


##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -0,0 +1,173 @@
+#
+# 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.
+#
+
+import json
+import sys
+from typing import TYPE_CHECKING, Any, cast, Dict, List, Optional
+
+from pyspark.errors import StreamingQueryException
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.streaming.query import (
+    StreamingQuery as PySparkStreamingQuery,
+)
+
+__all__ = [
+    "StreamingQuery",  # TODO(WIP): "StreamingQueryManager"
+]
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+
+class StreamingQuery:
+
+    def __init__(self, session: "SparkSession", queryId: str, runId: str, name: Optional[str] = None) -> None:
+        self._session = session
+        self._id = queryId
+        self._runId = runId
+        self._name = name
+
+    @property
+    def id(self) -> str:
+        return self._id
+
+    id.__doc__ = PySparkStreamingQuery.id.__doc__
+
+    @property
+    def runId(self) -> str:
+        return self._runId
+
+    runId.__doc__ = PySparkStreamingQuery.runId.__doc__
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    name.__doc__ = PySparkStreamingQuery.name.__doc__
+
+    @property
+    def isActive(self) -> bool:
+        return self._fetch_status().is_active
+
+    isActive.__doc__ = PySparkStreamingQuery.isActive.__doc__
+
+    def awaitTermination(self, timeout: Optional[int] = None) -> Optional[bool]:
+        raise NotImplementedError()
+
+    awaitTermination.__doc__ = PySparkStreamingQuery.awaitTermination.__doc__
+
+    @property
+    def status(self) -> Dict[str, Any]:
+        proto = self._fetch_status()
+        return {
+            "message": proto.status_message,
+            "isDataAvailable": proto.is_data_available,
+            "isTriggerActive": proto.is_trigger_active
+        }
+
+    status.__doc__ = PySparkStreamingQuery.status.__doc__
+
+    @property
+    def recentProgress(self) -> List[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=-1).recent_progress_json)
+        return [json.loads(p) for p in progress]
+
+    recentProgress.__doc__ = PySparkStreamingQuery.recentProgress.__doc__
+
+    @property
+    def lastProgress(self) -> Optional[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=1).recent_progress_json)
+        if len(progress) > 0:
+            return json.loads(progress[-1])
+        else:
+            return None
+
+    lastProgress.__doc__ = PySparkStreamingQuery.lastProgress.__doc__
+
+    def processAllAvailable(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.process_all_available = True
+        self._execute_streaming_query_cmd(cmd)
+
+    processAllAvailable.__doc__ = PySparkStreamingQuery.processAllAvailable.__doc__
+
+    def stop(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.stop = True
+        self._execute_streaming_query_cmd(cmd)
+
+    stop.__doc__ = PySparkStreamingQuery.stop.__doc__
+
+    def explain(self, extended: bool = False) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.explain.extended = extended
+        result = self._execute_streaming_query_cmd(cmd).explain.result

Review Comment:
   currently if we call stop() first and then do explain() or lastProgress(), it throws exception because server would do so. I was thinking if we could cache those results. And if query is stopped, just return cached results and dont contact server



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


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

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1153904387


##########
python/pyspark/sql/connect/readwriter.py:
##########
@@ -37,7 +37,7 @@
     from pyspark.sql.connect._typing import ColumnOrName, OptionalPrimitiveType
     from pyspark.sql.connect.session import SparkSession
 
-__all__ = ["DataFrameReader", "DataFrameWriter"]
+__all__ = ["DataFrameReader", "DataFrameWriter", "OptionUtils", "to_str"]

Review Comment:
   Both are used in streaming/readerwriter.py for similar purposes as in this file. 



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


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

Posted by "grundprinzip (via GitHub)" <gi...@apache.org>.
grundprinzip commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1152826039


##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,97 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+message WriteStreamOperation {
+
+  // (Required) The output of the `input` streaming relation will be written.
+  Relation input = 1;
+
+  string format = 2;
+  map<string, string> options = 3;
+  repeated string partitioning_column_names = 4;
+
+  oneof trigger {
+    string processing_time_interval = 5;
+    bool available_now = 6;
+    bool one_time = 7;
+    string continuous_checkpoint_interval = 8;
+  }
+
+  string output_mode = 9;

Review Comment:
   this sounds like an enum?



##########
connector/connect/common/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -729,6 +736,18 @@ message WithColumns {
   repeated Expression.Alias aliases = 2;
 }
 
+message WithWatermark {
+
+  // (Required) The input relation
+  Relation input = 1;
+
+  // (Required)
+  string event_time = 2;

Review Comment:
   string or long?



##########
connector/connect/common/src/main/protobuf/spark/connect/base.proto:
##########
@@ -298,6 +298,10 @@ message ExecutePlanResponse {
     // Special case for executing SQL commands.
     SqlCommandResult sql_command_result = 5;
 
+    StreamingQueryStartResult streaming_query_start_result = 8;
+
+    StreamingQueryCommandResult streaming_query_command_result = 9;

Review Comment:
   Please provide some basic doc.



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -762,22 +779,32 @@ class SparkConnectPlanner(val session: SparkSession) {
   }
 
   private def transformReadRel(rel: proto.Read): LogicalPlan = {
+
+    def parseSchema(schema: String): StructType = {
+      DataType.parseTypeWithFallback(
+        schema,
+        StructType.fromDDL,
+        fallbackParser = DataType.fromJson) match {
+        case s: StructType => s
+        case other => throw InvalidPlanInput(s"Invalid schema $other")
+      }
+    }

Review Comment:
   When you're moving it the couple of lines up, shouldn't we either make this a proper private method or add this as a helper method to `StructType`?



##########
connector/connect/common/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -729,6 +736,18 @@ message WithColumns {
   repeated Expression.Alias aliases = 2;
 }
 
+message WithWatermark {
+
+  // (Required) The input relation
+  Relation input = 1;
+
+  // (Required)
+  string event_time = 2;
+
+  // (Required)
+  string delay_threshold = 3;

Review Comment:
   sounds like a numeric?



##########
python/pyspark/sql/connect/session.py:
##########
@@ -14,6 +14,7 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
+

Review Comment:
   nit: rm



##########
python/pyspark/sql/connect/streaming/__init__.py:
##########
@@ -0,0 +1,21 @@
+#
+# 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.
+#
+
+from pyspark.sql.connect.streaming.query import StreamingQuery  # StreamingQueryManager  noqa: F401
+from pyspark.sql.connect.streaming.readwriter import DataStreamReader, DataStreamWriter  # noqa: F401
+# from pyspark.sql.connect.streaming.listener import StreamingQueryListener  # noqa: F401

Review Comment:
   ?



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -1969,6 +2014,135 @@ class SparkConnectPlanner(val session: SparkSession) {
     }
   }
 
+  def handleWriteStreamOperation(
+      writeOp: WriteStreamOperation,
+      sessionId: String,
+      responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
+    val plan = transformRelation(writeOp.getInput)
+    val dataset = Dataset.ofRows(session, logicalPlan = plan)
+
+    val writer = dataset.writeStream
+
+    if (writeOp.getFormat.nonEmpty) {
+      writer.format(writeOp.getFormat)
+    }
+
+    writer.options(writeOp.getOptionsMap)
+
+    if (writeOp.getPartitioningColumnNamesCount > 0) {
+      writer.partitionBy(writeOp.getPartitioningColumnNamesList.asScala.toList: _*)
+    }
+
+    writeOp.getTriggerCase match {
+      case TriggerCase.PROCESSING_TIME_INTERVAL =>
+        writer.trigger(Trigger.ProcessingTime(writeOp.getProcessingTimeInterval))
+      case TriggerCase.AVAILABLE_NOW =>
+        writer.trigger(Trigger.AvailableNow())
+      case TriggerCase.ONE_TIME =>
+        writer.trigger(Trigger.Once())
+      case TriggerCase.CONTINUOUS_CHECKPOINT_INTERVAL =>
+        writer.trigger(Trigger.Continuous(writeOp.getContinuousCheckpointInterval))
+      case TriggerCase.TRIGGER_NOT_SET =>
+    }
+
+    if (writeOp.getOutputMode.nonEmpty) {
+      writer.outputMode(writeOp.getOutputMode)
+    }
+
+    if (writeOp.getQueryName.nonEmpty) {
+      writer.queryName(writeOp.getQueryName)
+    }
+
+    val query = writeOp.getPath match {
+      case "" if writeOp.hasTableName => writer.toTable(writeOp.getTableName)
+      case "" => writer.start()
+      case path => writer.start(path)
+    }
+
+    val result = StreamingQueryStartResult
+      .newBuilder()
+      .setId(query.id.toString)
+      .setRunId(query.runId.toString)
+      .setName(Option(query.name).getOrElse(""))
+      .build()
+
+    responseObserver.onNext(
+      ExecutePlanResponse
+        .newBuilder()
+        .setSessionId(sessionId)
+        .setStreamingQueryStartResult(result)
+        .build())
+  }
+
+  def handleStreamingQueryCommand(
+      command: StreamingQueryCommand,
+      sessionId: String,
+      responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
+
+    val id = command.getId
+
+    val respBuilder = StreamingQueryCommandResult
+      .newBuilder()
+      .setId(command.getId)
+
+    val query = Option(session.streams.get(command.getId)).getOrElse {
+      throw new IllegalArgumentException(s"Streaming query $id is not found")
+      // TODO(SPARK-42962): Handle this better. May be cache stopped queries for a few minutes.
+    }
+
+    command.getCommandTypeCase match {
+      case StreamingQueryCommand.CommandTypeCase.STATUS =>
+        val recentProgress: Seq[String] = command.getStatus.getRecentProgressLimit match {
+          case 0 => Seq.empty
+          case limit if limit < 0 =>
+            query.recentProgress.map(_.json) // All the cached progresses.
+          case limit => query.recentProgress.takeRight(limit).map(_.json) // Most recent
+        }
+
+        val queryStatus = query.status
+
+        val statusResult = StreamingQueryCommandResult.StatusResult
+          .newBuilder()
+          .setStatusMessage(queryStatus.message)
+          .setIsDataAvailable(queryStatus.isDataAvailable)
+          .setIsTriggerActive(queryStatus.isTriggerActive)
+          .setIsActive(query.isActive)
+          .addAllRecentProgressJson(recentProgress.asJava)
+          .build()
+
+        respBuilder.setStatus(statusResult)
+
+      case StreamingQueryCommand.CommandTypeCase.STOP =>
+        query.stop()
+
+      case StreamingQueryCommand.CommandTypeCase.PROCESS_ALL_AVAILABLE =>
+        query.processAllAvailable()

Review Comment:
   for these two cases the we don't add anything to the response?



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -1681,6 +1722,10 @@ class SparkConnectPlanner(val session: SparkSession) {
         handleCommandPlugin(command.getExtension)
       case proto.Command.CommandTypeCase.SQL_COMMAND =>
         handleSqlCommand(command.getSqlCommand, sessionId, responseObserver)
+      case proto.Command.CommandTypeCase.WRITE_STREAM_OPERATION =>
+        handleWriteStreamOperation(command.getWriteStreamOperation, sessionId, responseObserver)
+      case proto.Command.CommandTypeCase.STREAMING_QUERY_COMMAND =>
+        handleStreamingQueryCommand(command.getStreamingQueryCommand, sessionId, responseObserver)

Review Comment:
   Luckily it pays of that I had to plumb through the responseObserver to the command handler :)



##########
python/pyspark/sql/connect/readwriter.py:
##########
@@ -37,7 +37,7 @@
     from pyspark.sql.connect._typing import ColumnOrName, OptionalPrimitiveType
     from pyspark.sql.connect.session import SparkSession
 
-__all__ = ["DataFrameReader", "DataFrameWriter"]
+__all__ = ["DataFrameReader", "DataFrameWriter", "OptionUtils", "to_str"]

Review Comment:
   Are the changes here really needed?



##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,97 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+message WriteStreamOperation {

Review Comment:
   Optional is always implied by proto. However, the benefit of declaring fields as explicitly optional generates code that lets you easily check `hasField`. This improves the overall readability of the code.
   
   In addition, it lets you easily distinguish for integral types between unset and default values. 
   
   It would be great if you can be a good citizen and improve the documentation for your proto messages. Feel free to do this in the last round of the review when the protos are finalized.



##########
python/pyspark/sql/connect/session.py:
##########
@@ -489,10 +495,6 @@ def sparkContext(self) -> Any:
     def streams(self) -> Any:
         raise NotImplementedError("streams() is not implemented.")
 
-    @property
-    def readStream(self) -> Any:
-        raise NotImplementedError("readStream() is not implemented.")
-

Review Comment:
   why the move?



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


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

Posted by "amaliujia (via GitHub)" <gi...@apache.org>.
amaliujia commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1156209521


##########
connector/connect/common/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -154,6 +158,9 @@ message Read {
     //
     // This is only supported by the JDBC data source.
     repeated string predicates = 5;
+
+    // (Optional) Source table name for a streaming read. Not used in batch read.
+    string streaming_table_name = 6;

Review Comment:
   https://github.com/apache/spark/blob/d3338c6879fd3e0b986654889f2e1e6407988dcc/connector/connect/common/src/main/protobuf/spark/connect/relations.proto#L122
   
   This is how batch read a table. 



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


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

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1152452932


##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,97 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+message WriteStreamOperation {

Review Comment:
   `base.proto` has so many fields without 'Optional' or comment? 



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


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

Posted by "WweiL (via GitHub)" <gi...@apache.org>.
WweiL commented on PR #40586:
URL: https://github.com/apache/spark/pull/40586#issuecomment-1492687197

   I have less expertises in protobuf, otherwise LGTM


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


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

Posted by "amaliujia (via GitHub)" <gi...@apache.org>.
amaliujia commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1156212757


##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,118 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+// Starts write stream operation as streaming query. Query ID and Run ID of the streaming
+// query are returned.
+message WriteStreamOperationStart {
+
+  // (Required) The output of the `input` streaming relation will be written.
+  Relation input = 1;
+
+  // The following fields directly map to API for DataStreamWriter().
+  // Consult API documentation unless explicitly documented here.
+
+  string format = 2;
+  map<string, string> options = 3;
+  repeated string partitioning_column_names = 4;
+
+  oneof trigger {
+    string processing_time_interval = 5;
+    bool available_now = 6;
+    bool one_time = 7;
+    string continuous_checkpoint_interval = 8;
+  }
+
+  string output_mode = 9;
+  string query_name = 10;
+
+  // The destination is optional. When set, it can be a path or a table name.
+  oneof sink_destination {
+    string path = 11;
+    string table_name = 12;
+  }
+}
+
+message WriteStreamOperationStartResult {
+
+  // (Required)
+  string query_id = 1;

Review Comment:
   I still think we should add more documentation here. Proto is an API and developers read it. Right now I do not know what is a `query_id` and a `run_id` when reading these.
   
   I think a good example is what Beam document their proto: https://github.com/apache/beam/blob/master/model/fn-execution/src/main/proto/org/apache/beam/model/fn_execution/v1/beam_fn_api.proto



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


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

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1157037868


##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -0,0 +1,181 @@
+#
+# 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.
+#
+
+import json
+import sys
+from typing import TYPE_CHECKING, Any, cast, Dict, List, Optional
+
+from pyspark.errors import StreamingQueryException
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.streaming.query import (
+    StreamingQuery as PySparkStreamingQuery,
+)
+
+__all__ = [
+    "StreamingQuery",  # TODO(WIP): "StreamingQueryManager"
+]
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+
+
+class StreamingQuery:
+    def __init__(
+        self, session: "SparkSession", queryId: str, runId: str, name: Optional[str] = None
+    ) -> None:
+        self._session = session
+        self._query_id = queryId
+        self._run_id = runId
+        self._name = name
+
+    @property
+    def id(self) -> str:
+        return self._query_id
+
+    id.__doc__ = PySparkStreamingQuery.id.__doc__
+
+    @property
+    def runId(self) -> str:
+        return self._run_id
+
+    runId.__doc__ = PySparkStreamingQuery.runId.__doc__
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    name.__doc__ = PySparkStreamingQuery.name.__doc__
+
+    @property
+    def isActive(self) -> bool:
+        return self._fetch_status().is_active
+
+    isActive.__doc__ = PySparkStreamingQuery.isActive.__doc__
+
+    def awaitTermination(self, timeout: Optional[int] = None) -> Optional[bool]:
+        raise NotImplementedError()
+
+    awaitTermination.__doc__ = PySparkStreamingQuery.awaitTermination.__doc__
+
+    @property
+    def status(self) -> Dict[str, Any]:
+        proto = self._fetch_status()
+        return {
+            "message": proto.status_message,
+            "isDataAvailable": proto.is_data_available,
+            "isTriggerActive": proto.is_trigger_active,
+        }
+
+    status.__doc__ = PySparkStreamingQuery.status.__doc__
+
+    @property
+    def recentProgress(self) -> List[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=10000).recent_progress_json)
+        # Server only keeps 100, so 10000 limit is high enough.
+        return [json.loads(p) for p in progress]
+
+    recentProgress.__doc__ = PySparkStreamingQuery.recentProgress.__doc__
+
+    @property
+    def lastProgress(self) -> Optional[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=1).recent_progress_json)
+        if len(progress) > 0:
+            return json.loads(progress[-1])
+        else:
+            return None
+
+    lastProgress.__doc__ = PySparkStreamingQuery.lastProgress.__doc__
+
+    def processAllAvailable(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.process_all_available = True
+        self._execute_streaming_query_cmd(cmd)
+
+    processAllAvailable.__doc__ = PySparkStreamingQuery.processAllAvailable.__doc__
+
+    def stop(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.stop = True
+        self._execute_streaming_query_cmd(cmd)
+
+    stop.__doc__ = PySparkStreamingQuery.stop.__doc__
+
+    def explain(self, extended: bool = False) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.explain.extended = extended
+        result = self._execute_streaming_query_cmd(cmd).explain.result
+        print(result)
+
+    explain.__doc__ = PySparkStreamingQuery.explain.__doc__
+
+    def exception(self) -> Optional[StreamingQueryException]:
+        raise NotImplementedError()
+
+    exception.__doc__ = PySparkStreamingQuery.exception.__doc__
+
+    def _fetch_status(
+        self, recent_progress_limit=0
+    ) -> pb2.StreamingQueryCommandResult.StatusResult:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.status.recent_progress_limit = recent_progress_limit
+
+        return self._execute_streaming_query_cmd(cmd).status
+
+    def _execute_streaming_query_cmd(
+        self, cmd: pb2.StreamingQueryCommand
+    ) -> pb2.StreamingQueryCommandResult:
+        cmd.query_id = self._query_id
+        cmd.run_id = self._run_id
+        exec_cmd = pb2.Command()
+        exec_cmd.streaming_query_command.CopyFrom(cmd)
+        (_, properties) = self._session.client.execute_command(exec_cmd)
+        return cast(pb2.StreamingQueryCommandResult, properties["streaming_query_command_result"])
+
+
+# TODO(WIP) class StreamingQueryManager:
+
+
+def _test() -> None:
+    import doctest
+    import os
+    from pyspark.sql import SparkSession
+    import pyspark.sql.streaming.query
+    from py4j.protocol import Py4JError
+
+    os.chdir(os.environ["SPARK_HOME"])
+
+    globs = pyspark.sql.streaming.query.__dict__.copy()

Review Comment:
   ```suggestion
       globs = pyspark.sql.connect.streaming.query.__dict__.copy()
   ```



##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -0,0 +1,181 @@
+#
+# 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.
+#
+
+import json
+import sys
+from typing import TYPE_CHECKING, Any, cast, Dict, List, Optional
+
+from pyspark.errors import StreamingQueryException
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.streaming.query import (
+    StreamingQuery as PySparkStreamingQuery,
+)
+
+__all__ = [
+    "StreamingQuery",  # TODO(WIP): "StreamingQueryManager"
+]
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+
+
+class StreamingQuery:
+    def __init__(
+        self, session: "SparkSession", queryId: str, runId: str, name: Optional[str] = None
+    ) -> None:
+        self._session = session
+        self._query_id = queryId
+        self._run_id = runId
+        self._name = name
+
+    @property
+    def id(self) -> str:
+        return self._query_id
+
+    id.__doc__ = PySparkStreamingQuery.id.__doc__
+
+    @property
+    def runId(self) -> str:
+        return self._run_id
+
+    runId.__doc__ = PySparkStreamingQuery.runId.__doc__
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    name.__doc__ = PySparkStreamingQuery.name.__doc__
+
+    @property
+    def isActive(self) -> bool:
+        return self._fetch_status().is_active
+
+    isActive.__doc__ = PySparkStreamingQuery.isActive.__doc__
+
+    def awaitTermination(self, timeout: Optional[int] = None) -> Optional[bool]:
+        raise NotImplementedError()
+
+    awaitTermination.__doc__ = PySparkStreamingQuery.awaitTermination.__doc__
+
+    @property
+    def status(self) -> Dict[str, Any]:
+        proto = self._fetch_status()
+        return {
+            "message": proto.status_message,
+            "isDataAvailable": proto.is_data_available,
+            "isTriggerActive": proto.is_trigger_active,
+        }
+
+    status.__doc__ = PySparkStreamingQuery.status.__doc__
+
+    @property
+    def recentProgress(self) -> List[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=10000).recent_progress_json)
+        # Server only keeps 100, so 10000 limit is high enough.
+        return [json.loads(p) for p in progress]
+
+    recentProgress.__doc__ = PySparkStreamingQuery.recentProgress.__doc__
+
+    @property
+    def lastProgress(self) -> Optional[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=1).recent_progress_json)
+        if len(progress) > 0:
+            return json.loads(progress[-1])
+        else:
+            return None
+
+    lastProgress.__doc__ = PySparkStreamingQuery.lastProgress.__doc__
+
+    def processAllAvailable(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.process_all_available = True
+        self._execute_streaming_query_cmd(cmd)
+
+    processAllAvailable.__doc__ = PySparkStreamingQuery.processAllAvailable.__doc__
+
+    def stop(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.stop = True
+        self._execute_streaming_query_cmd(cmd)
+
+    stop.__doc__ = PySparkStreamingQuery.stop.__doc__
+
+    def explain(self, extended: bool = False) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.explain.extended = extended
+        result = self._execute_streaming_query_cmd(cmd).explain.result
+        print(result)
+
+    explain.__doc__ = PySparkStreamingQuery.explain.__doc__
+
+    def exception(self) -> Optional[StreamingQueryException]:
+        raise NotImplementedError()
+
+    exception.__doc__ = PySparkStreamingQuery.exception.__doc__
+
+    def _fetch_status(
+        self, recent_progress_limit=0
+    ) -> pb2.StreamingQueryCommandResult.StatusResult:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.status.recent_progress_limit = recent_progress_limit
+
+        return self._execute_streaming_query_cmd(cmd).status
+
+    def _execute_streaming_query_cmd(
+        self, cmd: pb2.StreamingQueryCommand
+    ) -> pb2.StreamingQueryCommandResult:
+        cmd.query_id = self._query_id
+        cmd.run_id = self._run_id
+        exec_cmd = pb2.Command()
+        exec_cmd.streaming_query_command.CopyFrom(cmd)
+        (_, properties) = self._session.client.execute_command(exec_cmd)
+        return cast(pb2.StreamingQueryCommandResult, properties["streaming_query_command_result"])
+
+
+# TODO(WIP) class StreamingQueryManager:
+
+
+def _test() -> None:
+    import doctest
+    import os
+    from pyspark.sql import SparkSession
+    import pyspark.sql.streaming.query

Review Comment:
   ```suggestion
       import pyspark.sql.connect.streaming.query
   ```



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


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

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1157036089


##########
python/pyspark/sql/connect/readwriter.py:
##########
@@ -37,7 +37,7 @@
     from pyspark.sql.connect._typing import ColumnOrName, OptionalPrimitiveType
     from pyspark.sql.connect.session import SparkSession
 
-__all__ = ["DataFrameReader", "DataFrameWriter"]
+__all__ = ["DataFrameReader", "DataFrameWriter", "OptionUtils", "to_str"]

Review Comment:
   I don't think we need this change. `OptionUtils` and `to_str` aren't API. They can still be imported without this at `python/pyspark/sql/connect/streaming/readwriter.py`.



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


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

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
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


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

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1156239889


##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,118 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+// Starts write stream operation as streaming query. Query ID and Run ID of the streaming
+// query are returned.
+message WriteStreamOperationStart {
+
+  // (Required) The output of the `input` streaming relation will be written.
+  Relation input = 1;
+
+  // The following fields directly map to API for DataStreamWriter().
+  // Consult API documentation unless explicitly documented here.
+
+  string format = 2;
+  map<string, string> options = 3;
+  repeated string partitioning_column_names = 4;
+
+  oneof trigger {
+    string processing_time_interval = 5;
+    bool available_now = 6;
+    bool one_time = 7;
+    string continuous_checkpoint_interval = 8;
+  }
+
+  string output_mode = 9;
+  string query_name = 10;
+
+  // The destination is optional. When set, it can be a path or a table name.
+  oneof sink_destination {
+    string path = 11;
+    string table_name = 12;
+  }
+}
+
+message WriteStreamOperationStartResult {
+
+  // (Required)
+  string query_id = 1;

Review Comment:
   Updated. Yes, lets have uniform standard across spark-connect and update documentation in the proto files. 
   
   Note that this is just the beginning. 90% streaming work still left. These protos will mostly change. 



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


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

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1157043732


##########
python/pyspark/sql/connect/streaming/readwriter.py:
##########
@@ -0,0 +1,484 @@
+#
+# 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.
+#
+
+from pyspark.sql.connect.utils import check_dependencies
+
+check_dependencies(__name__)
+
+import sys
+from typing import cast, overload, Callable, Dict, List, Optional, TYPE_CHECKING, Union
+
+from pyspark.sql.connect.plan import DataSource, LogicalPlan, WriteStreamOperation
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.connect.readwriter import OptionUtils, to_str
+from pyspark.sql.connect.streaming.query import StreamingQuery
+from pyspark.sql.streaming.readwriter import (
+    DataStreamReader as PySparkDataStreamReader,
+    DataStreamWriter as PySparkDataStreamWriter,
+)
+from pyspark.sql.types import Row, StructType
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.dataframe import DataFrame
+
+__all__ = ["DataStreamReader", "DataStreamWriter"]
+
+
+class DataStreamReader(OptionUtils):
+    def __init__(self, client: "SparkSession") -> None:
+        self._format: Optional[str] = None
+        self._schema = ""
+        self._client = client
+        self._options: Dict[str, str] = {}
+
+    def _df(self, plan: LogicalPlan) -> "DataFrame":
+        from pyspark.sql.connect.dataframe import DataFrame
+
+        return DataFrame.withPlan(plan, self._client)
+
+    def format(self, source: str) -> "DataStreamReader":
+        self._format = source
+        return self
+
+    format.__doc__ = PySparkDataStreamReader.format.__doc__
+
+    def schema(self, schema: Union[StructType, str]) -> "DataStreamReader":
+        if isinstance(schema, StructType):
+            self._schema = schema.json()
+        elif isinstance(schema, str):
+            self._schema = schema
+        else:
+            raise TypeError("schema should be StructType or string")
+        return self
+
+    schema.__doc__ = PySparkDataStreamReader.schema.__doc__
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataStreamReader":
+        self._options[key] = str(value)
+        return self
+
+    option.__doc__ = PySparkDataStreamReader.option.__doc__
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataStreamReader":
+        for k in options:
+            self.option(k, to_str(options[k]))
+        return self
+
+    options.__doc__ = PySparkDataStreamReader.options.__doc__
+
+    def load(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> "DataFrame":
+        if format is not None:
+            self.format(format)
+        if schema is not None:
+            self.schema(schema)
+        self.options(**options)
+        if path is not None:
+            if type(path) != str or len(path.strip()) == 0:
+                raise ValueError(
+                    "If the path is provided for stream, it needs to be a "
+                    + "non-empty string. List of paths are not supported."
+                )
+
+        plan = DataSource(
+            format=self._format,
+            schema=self._schema,
+            options=self._options,
+            paths=[path] if path else None,
+            is_streaming=True,
+        )
+
+        return self._df(plan)
+
+    load.__doc__ = PySparkDataStreamReader.load.__doc__
+
+    def json(
+        self,
+        path: str,
+        schema: Optional[Union[StructType, str]] = None,
+        primitivesAsString: Optional[Union[bool, str]] = None,
+        prefersDecimal: Optional[Union[bool, str]] = None,
+        allowComments: Optional[Union[bool, str]] = None,
+        allowUnquotedFieldNames: Optional[Union[bool, str]] = None,
+        allowSingleQuotes: Optional[Union[bool, str]] = None,
+        allowNumericLeadingZero: Optional[Union[bool, str]] = None,
+        allowBackslashEscapingAnyCharacter: Optional[Union[bool, str]] = None,
+        mode: Optional[str] = None,
+        columnNameOfCorruptRecord: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        multiLine: Optional[Union[bool, str]] = None,
+        allowUnquotedControlChars: Optional[Union[bool, str]] = None,
+        lineSep: Optional[str] = None,
+        locale: Optional[str] = None,
+        dropFieldIfAllNull: Optional[Union[bool, str]] = None,
+        encoding: Optional[str] = None,
+        pathGlobFilter: Optional[Union[bool, str]] = None,
+        recursiveFileLookup: Optional[Union[bool, str]] = None,
+        allowNonNumericNumbers: Optional[Union[bool, str]] = None,
+    ) -> "DataFrame":
+        self._set_opts(
+            schema=schema,
+            primitivesAsString=primitivesAsString,
+            prefersDecimal=prefersDecimal,
+            allowComments=allowComments,
+            allowUnquotedFieldNames=allowUnquotedFieldNames,
+            allowSingleQuotes=allowSingleQuotes,
+            allowNumericLeadingZero=allowNumericLeadingZero,
+            allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter,
+            mode=mode,
+            columnNameOfCorruptRecord=columnNameOfCorruptRecord,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            multiLine=multiLine,
+            allowUnquotedControlChars=allowUnquotedControlChars,
+            lineSep=lineSep,
+            locale=locale,
+            dropFieldIfAllNull=dropFieldIfAllNull,
+            encoding=encoding,
+            pathGlobFilter=pathGlobFilter,
+            recursiveFileLookup=recursiveFileLookup,
+            allowNonNumericNumbers=allowNonNumericNumbers,
+        )
+        if isinstance(path, str):
+            return self.load(path=path, format="json")
+        else:
+            raise TypeError("path can be only a single string")
+
+    json.__doc__ = PySparkDataStreamReader.json.__doc__
+
+    # def orc() TODO
+    # def parquet() TODO
+    # def text() TODO
+
+    def csv(
+        self,
+        path: str,
+        schema: Optional[Union[StructType, str]] = None,
+        sep: Optional[str] = None,
+        encoding: Optional[str] = None,
+        quote: Optional[str] = None,
+        escape: Optional[str] = None,
+        comment: Optional[str] = None,
+        header: Optional[Union[bool, str]] = None,
+        inferSchema: Optional[Union[bool, str]] = None,
+        ignoreLeadingWhiteSpace: Optional[Union[bool, str]] = None,
+        ignoreTrailingWhiteSpace: Optional[Union[bool, str]] = None,
+        nullValue: Optional[str] = None,
+        nanValue: Optional[str] = None,
+        positiveInf: Optional[str] = None,
+        negativeInf: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        maxColumns: Optional[Union[int, str]] = None,
+        maxCharsPerColumn: Optional[Union[int, str]] = None,
+        maxMalformedLogPerPartition: Optional[Union[int, str]] = None,
+        mode: Optional[str] = None,
+        columnNameOfCorruptRecord: Optional[str] = None,
+        multiLine: Optional[Union[bool, str]] = None,
+        charToEscapeQuoteEscaping: Optional[Union[bool, str]] = None,
+        enforceSchema: Optional[Union[bool, str]] = None,
+        emptyValue: Optional[str] = None,
+        locale: Optional[str] = None,
+        lineSep: Optional[str] = None,
+        pathGlobFilter: Optional[Union[bool, str]] = None,
+        recursiveFileLookup: Optional[Union[bool, str]] = None,
+        unescapedQuoteHandling: Optional[str] = None,
+    ) -> "DataFrame":
+        self._set_opts(
+            schema=schema,
+            sep=sep,
+            encoding=encoding,
+            quote=quote,
+            escape=escape,
+            comment=comment,
+            header=header,
+            inferSchema=inferSchema,
+            ignoreLeadingWhiteSpace=ignoreLeadingWhiteSpace,
+            ignoreTrailingWhiteSpace=ignoreTrailingWhiteSpace,
+            nullValue=nullValue,
+            nanValue=nanValue,
+            positiveInf=positiveInf,
+            negativeInf=negativeInf,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            maxColumns=maxColumns,
+            maxCharsPerColumn=maxCharsPerColumn,
+            maxMalformedLogPerPartition=maxMalformedLogPerPartition,
+            mode=mode,
+            columnNameOfCorruptRecord=columnNameOfCorruptRecord,
+            multiLine=multiLine,
+            charToEscapeQuoteEscaping=charToEscapeQuoteEscaping,
+            enforceSchema=enforceSchema,
+            emptyValue=emptyValue,
+            locale=locale,
+            lineSep=lineSep,
+            pathGlobFilter=pathGlobFilter,
+            recursiveFileLookup=recursiveFileLookup,
+            unescapedQuoteHandling=unescapedQuoteHandling,
+        )
+        if isinstance(path, str):
+            return self.load(path=path, format="csv")
+        else:
+            raise TypeError("path can be only a single string")
+
+    csv.__doc__ = PySparkDataStreamReader.csv.__doc__
+
+    # def table() TODO. Use Read(table_name) relation.
+
+
+DataStreamReader.__doc__ = PySparkDataStreamReader.__doc__
+
+
+class DataStreamWriter:
+    def __init__(self, plan: "LogicalPlan", session: "SparkSession") -> None:
+        self._session = session
+        self._write_stream = WriteStreamOperation(plan)
+        self._write_proto = self._write_stream.write_op
+
+    def outputMode(self, outputMode: str) -> "DataStreamWriter":
+        self._write_proto.output_mode = outputMode
+        return self
+
+    outputMode.__doc__ = PySparkDataStreamWriter.outputMode.__doc__
+
+    def format(self, source: str) -> "DataStreamWriter":
+        self._write_proto.format = source
+        return self
+
+    format.__doc__ = PySparkDataStreamWriter.format.__doc__
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataStreamWriter":
+        self._write_proto.options[key] = to_str(value)
+        return self
+
+    option.__doc__ = PySparkDataStreamWriter.option.__doc__
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataStreamWriter":
+        for k in options:
+            self.option(k, options[k])
+        return self
+
+    options.__doc__ = PySparkDataStreamWriter.options.__doc__
+
+    @overload
+    def partitionBy(self, *cols: str) -> "DataStreamWriter":
+        ...
+
+    @overload
+    def partitionBy(self, __cols: List[str]) -> "DataStreamWriter":
+        ...
+
+    def partitionBy(self, *cols: str) -> "DataStreamWriter":  # type: ignore[misc]
+        if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
+            cols = cols[0]
+        self._write_proto.partitioning_cols = cast(List[str], cols)
+        return self
+
+    partitionBy.__doc__ = PySparkDataStreamWriter.partitionBy.__doc__
+
+    def queryName(self, queryName: str) -> "DataStreamWriter":
+        self._write_proto.query_name = queryName
+        return self
+
+    queryName.__doc__ = PySparkDataStreamWriter.queryName.__doc__
+
+    @overload
+    def trigger(self, *, processingTime: str) -> "DataStreamWriter":
+        ...
+
+    @overload
+    def trigger(self, *, once: bool) -> "DataStreamWriter":
+        ...
+
+    @overload
+    def trigger(self, *, continuous: str) -> "DataStreamWriter":
+        ...
+
+    @overload
+    def trigger(self, *, availableNow: bool) -> "DataStreamWriter":
+        ...
+
+    def trigger(
+        self,
+        *,
+        processingTime: Optional[str] = None,
+        once: Optional[bool] = None,
+        continuous: Optional[str] = None,
+        availableNow: Optional[bool] = None,
+    ) -> "DataStreamWriter":
+        params = [processingTime, once, continuous, availableNow]
+
+        if params.count(None) == 4:
+            raise ValueError("No trigger provided")
+        elif params.count(None) < 3:
+            raise ValueError("Multiple triggers not allowed.")
+
+        if processingTime is not None:
+            if type(processingTime) != str or len(processingTime.strip()) == 0:
+                raise ValueError(
+                    "Value for processingTime must be a non empty string. Got: %s" % processingTime
+                )
+            self._write_proto.processing_time_interval = processingTime.strip()
+
+        elif once is not None:
+            if once is not True:
+                raise ValueError("Value for once must be True. Got: %s" % once)
+            self._write_proto.one_time = True
+
+        elif continuous is not None:
+            if type(continuous) != str or len(continuous.strip()) == 0:
+                raise ValueError(
+                    "Value for continuous must be a non empty string. Got: %s" % continuous
+                )
+            self._write_proto.continuous_checkpoint_interval = continuous.strip()
+
+        else:
+            if availableNow is not True:
+                raise ValueError("Value for availableNow must be True. Got: %s" % availableNow)
+            self._write_proto.available_now = True
+
+        return self
+
+    trigger.__doc__ = PySparkDataStreamWriter.trigger.__doc__
+
+    @overload
+    def foreach(self, f: Callable[[Row], None]) -> "DataStreamWriter":
+        ...
+
+    @overload
+    def foreach(self, f: "SupportsProcess") -> "DataStreamWriter":
+        ...
+
+    def foreach(self, f: Union[Callable[[Row], None], "SupportsProcess"]) -> "DataStreamWriter":
+        raise NotImplementedError("foreach() is not implemented.")
+
+    foreach.__doc__ = PySparkDataStreamWriter.foreach.__doc__
+
+    def _start_internal(
+        self,
+        path: Optional[str] = None,
+        tableName: Optional[str] = None,
+        format: Optional[str] = None,
+        outputMode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        queryName: Optional[str] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> StreamingQuery:
+        self.options(**options)
+        if outputMode is not None:
+            self.outputMode(outputMode)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        if queryName is not None:
+            self.queryName(queryName)
+        if path:
+            self._write_proto.path = path
+        if tableName:
+            self._write_proto.table_name = tableName
+
+        cmd = self._write_stream.command(self._session.client)
+        (_, properties) = self._session.client.execute_command(cmd)
+
+        start_result = cast(
+            pb2.WriteStreamOperationStartResult, properties["write_stream_operation_start_result"]
+        )
+        return StreamingQuery(
+            session=self._session,
+            queryId=start_result.query_id,
+            runId=start_result.run_id,
+            name=start_result.name,
+        )
+
+    def start(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        outputMode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        queryName: Optional[str] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> StreamingQuery:
+        return self._start_internal(
+            path=path,
+            format=format,
+            outputMode=outputMode,
+            partitionBy=partitionBy,
+            queryName=queryName,
+            **options,
+        )
+
+    start.__doc__ = PySparkDataStreamWriter.start.__doc__
+
+    def toTable(
+        self,
+        tableName: str,
+        format: Optional[str] = None,
+        outputMode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        queryName: Optional[str] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> StreamingQuery:
+        return self._start_internal(
+            tableName=tableName,
+            format=format,
+            outputMode=outputMode,
+            partitionBy=partitionBy,
+            queryName=queryName,
+            **options,
+        )
+
+    toTable.__doc__ = PySparkDataStreamWriter.toTable.__doc__
+
+
+def _test() -> None:
+    import doctest
+    import os
+    from pyspark.sql import SparkSession
+    import pyspark.sql.streaming.readwriter

Review Comment:
   ```suggestion
       import pyspark.sql.connect.streaming.readwriter
   ```



##########
python/pyspark/sql/connect/streaming/readwriter.py:
##########
@@ -0,0 +1,484 @@
+#
+# 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.
+#
+
+from pyspark.sql.connect.utils import check_dependencies
+
+check_dependencies(__name__)
+
+import sys
+from typing import cast, overload, Callable, Dict, List, Optional, TYPE_CHECKING, Union
+
+from pyspark.sql.connect.plan import DataSource, LogicalPlan, WriteStreamOperation
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.connect.readwriter import OptionUtils, to_str
+from pyspark.sql.connect.streaming.query import StreamingQuery
+from pyspark.sql.streaming.readwriter import (
+    DataStreamReader as PySparkDataStreamReader,
+    DataStreamWriter as PySparkDataStreamWriter,
+)
+from pyspark.sql.types import Row, StructType
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.dataframe import DataFrame
+
+__all__ = ["DataStreamReader", "DataStreamWriter"]
+
+
+class DataStreamReader(OptionUtils):
+    def __init__(self, client: "SparkSession") -> None:
+        self._format: Optional[str] = None
+        self._schema = ""
+        self._client = client
+        self._options: Dict[str, str] = {}
+
+    def _df(self, plan: LogicalPlan) -> "DataFrame":
+        from pyspark.sql.connect.dataframe import DataFrame
+
+        return DataFrame.withPlan(plan, self._client)
+
+    def format(self, source: str) -> "DataStreamReader":
+        self._format = source
+        return self
+
+    format.__doc__ = PySparkDataStreamReader.format.__doc__
+
+    def schema(self, schema: Union[StructType, str]) -> "DataStreamReader":
+        if isinstance(schema, StructType):
+            self._schema = schema.json()
+        elif isinstance(schema, str):
+            self._schema = schema
+        else:
+            raise TypeError("schema should be StructType or string")
+        return self
+
+    schema.__doc__ = PySparkDataStreamReader.schema.__doc__
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataStreamReader":
+        self._options[key] = str(value)
+        return self
+
+    option.__doc__ = PySparkDataStreamReader.option.__doc__
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataStreamReader":
+        for k in options:
+            self.option(k, to_str(options[k]))
+        return self
+
+    options.__doc__ = PySparkDataStreamReader.options.__doc__
+
+    def load(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> "DataFrame":
+        if format is not None:
+            self.format(format)
+        if schema is not None:
+            self.schema(schema)
+        self.options(**options)
+        if path is not None:
+            if type(path) != str or len(path.strip()) == 0:
+                raise ValueError(
+                    "If the path is provided for stream, it needs to be a "
+                    + "non-empty string. List of paths are not supported."
+                )
+
+        plan = DataSource(
+            format=self._format,
+            schema=self._schema,
+            options=self._options,
+            paths=[path] if path else None,
+            is_streaming=True,
+        )
+
+        return self._df(plan)
+
+    load.__doc__ = PySparkDataStreamReader.load.__doc__
+
+    def json(
+        self,
+        path: str,
+        schema: Optional[Union[StructType, str]] = None,
+        primitivesAsString: Optional[Union[bool, str]] = None,
+        prefersDecimal: Optional[Union[bool, str]] = None,
+        allowComments: Optional[Union[bool, str]] = None,
+        allowUnquotedFieldNames: Optional[Union[bool, str]] = None,
+        allowSingleQuotes: Optional[Union[bool, str]] = None,
+        allowNumericLeadingZero: Optional[Union[bool, str]] = None,
+        allowBackslashEscapingAnyCharacter: Optional[Union[bool, str]] = None,
+        mode: Optional[str] = None,
+        columnNameOfCorruptRecord: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        multiLine: Optional[Union[bool, str]] = None,
+        allowUnquotedControlChars: Optional[Union[bool, str]] = None,
+        lineSep: Optional[str] = None,
+        locale: Optional[str] = None,
+        dropFieldIfAllNull: Optional[Union[bool, str]] = None,
+        encoding: Optional[str] = None,
+        pathGlobFilter: Optional[Union[bool, str]] = None,
+        recursiveFileLookup: Optional[Union[bool, str]] = None,
+        allowNonNumericNumbers: Optional[Union[bool, str]] = None,
+    ) -> "DataFrame":
+        self._set_opts(
+            schema=schema,
+            primitivesAsString=primitivesAsString,
+            prefersDecimal=prefersDecimal,
+            allowComments=allowComments,
+            allowUnquotedFieldNames=allowUnquotedFieldNames,
+            allowSingleQuotes=allowSingleQuotes,
+            allowNumericLeadingZero=allowNumericLeadingZero,
+            allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter,
+            mode=mode,
+            columnNameOfCorruptRecord=columnNameOfCorruptRecord,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            multiLine=multiLine,
+            allowUnquotedControlChars=allowUnquotedControlChars,
+            lineSep=lineSep,
+            locale=locale,
+            dropFieldIfAllNull=dropFieldIfAllNull,
+            encoding=encoding,
+            pathGlobFilter=pathGlobFilter,
+            recursiveFileLookup=recursiveFileLookup,
+            allowNonNumericNumbers=allowNonNumericNumbers,
+        )
+        if isinstance(path, str):
+            return self.load(path=path, format="json")
+        else:
+            raise TypeError("path can be only a single string")
+
+    json.__doc__ = PySparkDataStreamReader.json.__doc__
+
+    # def orc() TODO
+    # def parquet() TODO
+    # def text() TODO
+
+    def csv(
+        self,
+        path: str,
+        schema: Optional[Union[StructType, str]] = None,
+        sep: Optional[str] = None,
+        encoding: Optional[str] = None,
+        quote: Optional[str] = None,
+        escape: Optional[str] = None,
+        comment: Optional[str] = None,
+        header: Optional[Union[bool, str]] = None,
+        inferSchema: Optional[Union[bool, str]] = None,
+        ignoreLeadingWhiteSpace: Optional[Union[bool, str]] = None,
+        ignoreTrailingWhiteSpace: Optional[Union[bool, str]] = None,
+        nullValue: Optional[str] = None,
+        nanValue: Optional[str] = None,
+        positiveInf: Optional[str] = None,
+        negativeInf: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        maxColumns: Optional[Union[int, str]] = None,
+        maxCharsPerColumn: Optional[Union[int, str]] = None,
+        maxMalformedLogPerPartition: Optional[Union[int, str]] = None,
+        mode: Optional[str] = None,
+        columnNameOfCorruptRecord: Optional[str] = None,
+        multiLine: Optional[Union[bool, str]] = None,
+        charToEscapeQuoteEscaping: Optional[Union[bool, str]] = None,
+        enforceSchema: Optional[Union[bool, str]] = None,
+        emptyValue: Optional[str] = None,
+        locale: Optional[str] = None,
+        lineSep: Optional[str] = None,
+        pathGlobFilter: Optional[Union[bool, str]] = None,
+        recursiveFileLookup: Optional[Union[bool, str]] = None,
+        unescapedQuoteHandling: Optional[str] = None,
+    ) -> "DataFrame":
+        self._set_opts(
+            schema=schema,
+            sep=sep,
+            encoding=encoding,
+            quote=quote,
+            escape=escape,
+            comment=comment,
+            header=header,
+            inferSchema=inferSchema,
+            ignoreLeadingWhiteSpace=ignoreLeadingWhiteSpace,
+            ignoreTrailingWhiteSpace=ignoreTrailingWhiteSpace,
+            nullValue=nullValue,
+            nanValue=nanValue,
+            positiveInf=positiveInf,
+            negativeInf=negativeInf,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            maxColumns=maxColumns,
+            maxCharsPerColumn=maxCharsPerColumn,
+            maxMalformedLogPerPartition=maxMalformedLogPerPartition,
+            mode=mode,
+            columnNameOfCorruptRecord=columnNameOfCorruptRecord,
+            multiLine=multiLine,
+            charToEscapeQuoteEscaping=charToEscapeQuoteEscaping,
+            enforceSchema=enforceSchema,
+            emptyValue=emptyValue,
+            locale=locale,
+            lineSep=lineSep,
+            pathGlobFilter=pathGlobFilter,
+            recursiveFileLookup=recursiveFileLookup,
+            unescapedQuoteHandling=unescapedQuoteHandling,
+        )
+        if isinstance(path, str):
+            return self.load(path=path, format="csv")
+        else:
+            raise TypeError("path can be only a single string")
+
+    csv.__doc__ = PySparkDataStreamReader.csv.__doc__
+
+    # def table() TODO. Use Read(table_name) relation.
+
+
+DataStreamReader.__doc__ = PySparkDataStreamReader.__doc__
+
+
+class DataStreamWriter:
+    def __init__(self, plan: "LogicalPlan", session: "SparkSession") -> None:
+        self._session = session
+        self._write_stream = WriteStreamOperation(plan)
+        self._write_proto = self._write_stream.write_op
+
+    def outputMode(self, outputMode: str) -> "DataStreamWriter":
+        self._write_proto.output_mode = outputMode
+        return self
+
+    outputMode.__doc__ = PySparkDataStreamWriter.outputMode.__doc__
+
+    def format(self, source: str) -> "DataStreamWriter":
+        self._write_proto.format = source
+        return self
+
+    format.__doc__ = PySparkDataStreamWriter.format.__doc__
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataStreamWriter":
+        self._write_proto.options[key] = to_str(value)
+        return self
+
+    option.__doc__ = PySparkDataStreamWriter.option.__doc__
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataStreamWriter":
+        for k in options:
+            self.option(k, options[k])
+        return self
+
+    options.__doc__ = PySparkDataStreamWriter.options.__doc__
+
+    @overload
+    def partitionBy(self, *cols: str) -> "DataStreamWriter":
+        ...
+
+    @overload
+    def partitionBy(self, __cols: List[str]) -> "DataStreamWriter":
+        ...
+
+    def partitionBy(self, *cols: str) -> "DataStreamWriter":  # type: ignore[misc]
+        if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
+            cols = cols[0]
+        self._write_proto.partitioning_cols = cast(List[str], cols)
+        return self
+
+    partitionBy.__doc__ = PySparkDataStreamWriter.partitionBy.__doc__
+
+    def queryName(self, queryName: str) -> "DataStreamWriter":
+        self._write_proto.query_name = queryName
+        return self
+
+    queryName.__doc__ = PySparkDataStreamWriter.queryName.__doc__
+
+    @overload
+    def trigger(self, *, processingTime: str) -> "DataStreamWriter":
+        ...
+
+    @overload
+    def trigger(self, *, once: bool) -> "DataStreamWriter":
+        ...
+
+    @overload
+    def trigger(self, *, continuous: str) -> "DataStreamWriter":
+        ...
+
+    @overload
+    def trigger(self, *, availableNow: bool) -> "DataStreamWriter":
+        ...
+
+    def trigger(
+        self,
+        *,
+        processingTime: Optional[str] = None,
+        once: Optional[bool] = None,
+        continuous: Optional[str] = None,
+        availableNow: Optional[bool] = None,
+    ) -> "DataStreamWriter":
+        params = [processingTime, once, continuous, availableNow]
+
+        if params.count(None) == 4:
+            raise ValueError("No trigger provided")
+        elif params.count(None) < 3:
+            raise ValueError("Multiple triggers not allowed.")
+
+        if processingTime is not None:
+            if type(processingTime) != str or len(processingTime.strip()) == 0:
+                raise ValueError(
+                    "Value for processingTime must be a non empty string. Got: %s" % processingTime
+                )
+            self._write_proto.processing_time_interval = processingTime.strip()
+
+        elif once is not None:
+            if once is not True:
+                raise ValueError("Value for once must be True. Got: %s" % once)
+            self._write_proto.one_time = True
+
+        elif continuous is not None:
+            if type(continuous) != str or len(continuous.strip()) == 0:
+                raise ValueError(
+                    "Value for continuous must be a non empty string. Got: %s" % continuous
+                )
+            self._write_proto.continuous_checkpoint_interval = continuous.strip()
+
+        else:
+            if availableNow is not True:
+                raise ValueError("Value for availableNow must be True. Got: %s" % availableNow)
+            self._write_proto.available_now = True
+
+        return self
+
+    trigger.__doc__ = PySparkDataStreamWriter.trigger.__doc__
+
+    @overload
+    def foreach(self, f: Callable[[Row], None]) -> "DataStreamWriter":
+        ...
+
+    @overload
+    def foreach(self, f: "SupportsProcess") -> "DataStreamWriter":
+        ...
+
+    def foreach(self, f: Union[Callable[[Row], None], "SupportsProcess"]) -> "DataStreamWriter":
+        raise NotImplementedError("foreach() is not implemented.")
+
+    foreach.__doc__ = PySparkDataStreamWriter.foreach.__doc__
+
+    def _start_internal(
+        self,
+        path: Optional[str] = None,
+        tableName: Optional[str] = None,
+        format: Optional[str] = None,
+        outputMode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        queryName: Optional[str] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> StreamingQuery:
+        self.options(**options)
+        if outputMode is not None:
+            self.outputMode(outputMode)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        if queryName is not None:
+            self.queryName(queryName)
+        if path:
+            self._write_proto.path = path
+        if tableName:
+            self._write_proto.table_name = tableName
+
+        cmd = self._write_stream.command(self._session.client)
+        (_, properties) = self._session.client.execute_command(cmd)
+
+        start_result = cast(
+            pb2.WriteStreamOperationStartResult, properties["write_stream_operation_start_result"]
+        )
+        return StreamingQuery(
+            session=self._session,
+            queryId=start_result.query_id,
+            runId=start_result.run_id,
+            name=start_result.name,
+        )
+
+    def start(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        outputMode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        queryName: Optional[str] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> StreamingQuery:
+        return self._start_internal(
+            path=path,
+            format=format,
+            outputMode=outputMode,
+            partitionBy=partitionBy,
+            queryName=queryName,
+            **options,
+        )
+
+    start.__doc__ = PySparkDataStreamWriter.start.__doc__
+
+    def toTable(
+        self,
+        tableName: str,
+        format: Optional[str] = None,
+        outputMode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        queryName: Optional[str] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> StreamingQuery:
+        return self._start_internal(
+            tableName=tableName,
+            format=format,
+            outputMode=outputMode,
+            partitionBy=partitionBy,
+            queryName=queryName,
+            **options,
+        )
+
+    toTable.__doc__ = PySparkDataStreamWriter.toTable.__doc__
+
+
+def _test() -> None:
+    import doctest
+    import os
+    from pyspark.sql import SparkSession
+    import pyspark.sql.streaming.readwriter
+
+    os.chdir(os.environ["SPARK_HOME"])
+
+    globs = pyspark.sql.streaming.readwriter.__dict__.copy()

Review Comment:
   ```suggestion
       globs = pyspark.sql.connect.streaming.readwriter.__dict__.copy()
   ```



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


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

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1157045341


##########
python/pyspark/sql/dataframe.py:
##########
@@ -529,14 +529,10 @@ def writeStream(self) -> DataStreamWriter:
         --------
         >>> import tempfile
         >>> df = spark.readStream.format("rate").load()
-        >>> type(df.writeStream)

Review Comment:
   You can fix the type like:
   
   ```python
   type(df.writeStream)
   <class '...readwriter.DataFrameReader'>
   ```
   
   



##########
python/pyspark/sql/dataframe.py:
##########
@@ -529,14 +529,10 @@ def writeStream(self) -> DataStreamWriter:
         --------
         >>> import tempfile
         >>> df = spark.readStream.format("rate").load()
-        >>> type(df.writeStream)

Review Comment:
   You can fix the type like:
   
   ```python
   type(df.writeStream)  # doctest: +ELLIPSIS
   <class '...readwriter.DataFrameReader'>
   ```
   
   



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


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

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1156212372


##########
connector/connect/common/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -154,6 +158,9 @@ message Read {
     //
     // This is only supported by the JDBC data source.
     repeated string predicates = 5;
+
+    // (Optional) Source table name for a streaming read. Not used in batch read.
+    string streaming_table_name = 6;

Review Comment:
   NamedTable does not take options. It is not yet usable for Streaming yet. 
   
   
   >  A read with [NamedTable](https://github.com/apache/spark/blob/42481a047b34c39b1feaf5a9081b6c3432409bae/connector/connect/common/src/main/protobuf/spark/connect/relations.proto#L122-L125) does not take any options in spark-connect. This is unlike [DataSource](https://github.com/apache/spark/blob/42481a047b34c39b1feaf5a9081b6c3432409bae/connector/connect/common/src/main/protobuf/spark/connect/relations.proto#L127). Is that intentional? The interface allows specifying options.



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


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

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1157042806


##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -0,0 +1,181 @@
+#
+# 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.
+#
+
+import json
+import sys
+from typing import TYPE_CHECKING, Any, cast, Dict, List, Optional
+
+from pyspark.errors import StreamingQueryException
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.streaming.query import (
+    StreamingQuery as PySparkStreamingQuery,
+)
+
+__all__ = [
+    "StreamingQuery",  # TODO(WIP): "StreamingQueryManager"
+]
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+
+
+class StreamingQuery:
+    def __init__(
+        self, session: "SparkSession", queryId: str, runId: str, name: Optional[str] = None
+    ) -> None:
+        self._session = session
+        self._query_id = queryId
+        self._run_id = runId
+        self._name = name
+
+    @property
+    def id(self) -> str:
+        return self._query_id
+
+    id.__doc__ = PySparkStreamingQuery.id.__doc__
+
+    @property
+    def runId(self) -> str:
+        return self._run_id
+
+    runId.__doc__ = PySparkStreamingQuery.runId.__doc__
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    name.__doc__ = PySparkStreamingQuery.name.__doc__
+
+    @property
+    def isActive(self) -> bool:
+        return self._fetch_status().is_active
+
+    isActive.__doc__ = PySparkStreamingQuery.isActive.__doc__
+
+    def awaitTermination(self, timeout: Optional[int] = None) -> Optional[bool]:
+        raise NotImplementedError()
+
+    awaitTermination.__doc__ = PySparkStreamingQuery.awaitTermination.__doc__
+
+    @property
+    def status(self) -> Dict[str, Any]:
+        proto = self._fetch_status()
+        return {
+            "message": proto.status_message,
+            "isDataAvailable": proto.is_data_available,
+            "isTriggerActive": proto.is_trigger_active,
+        }
+
+    status.__doc__ = PySparkStreamingQuery.status.__doc__
+
+    @property
+    def recentProgress(self) -> List[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=10000).recent_progress_json)
+        # Server only keeps 100, so 10000 limit is high enough.
+        return [json.loads(p) for p in progress]
+
+    recentProgress.__doc__ = PySparkStreamingQuery.recentProgress.__doc__
+
+    @property
+    def lastProgress(self) -> Optional[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=1).recent_progress_json)
+        if len(progress) > 0:
+            return json.loads(progress[-1])
+        else:
+            return None
+
+    lastProgress.__doc__ = PySparkStreamingQuery.lastProgress.__doc__
+
+    def processAllAvailable(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.process_all_available = True
+        self._execute_streaming_query_cmd(cmd)
+
+    processAllAvailable.__doc__ = PySparkStreamingQuery.processAllAvailable.__doc__
+
+    def stop(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.stop = True
+        self._execute_streaming_query_cmd(cmd)
+
+    stop.__doc__ = PySparkStreamingQuery.stop.__doc__
+
+    def explain(self, extended: bool = False) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.explain.extended = extended
+        result = self._execute_streaming_query_cmd(cmd).explain.result
+        print(result)
+
+    explain.__doc__ = PySparkStreamingQuery.explain.__doc__
+
+    def exception(self) -> Optional[StreamingQueryException]:
+        raise NotImplementedError()
+
+    exception.__doc__ = PySparkStreamingQuery.exception.__doc__
+
+    def _fetch_status(
+        self, recent_progress_limit=0
+    ) -> pb2.StreamingQueryCommandResult.StatusResult:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.status.recent_progress_limit = recent_progress_limit
+
+        return self._execute_streaming_query_cmd(cmd).status
+
+    def _execute_streaming_query_cmd(
+        self, cmd: pb2.StreamingQueryCommand
+    ) -> pb2.StreamingQueryCommandResult:
+        cmd.query_id = self._query_id
+        cmd.run_id = self._run_id
+        exec_cmd = pb2.Command()
+        exec_cmd.streaming_query_command.CopyFrom(cmd)
+        (_, properties) = self._session.client.execute_command(exec_cmd)
+        return cast(pb2.StreamingQueryCommandResult, properties["streaming_query_command_result"])
+
+
+# TODO(WIP) class StreamingQueryManager:
+
+
+def _test() -> None:
+    import doctest
+    import os
+    from pyspark.sql import SparkSession
+    import pyspark.sql.streaming.query
+    from py4j.protocol import Py4JError
+
+    os.chdir(os.environ["SPARK_HOME"])
+
+    globs = pyspark.sql.streaming.query.__dict__.copy()
+    try:
+        spark = SparkSession._getActiveSessionOrCreate()
+    except Py4JError:  # noqa: F821
+        spark = SparkSession(sc)  # type: ignore[name-defined] # noqa: F821
+
+    globs["spark"] = spark
+
+    (failure_count, test_count) = doctest.testmod(
+        pyspark.sql.streaming.query,

Review Comment:
   ```suggestion
           pyspark.sql.connect.streaming.query,
   ```



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


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

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1157042286


##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -0,0 +1,181 @@
+#

Review Comment:
   Should add this file to https://github.com/apache/spark/blob/master/dev/sparktestsupport/modules.py#L740. otherwise, the tests won't actually run in CI.



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


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

Posted by "amaliujia (via GitHub)" <gi...@apache.org>.
amaliujia commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1152387534


##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,97 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+message WriteStreamOperation {

Review Comment:
   Can you follow https://github.com/apache/spark/blob/master/connector/connect/docs/adding-proto-messages.md over the proto style?



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


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

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1156271993


##########
python/pyspark/sql/dataframe.py:
##########
@@ -529,14 +529,10 @@ def writeStream(self) -> DataStreamWriter:
         --------
         >>> import tempfile
         >>> df = spark.readStream.format("rate").load()
-        >>> type(df.writeStream)

Review Comment:
   This is checking the type of writeStream. The python type differs between legacy and spark-connect. This is not any functionality testing. 



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


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

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1156569175


##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,118 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+// Starts write stream operation as streaming query. Query ID and Run ID of the streaming
+// query are returned.
+message WriteStreamOperationStart {
+
+  // (Required) The output of the `input` streaming relation will be written.
+  Relation input = 1;
+
+  // The following fields directly map to API for DataStreamWriter().
+  // Consult API documentation unless explicitly documented here.
+
+  string format = 2;
+  map<string, string> options = 3;
+  repeated string partitioning_column_names = 4;
+
+  oneof trigger {
+    string processing_time_interval = 5;
+    bool available_now = 6;
+    bool one_time = 7;
+    string continuous_checkpoint_interval = 8;
+  }
+
+  string output_mode = 9;
+  string query_name = 10;
+
+  // The destination is optional. When set, it can be a path or a table name.
+  oneof sink_destination {
+    string path = 11;
+    string table_name = 12;
+  }
+}
+
+message WriteStreamOperationStartResult {
+
+  // (Required)
+  string query_id = 1;
+
+  // (Required)
+  string run_id = 2;
+
+  // An optional query name.
+  string name = 3;
+
+  // TODO: How do we indicate errors?
+  // TODO: Consider adding StreamingQueryStatusResult here.
+}
+
+// Commands for a streaming query.
+message StreamingQueryCommand {
+
+  // (Required) query id of the streaming query.
+  string query_id = 1;
+  // (Required) run id of the streaming query.
+  string run_id = 2;
+
+  // A running query is identified by both run_id and query_id.
+
+  oneof command_type {
+    // Status of the query. Used to support multiple status related API like lastProgress().
+    StatusCommand status = 3;
+    // Stops the query.
+    bool stop = 4;
+    // Waits till all the available data is processed. See processAllAvailable() API doc.
+    bool process_all_available = 5;
+    // Returns logical and physical plans.
+    ExplainCommand explain = 6;
+
+    // TODO(SPARK-42960) Add more commands: await_termination(), exception() etc.
+  }
+
+  message StatusCommand {
+    // A limit on how many progress reports to return.
+    int32 recent_progress_limit = 1;
+  }
+
+  message ExplainCommand {
+    // TODO: Consider reusing Explain from AnalyzePlanRequest message.
+    //       We can not do this right now since it base.proto imports this file.
+    bool extended = 1;
+  }
+
+}
+
+// Response for commands on a streaming query.
+message StreamingQueryCommandResult {
+  // (Required)
+  string query_id = 1;
+
+  oneof result_type {
+    StatusResult status = 2;
+    ExplainResult explain = 3;
+  }
+
+  message StatusResult {
+    // This status includes all the available to status, including progress messages.
+
+    // Fields from Scala 'StreamingQueryStatus' struct
+    string status_message = 1;
+    bool is_data_available = 2;
+    bool is_trigger_active = 3;
+
+    bool is_active = 4;
+
+    // Progress reports as an array of json strings.
+    repeated string recent_progress_json = 5;

Review Comment:
   Progress is sent only when requested (this keeps the number of RPCs lower). See `recent_progress_limit` in `StatusComand` above. 
   Let me know if you prefer to split these into multiple commands in this PR. 



##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -0,0 +1,175 @@
+#
+# 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.
+#
+
+import json
+import sys
+from typing import TYPE_CHECKING, Any, cast, Dict, List, Optional
+
+from pyspark.errors import StreamingQueryException
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.streaming.query import (
+    StreamingQuery as PySparkStreamingQuery,
+)
+
+__all__ = [
+    "StreamingQuery",  # TODO(WIP): "StreamingQueryManager"
+]
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+
+
+class StreamingQuery:
+
+    def __init__(self, session: "SparkSession", queryId: str, runId: str, name: Optional[str] = None) -> None:
+        self._session = session
+        self._query_id = queryId
+        self._run_id = runId
+        self._name = name
+
+    @property
+    def id(self) -> str:
+        return self._query_id
+
+    id.__doc__ = PySparkStreamingQuery.id.__doc__
+
+    @property
+    def runId(self) -> str:
+        return self._run_id
+
+    runId.__doc__ = PySparkStreamingQuery.runId.__doc__
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    name.__doc__ = PySparkStreamingQuery.name.__doc__
+
+    @property
+    def isActive(self) -> bool:
+        return self._fetch_status().is_active
+
+    isActive.__doc__ = PySparkStreamingQuery.isActive.__doc__
+
+    def awaitTermination(self, timeout: Optional[int] = None) -> Optional[bool]:
+        raise NotImplementedError()
+
+    awaitTermination.__doc__ = PySparkStreamingQuery.awaitTermination.__doc__
+
+    @property
+    def status(self) -> Dict[str, Any]:
+        proto = self._fetch_status()
+        return {
+            "message": proto.status_message,
+            "isDataAvailable": proto.is_data_available,
+            "isTriggerActive": proto.is_trigger_active
+        }
+
+    status.__doc__ = PySparkStreamingQuery.status.__doc__
+
+    @property
+    def recentProgress(self) -> List[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=-1).recent_progress_json)
+        return [json.loads(p) for p in progress]
+
+    recentProgress.__doc__ = PySparkStreamingQuery.recentProgress.__doc__
+
+    @property
+    def lastProgress(self) -> Optional[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=1).recent_progress_json)
+        if len(progress) > 0:
+            return json.loads(progress[-1])
+        else:
+            return None
+
+    lastProgress.__doc__ = PySparkStreamingQuery.lastProgress.__doc__
+
+    def processAllAvailable(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.process_all_available = True
+        self._execute_streaming_query_cmd(cmd)
+
+    processAllAvailable.__doc__ = PySparkStreamingQuery.processAllAvailable.__doc__
+
+    def stop(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.stop = True
+        self._execute_streaming_query_cmd(cmd)
+
+    stop.__doc__ = PySparkStreamingQuery.stop.__doc__
+
+    def explain(self, extended: bool = False) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.explain.extended = extended
+        result = self._execute_streaming_query_cmd(cmd).explain.result
+        print(result)
+
+    processAllAvailable.__doc__ = PySparkStreamingQuery.processAllAvailable.__doc__
+
+    def exception(self) -> Optional[StreamingQueryException]:
+        raise NotImplementedError()
+
+    processAllAvailable.__doc__ = PySparkStreamingQuery.processAllAvailable.__doc__

Review Comment:
   yes. Fixed. 



##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -0,0 +1,175 @@
+#
+# 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.
+#
+
+import json
+import sys
+from typing import TYPE_CHECKING, Any, cast, Dict, List, Optional
+
+from pyspark.errors import StreamingQueryException
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.streaming.query import (
+    StreamingQuery as PySparkStreamingQuery,
+)
+
+__all__ = [
+    "StreamingQuery",  # TODO(WIP): "StreamingQueryManager"
+]
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+
+
+class StreamingQuery:
+
+    def __init__(self, session: "SparkSession", queryId: str, runId: str, name: Optional[str] = None) -> None:
+        self._session = session
+        self._query_id = queryId
+        self._run_id = runId
+        self._name = name
+
+    @property
+    def id(self) -> str:
+        return self._query_id
+
+    id.__doc__ = PySparkStreamingQuery.id.__doc__
+
+    @property
+    def runId(self) -> str:
+        return self._run_id
+
+    runId.__doc__ = PySparkStreamingQuery.runId.__doc__
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    name.__doc__ = PySparkStreamingQuery.name.__doc__
+
+    @property
+    def isActive(self) -> bool:
+        return self._fetch_status().is_active
+
+    isActive.__doc__ = PySparkStreamingQuery.isActive.__doc__
+
+    def awaitTermination(self, timeout: Optional[int] = None) -> Optional[bool]:
+        raise NotImplementedError()
+
+    awaitTermination.__doc__ = PySparkStreamingQuery.awaitTermination.__doc__
+
+    @property
+    def status(self) -> Dict[str, Any]:
+        proto = self._fetch_status()
+        return {
+            "message": proto.status_message,
+            "isDataAvailable": proto.is_data_available,
+            "isTriggerActive": proto.is_trigger_active
+        }
+
+    status.__doc__ = PySparkStreamingQuery.status.__doc__
+
+    @property
+    def recentProgress(self) -> List[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=-1).recent_progress_json)

Review Comment:
   Changed it to 10000 and added a comment. Server only keeps 100.



##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -0,0 +1,175 @@
+#
+# 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.
+#
+
+import json
+import sys
+from typing import TYPE_CHECKING, Any, cast, Dict, List, Optional
+
+from pyspark.errors import StreamingQueryException
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.streaming.query import (
+    StreamingQuery as PySparkStreamingQuery,
+)
+
+__all__ = [
+    "StreamingQuery",  # TODO(WIP): "StreamingQueryManager"
+]
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+
+
+class StreamingQuery:
+
+    def __init__(self, session: "SparkSession", queryId: str, runId: str, name: Optional[str] = None) -> None:
+        self._session = session
+        self._query_id = queryId
+        self._run_id = runId
+        self._name = name
+
+    @property
+    def id(self) -> str:
+        return self._query_id
+
+    id.__doc__ = PySparkStreamingQuery.id.__doc__
+
+    @property
+    def runId(self) -> str:
+        return self._run_id
+
+    runId.__doc__ = PySparkStreamingQuery.runId.__doc__
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    name.__doc__ = PySparkStreamingQuery.name.__doc__
+
+    @property
+    def isActive(self) -> bool:
+        return self._fetch_status().is_active
+
+    isActive.__doc__ = PySparkStreamingQuery.isActive.__doc__
+
+    def awaitTermination(self, timeout: Optional[int] = None) -> Optional[bool]:
+        raise NotImplementedError()
+
+    awaitTermination.__doc__ = PySparkStreamingQuery.awaitTermination.__doc__
+
+    @property
+    def status(self) -> Dict[str, Any]:
+        proto = self._fetch_status()
+        return {
+            "message": proto.status_message,
+            "isDataAvailable": proto.is_data_available,
+            "isTriggerActive": proto.is_trigger_active
+        }
+
+    status.__doc__ = PySparkStreamingQuery.status.__doc__
+
+    @property
+    def recentProgress(self) -> List[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=-1).recent_progress_json)
+        return [json.loads(p) for p in progress]
+
+    recentProgress.__doc__ = PySparkStreamingQuery.recentProgress.__doc__
+
+    @property
+    def lastProgress(self) -> Optional[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=1).recent_progress_json)
+        if len(progress) > 0:
+            return json.loads(progress[-1])
+        else:
+            return None
+
+    lastProgress.__doc__ = PySparkStreamingQuery.lastProgress.__doc__
+
+    def processAllAvailable(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.process_all_available = True
+        self._execute_streaming_query_cmd(cmd)
+
+    processAllAvailable.__doc__ = PySparkStreamingQuery.processAllAvailable.__doc__
+
+    def stop(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.stop = True
+        self._execute_streaming_query_cmd(cmd)
+
+    stop.__doc__ = PySparkStreamingQuery.stop.__doc__
+
+    def explain(self, extended: bool = False) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.explain.extended = extended
+        result = self._execute_streaming_query_cmd(cmd).explain.result
+        print(result)
+
+    processAllAvailable.__doc__ = PySparkStreamingQuery.processAllAvailable.__doc__

Review Comment:
   yes. Fixed both. 



##########
python/pyspark/sql/connect/streaming/readwriter.py:
##########
@@ -0,0 +1,485 @@
+#
+# 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.
+#
+
+from pyspark.sql.connect.utils import check_dependencies
+
+check_dependencies(__name__)
+
+import sys
+from typing import cast, overload, Callable, Dict, List, Optional, TYPE_CHECKING, Union
+
+from pyspark.sql.connect.plan import DataSource, LogicalPlan, WriteStreamOperation
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.connect.readwriter import OptionUtils, to_str
+from pyspark.sql.connect.streaming.query import StreamingQuery
+from pyspark.sql.streaming.readwriter import (
+    DataStreamReader as PySparkDataStreamReader,
+    DataStreamWriter as PySparkDataStreamWriter,
+)
+from pyspark.sql.types import Row, StructType
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.dataframe import DataFrame
+
+__all__ = ["DataStreamReader", "DataStreamWriter"]
+
+
+class DataStreamReader(OptionUtils):
+
+    def __init__(self, client: "SparkSession") -> None:
+        self._format: Optional[str] = None
+        self._schema = ""
+        self._client = client
+        self._options: Dict[str, str] = {}
+
+    def _df(self, plan: LogicalPlan) -> "DataFrame":
+        from pyspark.sql.connect.dataframe import DataFrame
+
+        return DataFrame.withPlan(plan, self._client)
+
+    def format(self, source: str) -> "DataStreamReader":
+        self._format = source
+        return self
+
+    format.__doc__ = PySparkDataStreamReader.format.__doc__
+
+    def schema(self, schema: Union[StructType, str]) -> "DataStreamReader":
+        if isinstance(schema, StructType):
+            self._schema = schema.json()
+        elif isinstance(schema, str):
+            self._schema = schema
+        else:
+            raise TypeError("schema should be StructType or string")
+        return self
+
+    schema.__doc__ = PySparkDataStreamReader.schema.__doc__
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataStreamReader":
+        self._options[key] = str(value)
+        return self
+
+    option.__doc__ = PySparkDataStreamReader.option.__doc__
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataStreamReader":
+        for k in options:
+            self.option(k, to_str(options[k]))
+        return self
+
+    options.__doc__ = PySparkDataStreamReader.options.__doc__
+
+    def load(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> "DataFrame":
+        if format is not None:
+            self.format(format)
+        if schema is not None:
+            self.schema(schema)
+        self.options(**options)
+        if path is not None:
+            if type(path) != str or len(path.strip()) == 0:
+                raise ValueError(
+                    "If the path is provided for stream, it needs to be a "
+                    + "non-empty string. List of paths are not supported."
+                )
+
+        plan = DataSource(
+            format=self._format,
+            schema=self._schema,
+            options=self._options,
+            paths=[path] if path else None,
+            is_streaming=True
+        )
+
+        return self._df(plan)
+
+    load.__doc__ = PySparkDataStreamReader.load.__doc__
+
+    def json(
+        self,
+        path: str,
+        schema: Optional[Union[StructType, str]] = None,
+        primitivesAsString: Optional[Union[bool, str]] = None,
+        prefersDecimal: Optional[Union[bool, str]] = None,
+        allowComments: Optional[Union[bool, str]] = None,
+        allowUnquotedFieldNames: Optional[Union[bool, str]] = None,
+        allowSingleQuotes: Optional[Union[bool, str]] = None,
+        allowNumericLeadingZero: Optional[Union[bool, str]] = None,
+        allowBackslashEscapingAnyCharacter: Optional[Union[bool, str]] = None,
+        mode: Optional[str] = None,
+        columnNameOfCorruptRecord: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        multiLine: Optional[Union[bool, str]] = None,
+        allowUnquotedControlChars: Optional[Union[bool, str]] = None,
+        lineSep: Optional[str] = None,
+        locale: Optional[str] = None,
+        dropFieldIfAllNull: Optional[Union[bool, str]] = None,
+        encoding: Optional[str] = None,
+        pathGlobFilter: Optional[Union[bool, str]] = None,
+        recursiveFileLookup: Optional[Union[bool, str]] = None,
+        allowNonNumericNumbers: Optional[Union[bool, str]] = None,
+    ) -> "DataFrame":
+        self._set_opts(
+            schema=schema,
+            primitivesAsString=primitivesAsString,
+            prefersDecimal=prefersDecimal,
+            allowComments=allowComments,
+            allowUnquotedFieldNames=allowUnquotedFieldNames,
+            allowSingleQuotes=allowSingleQuotes,
+            allowNumericLeadingZero=allowNumericLeadingZero,
+            allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter,
+            mode=mode,
+            columnNameOfCorruptRecord=columnNameOfCorruptRecord,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            multiLine=multiLine,
+            allowUnquotedControlChars=allowUnquotedControlChars,
+            lineSep=lineSep,
+            locale=locale,
+            dropFieldIfAllNull=dropFieldIfAllNull,
+            encoding=encoding,
+            pathGlobFilter=pathGlobFilter,
+            recursiveFileLookup=recursiveFileLookup,
+            allowNonNumericNumbers=allowNonNumericNumbers,
+        )
+        if isinstance(path, str):
+            return self.load(path=path, format="json")
+        else:
+            raise TypeError("path can be only a single string")
+
+    json.__doc__ = PySparkDataStreamReader.json.__doc__
+
+    # def orc() TODO
+    # def parquet() TODO
+    # def text() TODO
+
+    def csv(
+        self,
+        path: str,
+        schema: Optional[Union[StructType, str]] = None,
+        sep: Optional[str] = None,
+        encoding: Optional[str] = None,
+        quote: Optional[str] = None,
+        escape: Optional[str] = None,
+        comment: Optional[str] = None,
+        header: Optional[Union[bool, str]] = None,
+        inferSchema: Optional[Union[bool, str]] = None,
+        ignoreLeadingWhiteSpace: Optional[Union[bool, str]] = None,
+        ignoreTrailingWhiteSpace: Optional[Union[bool, str]] = None,
+        nullValue: Optional[str] = None,
+        nanValue: Optional[str] = None,
+        positiveInf: Optional[str] = None,
+        negativeInf: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        maxColumns: Optional[Union[int, str]] = None,
+        maxCharsPerColumn: Optional[Union[int, str]] = None,
+        maxMalformedLogPerPartition: Optional[Union[int, str]] = None,
+        mode: Optional[str] = None,
+        columnNameOfCorruptRecord: Optional[str] = None,
+        multiLine: Optional[Union[bool, str]] = None,
+        charToEscapeQuoteEscaping: Optional[Union[bool, str]] = None,
+        enforceSchema: Optional[Union[bool, str]] = None,
+        emptyValue: Optional[str] = None,
+        locale: Optional[str] = None,
+        lineSep: Optional[str] = None,
+        pathGlobFilter: Optional[Union[bool, str]] = None,
+        recursiveFileLookup: Optional[Union[bool, str]] = None,
+        unescapedQuoteHandling: Optional[str] = None,
+    ) -> "DataFrame":
+        self._set_opts(
+            schema=schema,
+            sep=sep,
+            encoding=encoding,
+            quote=quote,
+            escape=escape,
+            comment=comment,
+            header=header,
+            inferSchema=inferSchema,
+            ignoreLeadingWhiteSpace=ignoreLeadingWhiteSpace,
+            ignoreTrailingWhiteSpace=ignoreTrailingWhiteSpace,
+            nullValue=nullValue,
+            nanValue=nanValue,
+            positiveInf=positiveInf,
+            negativeInf=negativeInf,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            maxColumns=maxColumns,
+            maxCharsPerColumn=maxCharsPerColumn,
+            maxMalformedLogPerPartition=maxMalformedLogPerPartition,
+            mode=mode,
+            columnNameOfCorruptRecord=columnNameOfCorruptRecord,
+            multiLine=multiLine,
+            charToEscapeQuoteEscaping=charToEscapeQuoteEscaping,
+            enforceSchema=enforceSchema,
+            emptyValue=emptyValue,
+            locale=locale,
+            lineSep=lineSep,
+            pathGlobFilter=pathGlobFilter,
+            recursiveFileLookup=recursiveFileLookup,
+            unescapedQuoteHandling=unescapedQuoteHandling,
+        )
+        if isinstance(path, str):
+            return self.load(path=path, format="csv")
+        else:
+            raise TypeError("path can be only a single string")
+
+    csv.__doc__ = PySparkDataStreamReader.csv.__doc__
+
+    # def table() TODO
+
+
+DataStreamReader.__doc__ = PySparkDataStreamReader.__doc__
+
+
+class DataStreamWriter:
+
+    def __init__(self, plan: "LogicalPlan", session: "SparkSession") -> None:
+        self._session = session
+        self._write_stream = WriteStreamOperation(plan)
+        self._write_proto = self._write_stream.write_op
+
+    def outputMode(self, outputMode: str) -> "DataStreamWriter":
+        self._write_proto.output_mode = outputMode
+        return self
+
+    outputMode.__doc__ = PySparkDataStreamWriter.outputMode.__doc__
+
+    def format(self, source: str) -> "DataStreamWriter":
+        self._write_proto.format = source
+        return self
+
+    format.__doc__ = PySparkDataStreamWriter.format.__doc__
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataStreamWriter":
+        self._write_proto.options[key] = to_str(value)
+        return self
+
+    option.__doc__ = PySparkDataStreamWriter.option.__doc__
+    
+    def options(self, **options: "OptionalPrimitiveType") -> "DataStreamWriter":
+        for k in options:
+            self.option(k, options[k])        
+        return self
+
+    options.__doc__ = PySparkDataStreamWriter.options.__doc__
+    
+    @overload
+    def partitionBy(self, *cols: str) -> "DataStreamWriter":
+        ...
+
+    @overload
+    def partitionBy(self, __cols: List[str]) -> "DataStreamWriter":
+        ...
+
+    def partitionBy(self, *cols: str) -> "DataStreamWriter":  # type: ignore[misc]
+        if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
+            cols = cols[0]
+        self._write_proto.partitioning_cols = cast(List[str], cols)
+        return self
+
+    partitionBy.__doc__ = PySparkDataStreamWriter.partitionBy.__doc__
+
+    def queryName(self, queryName: str) -> "DataStreamWriter":
+        self._write_proto.query_name = queryName
+        return self
+
+    queryName.__doc__ = PySparkDataStreamWriter.queryName.__doc__
+
+    @overload
+    def trigger(self, *, processingTime: str) -> "DataStreamWriter":
+        ...
+
+    @overload
+    def trigger(self, *, once: bool) -> "DataStreamWriter":
+        ...
+
+    @overload
+    def trigger(self, *, continuous: str) -> "DataStreamWriter":
+        ...
+
+    @overload
+    def trigger(self, *, availableNow: bool) -> "DataStreamWriter":
+        ...
+
+    def trigger(
+        self,
+        *,
+        processingTime: Optional[str] = None,
+        once: Optional[bool] = None,
+        continuous: Optional[str] = None,
+        availableNow: Optional[bool] = None,
+    ) -> "DataStreamWriter":
+        params = [processingTime, once, continuous, availableNow]
+
+        if params.count(None) == 4:
+            raise ValueError("No trigger provided")
+        elif params.count(None) < 3:
+            raise ValueError("Multiple triggers not allowed.")
+
+        if processingTime is not None:
+            if type(processingTime) != str or len(processingTime.strip()) == 0:
+                raise ValueError(
+                    "Value for processingTime must be a non empty string. Got: %s" % processingTime
+                )
+            self._write_proto.processing_time_interval = processingTime.strip()
+
+        elif once is not None:
+            if once is not True:
+                raise ValueError("Value for once must be True. Got: %s" % once)

Review Comment:
   This is copied from existing implementation. The interface is only for setting a trigger. `once=false` is not needed and can't be supported. 



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


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

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1152451343


##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -0,0 +1,173 @@
+#
+# 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.
+#
+
+import json
+import sys
+from typing import TYPE_CHECKING, Any, cast, Dict, List, Optional
+
+from pyspark.errors import StreamingQueryException
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.streaming.query import (
+    StreamingQuery as PySparkStreamingQuery,
+)
+
+__all__ = [
+    "StreamingQuery",  # TODO(WIP): "StreamingQueryManager"
+]
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+
+class StreamingQuery:
+
+    def __init__(self, session: "SparkSession", queryId: str, runId: str, name: Optional[str] = None) -> None:
+        self._session = session
+        self._id = queryId
+        self._runId = runId
+        self._name = name
+
+    @property
+    def id(self) -> str:
+        return self._id
+
+    id.__doc__ = PySparkStreamingQuery.id.__doc__
+
+    @property
+    def runId(self) -> str:
+        return self._runId
+
+    runId.__doc__ = PySparkStreamingQuery.runId.__doc__
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    name.__doc__ = PySparkStreamingQuery.name.__doc__
+
+    @property
+    def isActive(self) -> bool:
+        return self._fetch_status().is_active
+
+    isActive.__doc__ = PySparkStreamingQuery.isActive.__doc__
+
+    def awaitTermination(self, timeout: Optional[int] = None) -> Optional[bool]:
+        raise NotImplementedError()
+
+    awaitTermination.__doc__ = PySparkStreamingQuery.awaitTermination.__doc__
+
+    @property
+    def status(self) -> Dict[str, Any]:
+        proto = self._fetch_status()
+        return {
+            "message": proto.status_message,
+            "isDataAvailable": proto.is_data_available,
+            "isTriggerActive": proto.is_trigger_active
+        }
+
+    status.__doc__ = PySparkStreamingQuery.status.__doc__
+
+    @property
+    def recentProgress(self) -> List[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=-1).recent_progress_json)
+        return [json.loads(p) for p in progress]
+
+    recentProgress.__doc__ = PySparkStreamingQuery.recentProgress.__doc__
+
+    @property
+    def lastProgress(self) -> Optional[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=1).recent_progress_json)
+        if len(progress) > 0:
+            return json.loads(progress[-1])
+        else:
+            return None
+
+    lastProgress.__doc__ = PySparkStreamingQuery.lastProgress.__doc__
+
+    def processAllAvailable(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.process_all_available = True
+        self._execute_streaming_query_cmd(cmd)
+
+    processAllAvailable.__doc__ = PySparkStreamingQuery.processAllAvailable.__doc__
+
+    def stop(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.stop = True
+        self._execute_streaming_query_cmd(cmd)
+
+    stop.__doc__ = PySparkStreamingQuery.stop.__doc__
+
+    def explain(self, extended: bool = False) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.explain.extended = extended
+        result = self._execute_streaming_query_cmd(cmd).explain.result

Review Comment:
   I have TODO above in SparkConnectPlanner.scala on the server side. We will do something like that in a follow up. 
   
   ```
    // TODO(SPARK-42962): Handle this better. May be cache stopped queries for a few minutes.
   ```



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


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

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1156265721


##########
connector/connect/common/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -154,6 +158,9 @@ message Read {
     //
     // This is only supported by the JDBC data source.
     repeated string predicates = 5;
+
+    // (Optional) Source table name for a streaming read. Not used in batch read.
+    string streaming_table_name = 6;

Review Comment:
   I removed this field. I hadn't implemented `table()` in this branch. Will use `NamedTable` in our follow up PR to implement `table()`. 



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


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

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1157385141


##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,118 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+// Starts write stream operation as streaming query. Query ID and Run ID of the streaming
+// query are returned.
+message WriteStreamOperationStart {
+
+  // (Required) The output of the `input` streaming relation will be written.
+  Relation input = 1;
+
+  // The following fields directly map to API for DataStreamWriter().
+  // Consult API documentation unless explicitly documented here.
+
+  string format = 2;
+  map<string, string> options = 3;
+  repeated string partitioning_column_names = 4;
+
+  oneof trigger {
+    string processing_time_interval = 5;
+    bool available_now = 6;
+    bool one_time = 7;

Review Comment:
   Fixed. 'one_time' matched the name on the server side in streaming. `once` is better.



##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,118 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+// Starts write stream operation as streaming query. Query ID and Run ID of the streaming
+// query are returned.
+message WriteStreamOperationStart {
+
+  // (Required) The output of the `input` streaming relation will be written.
+  Relation input = 1;
+
+  // The following fields directly map to API for DataStreamWriter().
+  // Consult API documentation unless explicitly documented here.
+
+  string format = 2;
+  map<string, string> options = 3;
+  repeated string partitioning_column_names = 4;
+
+  oneof trigger {
+    string processing_time_interval = 5;
+    bool available_now = 6;
+    bool one_time = 7;
+    string continuous_checkpoint_interval = 8;
+  }
+
+  string output_mode = 9;
+  string query_name = 10;
+
+  // The destination is optional. When set, it can be a path or a table name.
+  oneof sink_destination {
+    string path = 11;
+    string table_name = 12;
+  }
+}
+
+message WriteStreamOperationStartResult {
+
+  // (Required)
+  string query_id = 1;
+
+  // (Required)
+  string run_id = 2;
+
+  // An optional query name.
+  string name = 3;
+
+  // TODO: How do we indicate errors?
+  // TODO: Consider adding StreamingQueryStatusResult here.
+}
+
+// Commands for a streaming query.
+message StreamingQueryCommand {
+
+  // (Required) query id of the streaming query.
+  string query_id = 1;
+  // (Required) run id of the streaming query.
+  string run_id = 2;
+
+  // A running query is identified by both run_id and query_id.
+
+  oneof command_type {
+    // Status of the query. Used to support multiple status related API like lastProgress().
+    StatusCommand status = 3;
+    // Stops the query.
+    bool stop = 4;
+    // Waits till all the available data is processed. See processAllAvailable() API doc.
+    bool process_all_available = 5;
+    // Returns logical and physical plans.
+    ExplainCommand explain = 6;
+
+    // TODO(SPARK-42960) Add more commands: await_termination(), exception() etc.
+  }
+
+  message StatusCommand {
+    // A limit on how many progress reports to return.
+    int32 recent_progress_limit = 1;
+  }
+
+  message ExplainCommand {
+    // TODO: Consider reusing Explain from AnalyzePlanRequest message.
+    //       We can not do this right now since it base.proto imports this file.
+    bool extended = 1;
+  }
+
+}
+
+// Response for commands on a streaming query.
+message StreamingQueryCommandResult {
+  // (Required)
+  string query_id = 1;

Review Comment:
   Let me move this to a 'RunningQueryId' that includes both these ids to avoid this issue. 



##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,118 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+// Starts write stream operation as streaming query. Query ID and Run ID of the streaming
+// query are returned.
+message WriteStreamOperationStart {
+
+  // (Required) The output of the `input` streaming relation will be written.
+  Relation input = 1;
+
+  // The following fields directly map to API for DataStreamWriter().
+  // Consult API documentation unless explicitly documented here.
+
+  string format = 2;
+  map<string, string> options = 3;
+  repeated string partitioning_column_names = 4;
+
+  oneof trigger {
+    string processing_time_interval = 5;
+    bool available_now = 6;
+    bool one_time = 7;
+    string continuous_checkpoint_interval = 8;
+  }
+
+  string output_mode = 9;
+  string query_name = 10;
+
+  // The destination is optional. When set, it can be a path or a table name.
+  oneof sink_destination {
+    string path = 11;
+    string table_name = 12;
+  }
+}
+
+message WriteStreamOperationStartResult {
+
+  // (Required)
+  string query_id = 1;
+
+  // (Required)
+  string run_id = 2;
+
+  // An optional query name.
+  string name = 3;
+
+  // TODO: How do we indicate errors?
+  // TODO: Consider adding StreamingQueryStatusResult here.
+}
+
+// Commands for a streaming query.
+message StreamingQueryCommand {
+
+  // (Required) query id of the streaming query.
+  string query_id = 1;
+  // (Required) run id of the streaming query.
+  string run_id = 2;
+
+  // A running query is identified by both run_id and query_id.
+
+  oneof command_type {
+    // Status of the query. Used to support multiple status related API like lastProgress().
+    StatusCommand status = 3;
+    // Stops the query.
+    bool stop = 4;
+    // Waits till all the available data is processed. See processAllAvailable() API doc.
+    bool process_all_available = 5;
+    // Returns logical and physical plans.
+    ExplainCommand explain = 6;
+
+    // TODO(SPARK-42960) Add more commands: await_termination(), exception() etc.
+  }
+
+  message StatusCommand {
+    // A limit on how many progress reports to return.
+    int32 recent_progress_limit = 1;
+  }
+
+  message ExplainCommand {
+    // TODO: Consider reusing Explain from AnalyzePlanRequest message.
+    //       We can not do this right now since it base.proto imports this file.
+    bool extended = 1;
+  }
+
+}
+
+// Response for commands on a streaming query.
+message StreamingQueryCommandResult {
+  // (Required)
+  string query_id = 1;
+
+  oneof result_type {
+    StatusResult status = 2;
+    ExplainResult explain = 3;
+  }
+
+  message StatusResult {
+    // This status includes all the available to status, including progress messages.
+
+    // Fields from Scala 'StreamingQueryStatus' struct
+    string status_message = 1;
+    bool is_data_available = 2;
+    bool is_trigger_active = 3;
+
+    bool is_active = 4;
+
+    // Progress reports as an array of json strings.
+    repeated string recent_progress_json = 5;

Review Comment:
   I have split these into separate commands.



##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -0,0 +1,181 @@
+#
+# 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.
+#
+
+import json
+import sys
+from typing import TYPE_CHECKING, Any, cast, Dict, List, Optional
+
+from pyspark.errors import StreamingQueryException
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.streaming.query import (
+    StreamingQuery as PySparkStreamingQuery,
+)
+
+__all__ = [
+    "StreamingQuery",  # TODO(WIP): "StreamingQueryManager"
+]
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+
+
+class StreamingQuery:
+    def __init__(
+        self, session: "SparkSession", queryId: str, runId: str, name: Optional[str] = None
+    ) -> None:
+        self._session = session
+        self._query_id = queryId
+        self._run_id = runId
+        self._name = name
+
+    @property
+    def id(self) -> str:
+        return self._query_id
+
+    id.__doc__ = PySparkStreamingQuery.id.__doc__
+
+    @property
+    def runId(self) -> str:
+        return self._run_id
+
+    runId.__doc__ = PySparkStreamingQuery.runId.__doc__
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    name.__doc__ = PySparkStreamingQuery.name.__doc__
+
+    @property
+    def isActive(self) -> bool:
+        return self._fetch_status().is_active
+
+    isActive.__doc__ = PySparkStreamingQuery.isActive.__doc__
+
+    def awaitTermination(self, timeout: Optional[int] = None) -> Optional[bool]:
+        raise NotImplementedError()
+
+    awaitTermination.__doc__ = PySparkStreamingQuery.awaitTermination.__doc__
+
+    @property
+    def status(self) -> Dict[str, Any]:
+        proto = self._fetch_status()
+        return {
+            "message": proto.status_message,
+            "isDataAvailable": proto.is_data_available,
+            "isTriggerActive": proto.is_trigger_active,
+        }
+
+    status.__doc__ = PySparkStreamingQuery.status.__doc__
+
+    @property
+    def recentProgress(self) -> List[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=10000).recent_progress_json)
+        # Server only keeps 100, so 10000 limit is high enough.
+        return [json.loads(p) for p in progress]
+
+    recentProgress.__doc__ = PySparkStreamingQuery.recentProgress.__doc__
+
+    @property
+    def lastProgress(self) -> Optional[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=1).recent_progress_json)
+        if len(progress) > 0:
+            return json.loads(progress[-1])
+        else:
+            return None
+
+    lastProgress.__doc__ = PySparkStreamingQuery.lastProgress.__doc__
+
+    def processAllAvailable(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.process_all_available = True
+        self._execute_streaming_query_cmd(cmd)
+
+    processAllAvailable.__doc__ = PySparkStreamingQuery.processAllAvailable.__doc__
+
+    def stop(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.stop = True
+        self._execute_streaming_query_cmd(cmd)
+
+    stop.__doc__ = PySparkStreamingQuery.stop.__doc__
+
+    def explain(self, extended: bool = False) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.explain.extended = extended
+        result = self._execute_streaming_query_cmd(cmd).explain.result
+        print(result)
+
+    explain.__doc__ = PySparkStreamingQuery.explain.__doc__
+
+    def exception(self) -> Optional[StreamingQueryException]:
+        raise NotImplementedError()
+
+    exception.__doc__ = PySparkStreamingQuery.exception.__doc__
+
+    def _fetch_status(
+        self, recent_progress_limit=0
+    ) -> pb2.StreamingQueryCommandResult.StatusResult:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.status.recent_progress_limit = recent_progress_limit
+
+        return self._execute_streaming_query_cmd(cmd).status
+
+    def _execute_streaming_query_cmd(
+        self, cmd: pb2.StreamingQueryCommand
+    ) -> pb2.StreamingQueryCommandResult:
+        cmd.query_id = self._query_id
+        cmd.run_id = self._run_id
+        exec_cmd = pb2.Command()
+        exec_cmd.streaming_query_command.CopyFrom(cmd)
+        (_, properties) = self._session.client.execute_command(exec_cmd)
+        return cast(pb2.StreamingQueryCommandResult, properties["streaming_query_command_result"])
+
+
+# TODO(WIP) class StreamingQueryManager:
+
+
+def _test() -> None:
+    import doctest
+    import os
+    from pyspark.sql import SparkSession
+    import pyspark.sql.streaming.query

Review Comment:
   I removed this implementation with a TODO:
   ```  
      # TODO(SPARK-43031): port _test() from legacy query.py.
   ```



##########
python/pyspark/sql/dataframe.py:
##########
@@ -529,14 +529,10 @@ def writeStream(self) -> DataStreamWriter:
         --------
         >>> import tempfile
         >>> df = spark.readStream.format("rate").load()
-        >>> type(df.writeStream)

Review Comment:
   Nice. Done.



##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -0,0 +1,181 @@
+#
+# 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.
+#
+
+import json
+import sys
+from typing import TYPE_CHECKING, Any, cast, Dict, List, Optional
+
+from pyspark.errors import StreamingQueryException
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.streaming.query import (
+    StreamingQuery as PySparkStreamingQuery,
+)
+
+__all__ = [
+    "StreamingQuery",  # TODO(WIP): "StreamingQueryManager"

Review Comment:
   Done. TODO(SPARK-43032). 



##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -0,0 +1,181 @@
+#
+# 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.
+#
+
+import json
+import sys
+from typing import TYPE_CHECKING, Any, cast, Dict, List, Optional
+
+from pyspark.errors import StreamingQueryException
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.streaming.query import (
+    StreamingQuery as PySparkStreamingQuery,
+)
+
+__all__ = [
+    "StreamingQuery",  # TODO(WIP): "StreamingQueryManager"
+]
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+
+
+class StreamingQuery:
+    def __init__(
+        self, session: "SparkSession", queryId: str, runId: str, name: Optional[str] = None
+    ) -> None:
+        self._session = session
+        self._query_id = queryId
+        self._run_id = runId
+        self._name = name
+
+    @property
+    def id(self) -> str:
+        return self._query_id
+
+    id.__doc__ = PySparkStreamingQuery.id.__doc__
+
+    @property
+    def runId(self) -> str:
+        return self._run_id
+
+    runId.__doc__ = PySparkStreamingQuery.runId.__doc__
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    name.__doc__ = PySparkStreamingQuery.name.__doc__
+
+    @property
+    def isActive(self) -> bool:
+        return self._fetch_status().is_active
+
+    isActive.__doc__ = PySparkStreamingQuery.isActive.__doc__
+
+    def awaitTermination(self, timeout: Optional[int] = None) -> Optional[bool]:
+        raise NotImplementedError()
+
+    awaitTermination.__doc__ = PySparkStreamingQuery.awaitTermination.__doc__
+
+    @property
+    def status(self) -> Dict[str, Any]:
+        proto = self._fetch_status()
+        return {
+            "message": proto.status_message,
+            "isDataAvailable": proto.is_data_available,
+            "isTriggerActive": proto.is_trigger_active,
+        }
+
+    status.__doc__ = PySparkStreamingQuery.status.__doc__
+
+    @property
+    def recentProgress(self) -> List[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=10000).recent_progress_json)
+        # Server only keeps 100, so 10000 limit is high enough.
+        return [json.loads(p) for p in progress]
+
+    recentProgress.__doc__ = PySparkStreamingQuery.recentProgress.__doc__
+
+    @property
+    def lastProgress(self) -> Optional[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=1).recent_progress_json)
+        if len(progress) > 0:
+            return json.loads(progress[-1])
+        else:
+            return None
+
+    lastProgress.__doc__ = PySparkStreamingQuery.lastProgress.__doc__
+
+    def processAllAvailable(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.process_all_available = True
+        self._execute_streaming_query_cmd(cmd)
+
+    processAllAvailable.__doc__ = PySparkStreamingQuery.processAllAvailable.__doc__
+
+    def stop(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.stop = True
+        self._execute_streaming_query_cmd(cmd)
+
+    stop.__doc__ = PySparkStreamingQuery.stop.__doc__
+
+    def explain(self, extended: bool = False) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.explain.extended = extended
+        result = self._execute_streaming_query_cmd(cmd).explain.result
+        print(result)
+
+    explain.__doc__ = PySparkStreamingQuery.explain.__doc__
+
+    def exception(self) -> Optional[StreamingQueryException]:
+        raise NotImplementedError()
+
+    exception.__doc__ = PySparkStreamingQuery.exception.__doc__
+
+    def _fetch_status(
+        self, recent_progress_limit=0
+    ) -> pb2.StreamingQueryCommandResult.StatusResult:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.status.recent_progress_limit = recent_progress_limit
+
+        return self._execute_streaming_query_cmd(cmd).status
+
+    def _execute_streaming_query_cmd(
+        self, cmd: pb2.StreamingQueryCommand
+    ) -> pb2.StreamingQueryCommandResult:
+        cmd.query_id = self._query_id
+        cmd.run_id = self._run_id
+        exec_cmd = pb2.Command()
+        exec_cmd.streaming_query_command.CopyFrom(cmd)
+        (_, properties) = self._session.client.execute_command(exec_cmd)
+        return cast(pb2.StreamingQueryCommandResult, properties["streaming_query_command_result"])
+
+
+# TODO(WIP) class StreamingQueryManager:
+
+
+def _test() -> None:
+    import doctest
+    import os
+    from pyspark.sql import SparkSession
+    import pyspark.sql.streaming.query
+    from py4j.protocol import Py4JError
+
+    os.chdir(os.environ["SPARK_HOME"])
+
+    globs = pyspark.sql.streaming.query.__dict__.copy()

Review Comment:
   Same as above. We will port these in follow up. SPARK-43031
   Removed the code for now. 



##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -0,0 +1,181 @@
+#
+# 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.
+#
+
+import json
+import sys
+from typing import TYPE_CHECKING, Any, cast, Dict, List, Optional
+
+from pyspark.errors import StreamingQueryException
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.streaming.query import (
+    StreamingQuery as PySparkStreamingQuery,
+)
+
+__all__ = [
+    "StreamingQuery",  # TODO(WIP): "StreamingQueryManager"
+]
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+
+
+class StreamingQuery:
+    def __init__(
+        self, session: "SparkSession", queryId: str, runId: str, name: Optional[str] = None
+    ) -> None:
+        self._session = session
+        self._query_id = queryId
+        self._run_id = runId
+        self._name = name
+
+    @property
+    def id(self) -> str:
+        return self._query_id
+
+    id.__doc__ = PySparkStreamingQuery.id.__doc__
+
+    @property
+    def runId(self) -> str:
+        return self._run_id
+
+    runId.__doc__ = PySparkStreamingQuery.runId.__doc__
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    name.__doc__ = PySparkStreamingQuery.name.__doc__
+
+    @property
+    def isActive(self) -> bool:
+        return self._fetch_status().is_active
+
+    isActive.__doc__ = PySparkStreamingQuery.isActive.__doc__
+
+    def awaitTermination(self, timeout: Optional[int] = None) -> Optional[bool]:
+        raise NotImplementedError()
+
+    awaitTermination.__doc__ = PySparkStreamingQuery.awaitTermination.__doc__
+
+    @property
+    def status(self) -> Dict[str, Any]:
+        proto = self._fetch_status()
+        return {
+            "message": proto.status_message,
+            "isDataAvailable": proto.is_data_available,
+            "isTriggerActive": proto.is_trigger_active,
+        }
+
+    status.__doc__ = PySparkStreamingQuery.status.__doc__
+
+    @property
+    def recentProgress(self) -> List[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=10000).recent_progress_json)
+        # Server only keeps 100, so 10000 limit is high enough.
+        return [json.loads(p) for p in progress]
+
+    recentProgress.__doc__ = PySparkStreamingQuery.recentProgress.__doc__
+
+    @property
+    def lastProgress(self) -> Optional[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=1).recent_progress_json)
+        if len(progress) > 0:
+            return json.loads(progress[-1])
+        else:
+            return None
+
+    lastProgress.__doc__ = PySparkStreamingQuery.lastProgress.__doc__
+
+    def processAllAvailable(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.process_all_available = True
+        self._execute_streaming_query_cmd(cmd)
+
+    processAllAvailable.__doc__ = PySparkStreamingQuery.processAllAvailable.__doc__
+
+    def stop(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.stop = True
+        self._execute_streaming_query_cmd(cmd)
+
+    stop.__doc__ = PySparkStreamingQuery.stop.__doc__
+
+    def explain(self, extended: bool = False) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.explain.extended = extended
+        result = self._execute_streaming_query_cmd(cmd).explain.result
+        print(result)
+
+    explain.__doc__ = PySparkStreamingQuery.explain.__doc__
+
+    def exception(self) -> Optional[StreamingQueryException]:
+        raise NotImplementedError()
+
+    exception.__doc__ = PySparkStreamingQuery.exception.__doc__
+
+    def _fetch_status(
+        self, recent_progress_limit=0
+    ) -> pb2.StreamingQueryCommandResult.StatusResult:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.status.recent_progress_limit = recent_progress_limit
+
+        return self._execute_streaming_query_cmd(cmd).status
+
+    def _execute_streaming_query_cmd(
+        self, cmd: pb2.StreamingQueryCommand
+    ) -> pb2.StreamingQueryCommandResult:
+        cmd.query_id = self._query_id
+        cmd.run_id = self._run_id
+        exec_cmd = pb2.Command()
+        exec_cmd.streaming_query_command.CopyFrom(cmd)
+        (_, properties) = self._session.client.execute_command(exec_cmd)
+        return cast(pb2.StreamingQueryCommandResult, properties["streaming_query_command_result"])
+
+
+# TODO(WIP) class StreamingQueryManager:
+
+
+def _test() -> None:
+    import doctest
+    import os
+    from pyspark.sql import SparkSession
+    import pyspark.sql.streaming.query
+    from py4j.protocol import Py4JError
+
+    os.chdir(os.environ["SPARK_HOME"])
+
+    globs = pyspark.sql.streaming.query.__dict__.copy()
+    try:
+        spark = SparkSession._getActiveSessionOrCreate()

Review Comment:
   Noted. Will do it in SPARK-43031



##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -0,0 +1,181 @@
+#

Review Comment:
   Thanks. Added TODO(SPARK-43031). @WweiL looking into enabling these tests.



##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -0,0 +1,181 @@
+#
+# 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.
+#
+
+import json
+import sys
+from typing import TYPE_CHECKING, Any, cast, Dict, List, Optional
+
+from pyspark.errors import StreamingQueryException
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.streaming.query import (
+    StreamingQuery as PySparkStreamingQuery,
+)
+
+__all__ = [
+    "StreamingQuery",  # TODO(WIP): "StreamingQueryManager"
+]
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+
+
+class StreamingQuery:
+    def __init__(
+        self, session: "SparkSession", queryId: str, runId: str, name: Optional[str] = None
+    ) -> None:
+        self._session = session
+        self._query_id = queryId
+        self._run_id = runId
+        self._name = name
+
+    @property
+    def id(self) -> str:
+        return self._query_id
+
+    id.__doc__ = PySparkStreamingQuery.id.__doc__
+
+    @property
+    def runId(self) -> str:
+        return self._run_id
+
+    runId.__doc__ = PySparkStreamingQuery.runId.__doc__
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    name.__doc__ = PySparkStreamingQuery.name.__doc__
+
+    @property
+    def isActive(self) -> bool:
+        return self._fetch_status().is_active
+
+    isActive.__doc__ = PySparkStreamingQuery.isActive.__doc__
+
+    def awaitTermination(self, timeout: Optional[int] = None) -> Optional[bool]:
+        raise NotImplementedError()
+
+    awaitTermination.__doc__ = PySparkStreamingQuery.awaitTermination.__doc__
+
+    @property
+    def status(self) -> Dict[str, Any]:
+        proto = self._fetch_status()
+        return {
+            "message": proto.status_message,
+            "isDataAvailable": proto.is_data_available,
+            "isTriggerActive": proto.is_trigger_active,
+        }
+
+    status.__doc__ = PySparkStreamingQuery.status.__doc__
+
+    @property
+    def recentProgress(self) -> List[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=10000).recent_progress_json)
+        # Server only keeps 100, so 10000 limit is high enough.
+        return [json.loads(p) for p in progress]
+
+    recentProgress.__doc__ = PySparkStreamingQuery.recentProgress.__doc__
+
+    @property
+    def lastProgress(self) -> Optional[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=1).recent_progress_json)
+        if len(progress) > 0:
+            return json.loads(progress[-1])
+        else:
+            return None
+
+    lastProgress.__doc__ = PySparkStreamingQuery.lastProgress.__doc__
+
+    def processAllAvailable(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.process_all_available = True
+        self._execute_streaming_query_cmd(cmd)
+
+    processAllAvailable.__doc__ = PySparkStreamingQuery.processAllAvailable.__doc__
+
+    def stop(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.stop = True
+        self._execute_streaming_query_cmd(cmd)
+
+    stop.__doc__ = PySparkStreamingQuery.stop.__doc__
+
+    def explain(self, extended: bool = False) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.explain.extended = extended
+        result = self._execute_streaming_query_cmd(cmd).explain.result
+        print(result)
+
+    explain.__doc__ = PySparkStreamingQuery.explain.__doc__
+
+    def exception(self) -> Optional[StreamingQueryException]:
+        raise NotImplementedError()
+
+    exception.__doc__ = PySparkStreamingQuery.exception.__doc__
+
+    def _fetch_status(
+        self, recent_progress_limit=0
+    ) -> pb2.StreamingQueryCommandResult.StatusResult:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.status.recent_progress_limit = recent_progress_limit
+
+        return self._execute_streaming_query_cmd(cmd).status
+
+    def _execute_streaming_query_cmd(
+        self, cmd: pb2.StreamingQueryCommand
+    ) -> pb2.StreamingQueryCommandResult:
+        cmd.query_id = self._query_id
+        cmd.run_id = self._run_id
+        exec_cmd = pb2.Command()
+        exec_cmd.streaming_query_command.CopyFrom(cmd)
+        (_, properties) = self._session.client.execute_command(exec_cmd)
+        return cast(pb2.StreamingQueryCommandResult, properties["streaming_query_command_result"])
+
+
+# TODO(WIP) class StreamingQueryManager:
+
+
+def _test() -> None:
+    import doctest
+    import os
+    from pyspark.sql import SparkSession
+    import pyspark.sql.streaming.query
+    from py4j.protocol import Py4JError
+
+    os.chdir(os.environ["SPARK_HOME"])
+
+    globs = pyspark.sql.streaming.query.__dict__.copy()
+    try:
+        spark = SparkSession._getActiveSessionOrCreate()
+    except Py4JError:  # noqa: F821
+        spark = SparkSession(sc)  # type: ignore[name-defined] # noqa: F821
+
+    globs["spark"] = spark
+
+    (failure_count, test_count) = doctest.testmod(
+        pyspark.sql.streaming.query,

Review Comment:
   Same as above. will address in SPARK-43031. 



##########
python/pyspark/sql/dataframe.py:
##########
@@ -529,14 +529,10 @@ def writeStream(self) -> DataStreamWriter:
         --------
         >>> import tempfile
         >>> df = spark.readStream.format("rate").load()
-        >>> type(df.writeStream)
-        <class 'pyspark.sql.streaming.readwriter.DataStreamWriter'>
-
         >>> with tempfile.TemporaryDirectory() as d:
         ...     # Create a table with Rate source.
-        ...     df.writeStream.toTable(
+        ...     streaming_query = df.writeStream.toTable(
         ...         "my_table", checkpointLocation=d) # doctest: +ELLIPSIS
-        <pyspark.sql.streaming.query.StreamingQuery object at 0x...>

Review Comment:
   Done.



##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -0,0 +1,181 @@
+#
+# 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.
+#
+
+import json
+import sys
+from typing import TYPE_CHECKING, Any, cast, Dict, List, Optional
+
+from pyspark.errors import StreamingQueryException
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.streaming.query import (
+    StreamingQuery as PySparkStreamingQuery,
+)
+
+__all__ = [
+    "StreamingQuery",  # TODO(WIP): "StreamingQueryManager"
+]
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+
+
+class StreamingQuery:
+    def __init__(
+        self, session: "SparkSession", queryId: str, runId: str, name: Optional[str] = None
+    ) -> None:
+        self._session = session
+        self._query_id = queryId
+        self._run_id = runId
+        self._name = name
+
+    @property
+    def id(self) -> str:
+        return self._query_id
+
+    id.__doc__ = PySparkStreamingQuery.id.__doc__
+
+    @property
+    def runId(self) -> str:
+        return self._run_id
+
+    runId.__doc__ = PySparkStreamingQuery.runId.__doc__
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    name.__doc__ = PySparkStreamingQuery.name.__doc__
+
+    @property
+    def isActive(self) -> bool:
+        return self._fetch_status().is_active
+
+    isActive.__doc__ = PySparkStreamingQuery.isActive.__doc__
+
+    def awaitTermination(self, timeout: Optional[int] = None) -> Optional[bool]:
+        raise NotImplementedError()
+
+    awaitTermination.__doc__ = PySparkStreamingQuery.awaitTermination.__doc__
+
+    @property
+    def status(self) -> Dict[str, Any]:
+        proto = self._fetch_status()
+        return {
+            "message": proto.status_message,
+            "isDataAvailable": proto.is_data_available,
+            "isTriggerActive": proto.is_trigger_active,
+        }
+
+    status.__doc__ = PySparkStreamingQuery.status.__doc__
+
+    @property
+    def recentProgress(self) -> List[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=10000).recent_progress_json)
+        # Server only keeps 100, so 10000 limit is high enough.
+        return [json.loads(p) for p in progress]
+
+    recentProgress.__doc__ = PySparkStreamingQuery.recentProgress.__doc__
+
+    @property
+    def lastProgress(self) -> Optional[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=1).recent_progress_json)
+        if len(progress) > 0:
+            return json.loads(progress[-1])
+        else:
+            return None
+
+    lastProgress.__doc__ = PySparkStreamingQuery.lastProgress.__doc__
+
+    def processAllAvailable(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.process_all_available = True
+        self._execute_streaming_query_cmd(cmd)
+
+    processAllAvailable.__doc__ = PySparkStreamingQuery.processAllAvailable.__doc__
+
+    def stop(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.stop = True
+        self._execute_streaming_query_cmd(cmd)
+
+    stop.__doc__ = PySparkStreamingQuery.stop.__doc__
+
+    def explain(self, extended: bool = False) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.explain.extended = extended
+        result = self._execute_streaming_query_cmd(cmd).explain.result
+        print(result)
+
+    explain.__doc__ = PySparkStreamingQuery.explain.__doc__
+
+    def exception(self) -> Optional[StreamingQueryException]:
+        raise NotImplementedError()
+
+    exception.__doc__ = PySparkStreamingQuery.exception.__doc__
+
+    def _fetch_status(
+        self, recent_progress_limit=0
+    ) -> pb2.StreamingQueryCommandResult.StatusResult:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.status.recent_progress_limit = recent_progress_limit
+
+        return self._execute_streaming_query_cmd(cmd).status
+
+    def _execute_streaming_query_cmd(
+        self, cmd: pb2.StreamingQueryCommand
+    ) -> pb2.StreamingQueryCommandResult:
+        cmd.query_id = self._query_id
+        cmd.run_id = self._run_id
+        exec_cmd = pb2.Command()
+        exec_cmd.streaming_query_command.CopyFrom(cmd)
+        (_, properties) = self._session.client.execute_command(exec_cmd)
+        return cast(pb2.StreamingQueryCommandResult, properties["streaming_query_command_result"])
+
+
+# TODO(WIP) class StreamingQueryManager:
+
+
+def _test() -> None:
+    import doctest
+    import os
+    from pyspark.sql import SparkSession
+    import pyspark.sql.streaming.query
+    from py4j.protocol import Py4JError
+
+    os.chdir(os.environ["SPARK_HOME"])
+
+    globs = pyspark.sql.streaming.query.__dict__.copy()
+    try:
+        spark = SparkSession._getActiveSessionOrCreate()

Review Comment:
   Noted. 
   FYI: @WweiL 



##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,118 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+// Starts write stream operation as streaming query. Query ID and Run ID of the streaming
+// query are returned.
+message WriteStreamOperationStart {
+
+  // (Required) The output of the `input` streaming relation will be written.
+  Relation input = 1;
+
+  // The following fields directly map to API for DataStreamWriter().
+  // Consult API documentation unless explicitly documented here.
+
+  string format = 2;
+  map<string, string> options = 3;
+  repeated string partitioning_column_names = 4;
+
+  oneof trigger {
+    string processing_time_interval = 5;
+    bool available_now = 6;
+    bool one_time = 7;
+    string continuous_checkpoint_interval = 8;
+  }
+
+  string output_mode = 9;
+  string query_name = 10;
+
+  // The destination is optional. When set, it can be a path or a table name.
+  oneof sink_destination {
+    string path = 11;
+    string table_name = 12;
+  }
+}
+
+message WriteStreamOperationStartResult {
+
+  // (Required)
+  string query_id = 1;
+
+  // (Required)
+  string run_id = 2;
+
+  // An optional query name.
+  string name = 3;
+
+  // TODO: How do we indicate errors?
+  // TODO: Consider adding StreamingQueryStatusResult here.
+}
+
+// Commands for a streaming query.
+message StreamingQueryCommand {
+
+  // (Required) query id of the streaming query.
+  string query_id = 1;
+  // (Required) run id of the streaming query.
+  string run_id = 2;
+
+  // A running query is identified by both run_id and query_id.
+
+  oneof command_type {
+    // Status of the query. Used to support multiple status related API like lastProgress().
+    StatusCommand status = 3;
+    // Stops the query.
+    bool stop = 4;
+    // Waits till all the available data is processed. See processAllAvailable() API doc.
+    bool process_all_available = 5;
+    // Returns logical and physical plans.
+    ExplainCommand explain = 6;
+
+    // TODO(SPARK-42960) Add more commands: await_termination(), exception() etc.
+  }
+
+  message StatusCommand {
+    // A limit on how many progress reports to return.
+    int32 recent_progress_limit = 1;
+  }
+
+  message ExplainCommand {
+    // TODO: Consider reusing Explain from AnalyzePlanRequest message.
+    //       We can not do this right now since it base.proto imports this file.
+    bool extended = 1;
+  }
+
+}
+
+// Response for commands on a streaming query.
+message StreamingQueryCommandResult {
+  // (Required)
+  string query_id = 1;

Review Comment:
   Fixed. Added 'StreamingQueryInstanceId' struct and we use it in multiple places.



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -1969,6 +2017,142 @@ class SparkConnectPlanner(val session: SparkSession) {
     }
   }
 
+  def handleWriteStreamOperationStart(
+      writeOp: WriteStreamOperationStart,
+      sessionId: String,
+      responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
+    val plan = transformRelation(writeOp.getInput)
+    val dataset = Dataset.ofRows(session, logicalPlan = plan)
+
+    val writer = dataset.writeStream
+
+    if (writeOp.getFormat.nonEmpty) {
+      writer.format(writeOp.getFormat)
+    }
+
+    writer.options(writeOp.getOptionsMap)
+
+    if (writeOp.getPartitioningColumnNamesCount > 0) {
+      writer.partitionBy(writeOp.getPartitioningColumnNamesList.asScala.toList: _*)
+    }
+
+    writeOp.getTriggerCase match {
+      case TriggerCase.PROCESSING_TIME_INTERVAL =>
+        writer.trigger(Trigger.ProcessingTime(writeOp.getProcessingTimeInterval))
+      case TriggerCase.AVAILABLE_NOW =>
+        writer.trigger(Trigger.AvailableNow())
+      case TriggerCase.ONE_TIME =>
+        writer.trigger(Trigger.Once())
+      case TriggerCase.CONTINUOUS_CHECKPOINT_INTERVAL =>
+        writer.trigger(Trigger.Continuous(writeOp.getContinuousCheckpointInterval))
+      case TriggerCase.TRIGGER_NOT_SET =>
+    }
+
+    if (writeOp.getOutputMode.nonEmpty) {
+      writer.outputMode(writeOp.getOutputMode)
+    }
+
+    if (writeOp.getQueryName.nonEmpty) {
+      writer.queryName(writeOp.getQueryName)
+    }
+
+    val query = writeOp.getPath match {
+      case "" if writeOp.hasTableName => writer.toTable(writeOp.getTableName)
+      case "" => writer.start()
+      case path => writer.start(path)
+    }
+
+    val result = WriteStreamOperationStartResult
+      .newBuilder()
+      .setQueryId(query.id.toString)
+      .setRunId(query.runId.toString)
+      .setName(Option(query.name).getOrElse(""))
+      .build()
+
+    responseObserver.onNext(
+      ExecutePlanResponse
+        .newBuilder()
+        .setSessionId(sessionId)
+        .setWriteStreamOperationStartResult(result)
+        .build())
+  }
+
+  def handleStreamingQueryCommand(
+      command: StreamingQueryCommand,
+      sessionId: String,
+      responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
+
+    val queryId = command.getQueryId
+
+    val respBuilder = StreamingQueryCommandResult
+      .newBuilder()
+      .setQueryId(command.getQueryId)
+
+    val query = Option(session.streams.get(queryId)) match {
+      case Some(query) if query.runId.toString == command.getRunId =>
+        query
+      case Some(query) =>
+        throw new IllegalArgumentException(
+          s"Run id mismatch for query id $queryId. Run id in the request ${command.getRunId} " +
+            s"does not match one on the server ${query.runId}. The query might have restarted.")
+      case None =>
+        throw new IllegalArgumentException(s"Streaming query $queryId is not found")
+      // TODO(SPARK-42962): Handle this better. May be cache stopped queries for a few minutes.
+    }
+
+    command.getCommandTypeCase match {
+      case StreamingQueryCommand.CommandTypeCase.STATUS =>
+        val recentProgress: Seq[String] = command.getStatus.getRecentProgressLimit match {
+          case 0 => Seq.empty
+          case limit if limit < 0 =>
+            query.recentProgress.map(_.json) // All the cached progresses.
+          case limit => query.recentProgress.takeRight(limit).map(_.json) // Most recent
+        }
+
+        val queryStatus = query.status
+
+        val statusResult = StreamingQueryCommandResult.StatusResult
+          .newBuilder()
+          .setStatusMessage(queryStatus.message)
+          .setIsDataAvailable(queryStatus.isDataAvailable)
+          .setIsTriggerActive(queryStatus.isTriggerActive)
+          .setIsActive(query.isActive)
+          .addAllRecentProgressJson(recentProgress.asJava)
+          .build()
+
+        respBuilder.setStatus(statusResult)
+
+      case StreamingQueryCommand.CommandTypeCase.STOP =>
+        query.stop()
+
+      case StreamingQueryCommand.CommandTypeCase.PROCESS_ALL_AVAILABLE =>
+        query.processAllAvailable()

Review Comment:
   Updated the comment. RPC will stay active (because client sends heartbeats to keep connection alive). The issue could session getting closed. That will be handled separately. 



##########
python/pyspark/sql/connect/readwriter.py:
##########
@@ -37,7 +37,7 @@
     from pyspark.sql.connect._typing import ColumnOrName, OptionalPrimitiveType
     from pyspark.sql.connect.session import SparkSession
 
-__all__ = ["DataFrameReader", "DataFrameWriter"]
+__all__ = ["DataFrameReader", "DataFrameWriter", "OptionUtils", "to_str"]

Review Comment:
   Thats nice. Removed the exports. 



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


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

Posted by "pengzhon-db (via GitHub)" <gi...@apache.org>.
pengzhon-db commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1157815748


##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -0,0 +1,161 @@
+#
+# 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.
+#
+
+import json
+import sys
+from typing import TYPE_CHECKING, Any, cast, Dict, List, Optional
+
+from pyspark.errors import StreamingQueryException
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.streaming.query import (
+    StreamingQuery as PySparkStreamingQuery,
+)
+
+__all__ = [
+    "StreamingQuery",  # TODO(SPARK-43032): "StreamingQueryManager"
+]
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+
+
+class StreamingQuery:
+    def __init__(
+        self, session: "SparkSession", queryId: str, runId: str, name: Optional[str] = None
+    ) -> None:
+        self._session = session
+        self._query_id = queryId
+        self._run_id = runId
+        self._name = name
+
+    @property
+    def id(self) -> str:
+        return self._query_id
+
+    id.__doc__ = PySparkStreamingQuery.id.__doc__
+
+    @property
+    def runId(self) -> str:
+        return self._run_id
+
+    runId.__doc__ = PySparkStreamingQuery.runId.__doc__
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    name.__doc__ = PySparkStreamingQuery.name.__doc__
+
+    @property
+    def isActive(self) -> bool:
+        return self._fetch_status().is_active
+
+    isActive.__doc__ = PySparkStreamingQuery.isActive.__doc__
+
+    def awaitTermination(self, timeout: Optional[int] = None) -> Optional[bool]:
+        raise NotImplementedError()
+
+    awaitTermination.__doc__ = PySparkStreamingQuery.awaitTermination.__doc__
+
+    @property
+    def status(self) -> Dict[str, Any]:
+        proto = self._fetch_status()
+        return {
+            "message": proto.status_message,
+            "isDataAvailable": proto.is_data_available,
+            "isTriggerActive": proto.is_trigger_active,
+        }
+
+    status.__doc__ = PySparkStreamingQuery.status.__doc__
+
+    @property
+    def recentProgress(self) -> List[Dict[str, Any]]:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.recent_progress = True
+        progress = self._execute_streaming_query_cmd(cmd).recent_progress.recent_progress_json
+        return [json.loads(p) for p in progress]
+
+    recentProgress.__doc__ = PySparkStreamingQuery.recentProgress.__doc__
+
+    @property
+    def lastProgress(self) -> Optional[Dict[str, Any]]:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.last_progress = True
+        progress = self._execute_streaming_query_cmd(cmd).recent_progress.recent_progress_json
+        if len(progress) > 0:
+            return json.loads(progress[-1])
+        else:
+            return None
+
+    lastProgress.__doc__ = PySparkStreamingQuery.lastProgress.__doc__
+
+    def processAllAvailable(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.process_all_available = True
+        self._execute_streaming_query_cmd(cmd)
+
+    processAllAvailable.__doc__ = PySparkStreamingQuery.processAllAvailable.__doc__
+
+    def stop(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.stop = True
+        self._execute_streaming_query_cmd(cmd)
+
+    stop.__doc__ = PySparkStreamingQuery.stop.__doc__
+
+    def explain(self, extended: bool = False) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.explain.extended = extended
+        result = self._execute_streaming_query_cmd(cmd).explain.result
+        print(result)
+
+    explain.__doc__ = PySparkStreamingQuery.explain.__doc__
+
+    def exception(self) -> Optional[StreamingQueryException]:
+        raise NotImplementedError()
+
+    exception.__doc__ = PySparkStreamingQuery.exception.__doc__
+
+    def _fetch_status(
+        self, recent_progress_limit=0

Review Comment:
   `recent_progress_limit` is not used



##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,126 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+// Starts write stream operation as streaming query. Query ID and Run ID of the streaming
+// query are returned.
+message WriteStreamOperationStart {
+
+  // (Required) The output of the `input` streaming relation will be written.
+  Relation input = 1;
+
+  // The following fields directly map to API for DataStreamWriter().
+  // Consult API documentation unless explicitly documented here.
+
+  string format = 2;
+  map<string, string> options = 3;
+  repeated string partitioning_column_names = 4;
+
+  oneof trigger {
+    string processing_time_interval = 5;
+    bool available_now = 6;
+    bool once = 7;
+    string continuous_checkpoint_interval = 8;
+  }
+
+  string output_mode = 9;
+  string query_name = 10;
+
+  // The destination is optional. When set, it can be a path or a table name.
+  oneof sink_destination {
+    string path = 11;
+    string table_name = 12;
+  }
+}
+
+message WriteStreamOperationStartResult {
+
+  // (Required) Query instance. See `StreamingQueryInstanceId`.
+  StreamingQueryInstanceId query_id = 1;
+
+  // An optional query name.
+  string name = 2;
+
+  // TODO: How do we indicate errors?
+  // TODO: Consider adding status, last progress etc here.
+}
+
+// A tuple that uniquely identifies an instance of streaming query run. It consists of `id` that
+// persists across the streaming runs and `run_id` that changes between each run of the
+// streaming query that resumes from the checkpoint.
+message StreamingQueryInstanceId {
+
+  // (Required) The unique id of this query that persists across restarts from checkpoint data.
+  // That is, this id is generated when a query is started for the first time, and
+  // will be the same every time it is restarted from checkpoint data.
+  string id = 1;
+
+  // (Required) The unique id of this run of the query. That is, every start/restart of a query
+  // will generate a unique run_id. Therefore, every time a query is restarted from
+  // checkpoint, it will have the same `id` but different `run_id`s.
+  string run_id = 2;
+}
+
+// Commands for a streaming query.
+message StreamingQueryCommand {
+
+  // (Required) Query instance. See `StreamingQueryInstanceId`.
+  StreamingQueryInstanceId query_id = 1;
+
+  oneof command {
+    // See documentation for the corresponding API method in StreamingQuery.
+
+    // status() API.
+    bool status = 2;
+    // lastProgress() API.
+    bool last_progress = 3;
+    // recentProgress() API.
+    bool recent_progress = 4;
+    // stop() API. Stops the query.
+    bool stop = 5;
+    // processAllAvailable() API. Waits till all the available data is processed
+    bool process_all_available = 6;
+    // explain() API. Returns logical and physical plans.
+    ExplainCommand explain = 7;
+
+    // TODO(SPARK-42960) Add more commands: await_termination(), exception() etc.
+  }
+
+  message ExplainCommand {
+    // TODO: Consider reusing Explain from AnalyzePlanRequest message.
+    //       We can not do this right now since it base.proto imports this file.
+    bool extended = 1;
+  }
+

Review Comment:
   nit: remove empty line



##########
python/pyspark/sql/connect/dataframe.py:
##########
@@ -723,6 +724,33 @@ def to_jcols(
 
     melt = unpivot
 
+    def withWatermark(self, eventTime: str, delayThreshold: str) -> "DataFrame":
+        # TODO: reuse error handling code in sql.DataFrame.withWatermark()
+        if not eventTime or type(eventTime) is not str:

Review Comment:
   nit: why not use `isinstance(eventTime, str)` as other place



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


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

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1157050221


##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -0,0 +1,181 @@
+#
+# 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.
+#
+
+import json
+import sys
+from typing import TYPE_CHECKING, Any, cast, Dict, List, Optional
+
+from pyspark.errors import StreamingQueryException
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.streaming.query import (
+    StreamingQuery as PySparkStreamingQuery,
+)
+
+__all__ = [
+    "StreamingQuery",  # TODO(WIP): "StreamingQueryManager"
+]
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+
+
+class StreamingQuery:
+    def __init__(
+        self, session: "SparkSession", queryId: str, runId: str, name: Optional[str] = None
+    ) -> None:
+        self._session = session
+        self._query_id = queryId
+        self._run_id = runId
+        self._name = name
+
+    @property
+    def id(self) -> str:
+        return self._query_id
+
+    id.__doc__ = PySparkStreamingQuery.id.__doc__
+
+    @property
+    def runId(self) -> str:
+        return self._run_id
+
+    runId.__doc__ = PySparkStreamingQuery.runId.__doc__
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    name.__doc__ = PySparkStreamingQuery.name.__doc__
+
+    @property
+    def isActive(self) -> bool:
+        return self._fetch_status().is_active
+
+    isActive.__doc__ = PySparkStreamingQuery.isActive.__doc__
+
+    def awaitTermination(self, timeout: Optional[int] = None) -> Optional[bool]:
+        raise NotImplementedError()
+
+    awaitTermination.__doc__ = PySparkStreamingQuery.awaitTermination.__doc__
+
+    @property
+    def status(self) -> Dict[str, Any]:
+        proto = self._fetch_status()
+        return {
+            "message": proto.status_message,
+            "isDataAvailable": proto.is_data_available,
+            "isTriggerActive": proto.is_trigger_active,
+        }
+
+    status.__doc__ = PySparkStreamingQuery.status.__doc__
+
+    @property
+    def recentProgress(self) -> List[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=10000).recent_progress_json)
+        # Server only keeps 100, so 10000 limit is high enough.
+        return [json.loads(p) for p in progress]
+
+    recentProgress.__doc__ = PySparkStreamingQuery.recentProgress.__doc__
+
+    @property
+    def lastProgress(self) -> Optional[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=1).recent_progress_json)
+        if len(progress) > 0:
+            return json.loads(progress[-1])
+        else:
+            return None
+
+    lastProgress.__doc__ = PySparkStreamingQuery.lastProgress.__doc__
+
+    def processAllAvailable(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.process_all_available = True
+        self._execute_streaming_query_cmd(cmd)
+
+    processAllAvailable.__doc__ = PySparkStreamingQuery.processAllAvailable.__doc__
+
+    def stop(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.stop = True
+        self._execute_streaming_query_cmd(cmd)
+
+    stop.__doc__ = PySparkStreamingQuery.stop.__doc__
+
+    def explain(self, extended: bool = False) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.explain.extended = extended
+        result = self._execute_streaming_query_cmd(cmd).explain.result
+        print(result)
+
+    explain.__doc__ = PySparkStreamingQuery.explain.__doc__
+
+    def exception(self) -> Optional[StreamingQueryException]:
+        raise NotImplementedError()
+
+    exception.__doc__ = PySparkStreamingQuery.exception.__doc__
+
+    def _fetch_status(
+        self, recent_progress_limit=0
+    ) -> pb2.StreamingQueryCommandResult.StatusResult:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.status.recent_progress_limit = recent_progress_limit
+
+        return self._execute_streaming_query_cmd(cmd).status
+
+    def _execute_streaming_query_cmd(
+        self, cmd: pb2.StreamingQueryCommand
+    ) -> pb2.StreamingQueryCommandResult:
+        cmd.query_id = self._query_id
+        cmd.run_id = self._run_id
+        exec_cmd = pb2.Command()
+        exec_cmd.streaming_query_command.CopyFrom(cmd)
+        (_, properties) = self._session.client.execute_command(exec_cmd)
+        return cast(pb2.StreamingQueryCommandResult, properties["streaming_query_command_result"])
+
+
+# TODO(WIP) class StreamingQueryManager:
+
+
+def _test() -> None:
+    import doctest
+    import os
+    from pyspark.sql import SparkSession
+    import pyspark.sql.streaming.query
+    from py4j.protocol import Py4JError
+
+    os.chdir(os.environ["SPARK_HOME"])
+
+    globs = pyspark.sql.streaming.query.__dict__.copy()
+    try:
+        spark = SparkSession._getActiveSessionOrCreate()

Review Comment:
   BTW, you won't have to suffer from `_getActiveSessionOrCreate` here. I used that in listeners tests because it requires to use the same session (that registered the listener) but in this case I believe it's fine.



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


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

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1156216828


##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,118 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+// Starts write stream operation as streaming query. Query ID and Run ID of the streaming
+// query are returned.
+message WriteStreamOperationStart {
+
+  // (Required) The output of the `input` streaming relation will be written.
+  Relation input = 1;
+
+  // The following fields directly map to API for DataStreamWriter().
+  // Consult API documentation unless explicitly documented here.
+
+  string format = 2;
+  map<string, string> options = 3;
+  repeated string partitioning_column_names = 4;
+
+  oneof trigger {
+    string processing_time_interval = 5;
+    bool available_now = 6;
+    bool one_time = 7;
+    string continuous_checkpoint_interval = 8;
+  }
+
+  string output_mode = 9;
+  string query_name = 10;
+
+  // The destination is optional. When set, it can be a path or a table name.
+  oneof sink_destination {
+    string path = 11;
+    string table_name = 12;
+  }
+}
+
+message WriteStreamOperationStartResult {
+
+  // (Required)
+  string query_id = 1;

Review Comment:
   @amaliujia let's not repeat the existing Spark documentation here. 
   I mentioned multiple times in the is file that we should look at corresponding documentation for the API method. 
   In addition, lets have more thorough example of good documentation in spark-connect. I don't think we should start that that with this PR. 



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


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

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1157043324


##########
python/pyspark/sql/connect/streaming/readwriter.py:
##########
@@ -0,0 +1,484 @@
+#
+# 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.
+#
+
+from pyspark.sql.connect.utils import check_dependencies
+
+check_dependencies(__name__)
+
+import sys
+from typing import cast, overload, Callable, Dict, List, Optional, TYPE_CHECKING, Union
+
+from pyspark.sql.connect.plan import DataSource, LogicalPlan, WriteStreamOperation
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.connect.readwriter import OptionUtils, to_str
+from pyspark.sql.connect.streaming.query import StreamingQuery
+from pyspark.sql.streaming.readwriter import (
+    DataStreamReader as PySparkDataStreamReader,
+    DataStreamWriter as PySparkDataStreamWriter,
+)
+from pyspark.sql.types import Row, StructType
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.dataframe import DataFrame
+
+__all__ = ["DataStreamReader", "DataStreamWriter"]
+
+
+class DataStreamReader(OptionUtils):
+    def __init__(self, client: "SparkSession") -> None:
+        self._format: Optional[str] = None
+        self._schema = ""
+        self._client = client
+        self._options: Dict[str, str] = {}
+
+    def _df(self, plan: LogicalPlan) -> "DataFrame":
+        from pyspark.sql.connect.dataframe import DataFrame
+
+        return DataFrame.withPlan(plan, self._client)
+
+    def format(self, source: str) -> "DataStreamReader":
+        self._format = source
+        return self
+
+    format.__doc__ = PySparkDataStreamReader.format.__doc__
+
+    def schema(self, schema: Union[StructType, str]) -> "DataStreamReader":
+        if isinstance(schema, StructType):
+            self._schema = schema.json()
+        elif isinstance(schema, str):
+            self._schema = schema
+        else:
+            raise TypeError("schema should be StructType or string")
+        return self
+
+    schema.__doc__ = PySparkDataStreamReader.schema.__doc__
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataStreamReader":
+        self._options[key] = str(value)
+        return self
+
+    option.__doc__ = PySparkDataStreamReader.option.__doc__
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataStreamReader":
+        for k in options:
+            self.option(k, to_str(options[k]))
+        return self
+
+    options.__doc__ = PySparkDataStreamReader.options.__doc__
+
+    def load(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> "DataFrame":
+        if format is not None:
+            self.format(format)
+        if schema is not None:
+            self.schema(schema)
+        self.options(**options)
+        if path is not None:
+            if type(path) != str or len(path.strip()) == 0:
+                raise ValueError(
+                    "If the path is provided for stream, it needs to be a "
+                    + "non-empty string. List of paths are not supported."
+                )
+
+        plan = DataSource(
+            format=self._format,
+            schema=self._schema,
+            options=self._options,
+            paths=[path] if path else None,
+            is_streaming=True,
+        )
+
+        return self._df(plan)
+
+    load.__doc__ = PySparkDataStreamReader.load.__doc__
+
+    def json(
+        self,
+        path: str,
+        schema: Optional[Union[StructType, str]] = None,
+        primitivesAsString: Optional[Union[bool, str]] = None,
+        prefersDecimal: Optional[Union[bool, str]] = None,
+        allowComments: Optional[Union[bool, str]] = None,
+        allowUnquotedFieldNames: Optional[Union[bool, str]] = None,
+        allowSingleQuotes: Optional[Union[bool, str]] = None,
+        allowNumericLeadingZero: Optional[Union[bool, str]] = None,
+        allowBackslashEscapingAnyCharacter: Optional[Union[bool, str]] = None,
+        mode: Optional[str] = None,
+        columnNameOfCorruptRecord: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        multiLine: Optional[Union[bool, str]] = None,
+        allowUnquotedControlChars: Optional[Union[bool, str]] = None,
+        lineSep: Optional[str] = None,
+        locale: Optional[str] = None,
+        dropFieldIfAllNull: Optional[Union[bool, str]] = None,
+        encoding: Optional[str] = None,
+        pathGlobFilter: Optional[Union[bool, str]] = None,
+        recursiveFileLookup: Optional[Union[bool, str]] = None,
+        allowNonNumericNumbers: Optional[Union[bool, str]] = None,
+    ) -> "DataFrame":
+        self._set_opts(
+            schema=schema,
+            primitivesAsString=primitivesAsString,
+            prefersDecimal=prefersDecimal,
+            allowComments=allowComments,
+            allowUnquotedFieldNames=allowUnquotedFieldNames,
+            allowSingleQuotes=allowSingleQuotes,
+            allowNumericLeadingZero=allowNumericLeadingZero,
+            allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter,
+            mode=mode,
+            columnNameOfCorruptRecord=columnNameOfCorruptRecord,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            multiLine=multiLine,
+            allowUnquotedControlChars=allowUnquotedControlChars,
+            lineSep=lineSep,
+            locale=locale,
+            dropFieldIfAllNull=dropFieldIfAllNull,
+            encoding=encoding,
+            pathGlobFilter=pathGlobFilter,
+            recursiveFileLookup=recursiveFileLookup,
+            allowNonNumericNumbers=allowNonNumericNumbers,
+        )
+        if isinstance(path, str):
+            return self.load(path=path, format="json")
+        else:
+            raise TypeError("path can be only a single string")
+
+    json.__doc__ = PySparkDataStreamReader.json.__doc__
+
+    # def orc() TODO

Review Comment:
   Should actually also file JIRAs for each todos.



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


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

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1157040476


##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -0,0 +1,181 @@
+#
+# 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.
+#
+
+import json
+import sys
+from typing import TYPE_CHECKING, Any, cast, Dict, List, Optional
+
+from pyspark.errors import StreamingQueryException
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.streaming.query import (
+    StreamingQuery as PySparkStreamingQuery,
+)
+
+__all__ = [
+    "StreamingQuery",  # TODO(WIP): "StreamingQueryManager"
+]
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+
+
+class StreamingQuery:
+    def __init__(
+        self, session: "SparkSession", queryId: str, runId: str, name: Optional[str] = None
+    ) -> None:
+        self._session = session
+        self._query_id = queryId
+        self._run_id = runId
+        self._name = name
+
+    @property
+    def id(self) -> str:
+        return self._query_id
+
+    id.__doc__ = PySparkStreamingQuery.id.__doc__
+
+    @property
+    def runId(self) -> str:
+        return self._run_id
+
+    runId.__doc__ = PySparkStreamingQuery.runId.__doc__
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    name.__doc__ = PySparkStreamingQuery.name.__doc__
+
+    @property
+    def isActive(self) -> bool:
+        return self._fetch_status().is_active
+
+    isActive.__doc__ = PySparkStreamingQuery.isActive.__doc__
+
+    def awaitTermination(self, timeout: Optional[int] = None) -> Optional[bool]:
+        raise NotImplementedError()
+
+    awaitTermination.__doc__ = PySparkStreamingQuery.awaitTermination.__doc__
+
+    @property
+    def status(self) -> Dict[str, Any]:
+        proto = self._fetch_status()
+        return {
+            "message": proto.status_message,
+            "isDataAvailable": proto.is_data_available,
+            "isTriggerActive": proto.is_trigger_active,
+        }
+
+    status.__doc__ = PySparkStreamingQuery.status.__doc__
+
+    @property
+    def recentProgress(self) -> List[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=10000).recent_progress_json)
+        # Server only keeps 100, so 10000 limit is high enough.
+        return [json.loads(p) for p in progress]
+
+    recentProgress.__doc__ = PySparkStreamingQuery.recentProgress.__doc__
+
+    @property
+    def lastProgress(self) -> Optional[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=1).recent_progress_json)
+        if len(progress) > 0:
+            return json.loads(progress[-1])
+        else:
+            return None
+
+    lastProgress.__doc__ = PySparkStreamingQuery.lastProgress.__doc__
+
+    def processAllAvailable(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.process_all_available = True
+        self._execute_streaming_query_cmd(cmd)
+
+    processAllAvailable.__doc__ = PySparkStreamingQuery.processAllAvailable.__doc__
+
+    def stop(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.stop = True
+        self._execute_streaming_query_cmd(cmd)
+
+    stop.__doc__ = PySparkStreamingQuery.stop.__doc__
+
+    def explain(self, extended: bool = False) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.explain.extended = extended
+        result = self._execute_streaming_query_cmd(cmd).explain.result
+        print(result)
+
+    explain.__doc__ = PySparkStreamingQuery.explain.__doc__
+
+    def exception(self) -> Optional[StreamingQueryException]:
+        raise NotImplementedError()
+
+    exception.__doc__ = PySparkStreamingQuery.exception.__doc__
+
+    def _fetch_status(
+        self, recent_progress_limit=0
+    ) -> pb2.StreamingQueryCommandResult.StatusResult:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.status.recent_progress_limit = recent_progress_limit
+
+        return self._execute_streaming_query_cmd(cmd).status
+
+    def _execute_streaming_query_cmd(
+        self, cmd: pb2.StreamingQueryCommand
+    ) -> pb2.StreamingQueryCommandResult:
+        cmd.query_id = self._query_id
+        cmd.run_id = self._run_id
+        exec_cmd = pb2.Command()
+        exec_cmd.streaming_query_command.CopyFrom(cmd)
+        (_, properties) = self._session.client.execute_command(exec_cmd)
+        return cast(pb2.StreamingQueryCommandResult, properties["streaming_query_command_result"])
+
+
+# TODO(WIP) class StreamingQueryManager:
+
+
+def _test() -> None:
+    import doctest
+    import os
+    from pyspark.sql import SparkSession
+    import pyspark.sql.streaming.query
+    from py4j.protocol import Py4JError
+
+    os.chdir(os.environ["SPARK_HOME"])
+
+    globs = pyspark.sql.streaming.query.__dict__.copy()
+    try:
+        spark = SparkSession._getActiveSessionOrCreate()

Review Comment:
   Should use the remote session here to run the doctests with remote Spark session. 
   
   e.g.,
   
   ```python
       globs["spark"] = (
           PySparkSession.builder.appName("sql.connect.streaming.query tests")
           .remote("local[4]")
           .getOrCreate()
       )
   ```



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


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

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1157034490


##########
python/pyspark/sql/connect/readwriter.py:
##########
@@ -37,7 +37,7 @@
     from pyspark.sql.connect._typing import ColumnOrName, OptionalPrimitiveType
     from pyspark.sql.connect.session import SparkSession
 
-__all__ = ["DataFrameReader", "DataFrameWriter"]
+__all__ = ["DataFrameReader", "DataFrameWriter", "OptionUtils", "to_str"]

Review Comment:
   Sorry if I missed sth but why do we need this? This isn't an API, and it's not being imported now.



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


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

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1159246614


##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -0,0 +1,158 @@
+#
+# 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.
+#
+
+import json
+from typing import TYPE_CHECKING, Any, cast, Dict, List, Optional
+
+from pyspark.errors import StreamingQueryException
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.streaming.query import (
+    StreamingQuery as PySparkStreamingQuery,
+)
+
+__all__ = [
+    "StreamingQuery",  # TODO(SPARK-43032): "StreamingQueryManager"
+]
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+
+
+class StreamingQuery:

Review Comment:
   BTW, we should add something like `versionchanged: 3.5.0 Supports Spark Connect` in `pyspark.sql.streaming.query.StreamingQuery`'s docstring. I assume that'd be done in a followup or separate pr.



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


[GitHub] [spark] HyukjinKwon closed pull request #40586: [SPARK-42939][SS][CONNECT] Core streaming Python API for Spark Connect

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon closed pull request #40586: [SPARK-42939][SS][CONNECT] Core streaming Python API for Spark Connect 
URL: https://github.com/apache/spark/pull/40586


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


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

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1152453603


##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,97 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+message WriteStreamOperation {

Review Comment:
   Optional is always implied. May be it is required only if we want to emphasis optional part of it. 



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


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

Posted by "WweiL (via GitHub)" <gi...@apache.org>.
WweiL commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1157836737


##########
python/pyspark/sql/connect/streaming/readwriter.py:
##########
@@ -0,0 +1,484 @@
+#
+# 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.
+#
+
+from pyspark.sql.connect.utils import check_dependencies
+
+check_dependencies(__name__)
+
+import sys
+from typing import cast, overload, Callable, Dict, List, Optional, TYPE_CHECKING, Union
+
+from pyspark.sql.connect.plan import DataSource, LogicalPlan, WriteStreamOperation
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.connect.readwriter import OptionUtils, to_str
+from pyspark.sql.connect.streaming.query import StreamingQuery
+from pyspark.sql.streaming.readwriter import (
+    DataStreamReader as PySparkDataStreamReader,
+    DataStreamWriter as PySparkDataStreamWriter,
+)
+from pyspark.sql.types import Row, StructType
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.dataframe import DataFrame
+
+__all__ = ["DataStreamReader", "DataStreamWriter"]
+
+
+class DataStreamReader(OptionUtils):
+    def __init__(self, client: "SparkSession") -> None:
+        self._format: Optional[str] = None
+        self._schema = ""
+        self._client = client
+        self._options: Dict[str, str] = {}
+
+    def _df(self, plan: LogicalPlan) -> "DataFrame":
+        from pyspark.sql.connect.dataframe import DataFrame
+
+        return DataFrame.withPlan(plan, self._client)
+
+    def format(self, source: str) -> "DataStreamReader":
+        self._format = source
+        return self
+
+    format.__doc__ = PySparkDataStreamReader.format.__doc__
+
+    def schema(self, schema: Union[StructType, str]) -> "DataStreamReader":
+        if isinstance(schema, StructType):
+            self._schema = schema.json()
+        elif isinstance(schema, str):
+            self._schema = schema
+        else:
+            raise TypeError("schema should be StructType or string")
+        return self
+
+    schema.__doc__ = PySparkDataStreamReader.schema.__doc__
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataStreamReader":
+        self._options[key] = str(value)
+        return self
+
+    option.__doc__ = PySparkDataStreamReader.option.__doc__
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataStreamReader":
+        for k in options:
+            self.option(k, to_str(options[k]))
+        return self
+
+    options.__doc__ = PySparkDataStreamReader.options.__doc__
+
+    def load(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> "DataFrame":
+        if format is not None:
+            self.format(format)
+        if schema is not None:
+            self.schema(schema)
+        self.options(**options)
+        if path is not None:
+            if type(path) != str or len(path.strip()) == 0:
+                raise ValueError(
+                    "If the path is provided for stream, it needs to be a "
+                    + "non-empty string. List of paths are not supported."
+                )
+
+        plan = DataSource(
+            format=self._format,
+            schema=self._schema,
+            options=self._options,
+            paths=[path] if path else None,
+            is_streaming=True,
+        )
+
+        return self._df(plan)
+
+    load.__doc__ = PySparkDataStreamReader.load.__doc__
+
+    def json(
+        self,
+        path: str,
+        schema: Optional[Union[StructType, str]] = None,
+        primitivesAsString: Optional[Union[bool, str]] = None,
+        prefersDecimal: Optional[Union[bool, str]] = None,
+        allowComments: Optional[Union[bool, str]] = None,
+        allowUnquotedFieldNames: Optional[Union[bool, str]] = None,
+        allowSingleQuotes: Optional[Union[bool, str]] = None,
+        allowNumericLeadingZero: Optional[Union[bool, str]] = None,
+        allowBackslashEscapingAnyCharacter: Optional[Union[bool, str]] = None,
+        mode: Optional[str] = None,
+        columnNameOfCorruptRecord: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        multiLine: Optional[Union[bool, str]] = None,
+        allowUnquotedControlChars: Optional[Union[bool, str]] = None,
+        lineSep: Optional[str] = None,
+        locale: Optional[str] = None,
+        dropFieldIfAllNull: Optional[Union[bool, str]] = None,
+        encoding: Optional[str] = None,
+        pathGlobFilter: Optional[Union[bool, str]] = None,
+        recursiveFileLookup: Optional[Union[bool, str]] = None,
+        allowNonNumericNumbers: Optional[Union[bool, str]] = None,
+    ) -> "DataFrame":
+        self._set_opts(
+            schema=schema,
+            primitivesAsString=primitivesAsString,
+            prefersDecimal=prefersDecimal,
+            allowComments=allowComments,
+            allowUnquotedFieldNames=allowUnquotedFieldNames,
+            allowSingleQuotes=allowSingleQuotes,
+            allowNumericLeadingZero=allowNumericLeadingZero,
+            allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter,
+            mode=mode,
+            columnNameOfCorruptRecord=columnNameOfCorruptRecord,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            multiLine=multiLine,
+            allowUnquotedControlChars=allowUnquotedControlChars,
+            lineSep=lineSep,
+            locale=locale,
+            dropFieldIfAllNull=dropFieldIfAllNull,
+            encoding=encoding,
+            pathGlobFilter=pathGlobFilter,
+            recursiveFileLookup=recursiveFileLookup,
+            allowNonNumericNumbers=allowNonNumericNumbers,
+        )
+        if isinstance(path, str):
+            return self.load(path=path, format="json")
+        else:
+            raise TypeError("path can be only a single string")
+
+    json.__doc__ = PySparkDataStreamReader.json.__doc__
+
+    # def orc() TODO

Review Comment:
   Yes we've filed a JIRA, https://issues.apache.org/jira/projects/SPARK/issues/SPARK-42951



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


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

Posted by "amaliujia (via GitHub)" <gi...@apache.org>.
amaliujia commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1156235562


##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,118 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+// Starts write stream operation as streaming query. Query ID and Run ID of the streaming
+// query are returned.
+message WriteStreamOperationStart {
+
+  // (Required) The output of the `input` streaming relation will be written.
+  Relation input = 1;
+
+  // The following fields directly map to API for DataStreamWriter().
+  // Consult API documentation unless explicitly documented here.
+
+  string format = 2;
+  map<string, string> options = 3;
+  repeated string partitioning_column_names = 4;
+
+  oneof trigger {
+    string processing_time_interval = 5;
+    bool available_now = 6;
+    bool one_time = 7;
+    string continuous_checkpoint_interval = 8;
+  }
+
+  string output_mode = 9;
+  string query_name = 10;
+
+  // The destination is optional. When set, it can be a path or a table name.
+  oneof sink_destination {
+    string path = 11;
+    string table_name = 12;
+  }
+}
+
+message WriteStreamOperationStartResult {
+
+  // (Required)
+  string query_id = 1;

Review Comment:
   It's ok that we do not have it at the beginning. But I do hope eventually we could improve.



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


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

Posted by "amaliujia (via GitHub)" <gi...@apache.org>.
amaliujia commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1156243044


##########
connector/connect/common/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -154,6 +158,9 @@ message Read {
     //
     // This is only supported by the JDBC data source.
     repeated string predicates = 5;
+
+    // (Optional) Source table name for a streaming read. Not used in batch read.
+    string streaming_table_name = 6;

Review Comment:
   ```
     message NamedTable {
       // (Required) Unparsed identifier for the table.
       string unparsed_identifier = 1;
       // Options for the named table. The map key is case insensitive.
       map<string, string> options = 2;
     }
    ```
   
   NamedTable was fixed to support options now. But it is ok to migrate to that later given that this PR is already at a good shape.



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


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

Posted by "amaliujia (via GitHub)" <gi...@apache.org>.
amaliujia commented on PR #40586:
URL: https://github.com/apache/spark/pull/40586#issuecomment-1496448770

   The proto side overall looks good.


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


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

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1157045929


##########
python/pyspark/sql/dataframe.py:
##########
@@ -529,14 +529,10 @@ def writeStream(self) -> DataStreamWriter:
         --------
         >>> import tempfile
         >>> df = spark.readStream.format("rate").load()
-        >>> type(df.writeStream)
-        <class 'pyspark.sql.streaming.readwriter.DataStreamWriter'>
-
         >>> with tempfile.TemporaryDirectory() as d:
         ...     # Create a table with Rate source.
-        ...     df.writeStream.toTable(
+        ...     streaming_query = df.writeStream.toTable(
         ...         "my_table", checkpointLocation=d) # doctest: +ELLIPSIS
-        <pyspark.sql.streaming.query.StreamingQuery object at 0x...>

Review Comment:
   ```python
   <...streaming.query.StreamingQuery object ...>
   ```



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


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

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1156216828


##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,118 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+// Starts write stream operation as streaming query. Query ID and Run ID of the streaming
+// query are returned.
+message WriteStreamOperationStart {
+
+  // (Required) The output of the `input` streaming relation will be written.
+  Relation input = 1;
+
+  // The following fields directly map to API for DataStreamWriter().
+  // Consult API documentation unless explicitly documented here.
+
+  string format = 2;
+  map<string, string> options = 3;
+  repeated string partitioning_column_names = 4;
+
+  oneof trigger {
+    string processing_time_interval = 5;
+    bool available_now = 6;
+    bool one_time = 7;
+    string continuous_checkpoint_interval = 8;
+  }
+
+  string output_mode = 9;
+  string query_name = 10;
+
+  // The destination is optional. When set, it can be a path or a table name.
+  oneof sink_destination {
+    string path = 11;
+    string table_name = 12;
+  }
+}
+
+message WriteStreamOperationStartResult {
+
+  // (Required)
+  string query_id = 1;

Review Comment:
   @amaliujia let's not repeat the existing Spark documentation here. 
   I mentioned multiple times in this file that we should look at corresponding documentation for the API method. 
   In addition, lets have more thorough example of good documentation in spark-connect. I don't think we should start that that with this PR. 



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


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

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1156213953


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -816,6 +836,27 @@ class SparkConnectPlanner(val session: SparkSession) {
             s"Predicates are not supported for ${rel.getDataSource.getFormat} data sources.")
         }
 
+      case proto.Read.ReadTypeCase.DATA_SOURCE if rel.getIsStreaming =>
+        val streamSource = rel.getDataSource
+        val reader = session.readStream
+        if (streamSource.hasFormat) {

Review Comment:
   It is same as what happens when format is not set in normal API. It could derived from load method (json(), table() etc).



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


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

Posted by "pengzhon-db (via GitHub)" <gi...@apache.org>.
pengzhon-db commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1154944439


##########
python/pyspark/sql/connect/client.py:
##########
@@ -949,6 +949,10 @@ def _execute_and_fetch_as_iterator(
                         if b.HasField("sql_command_result"):
                             logger.debug("Received the SQL command result.")
                             yield {"sql_command_result": b.sql_command_result.relation}
+                        if b.HasField("write_stream_operation_start_result"):
+                            yield {"write_stream_operation_start_result": b.write_stream_operation_start_result}

Review Comment:
   Do u want to add a `logger.debug` as other cases?



##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,118 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+// Starts write stream operation as streaming query. Query ID and Run ID of the streaming
+// query are returned.
+message WriteStreamOperationStart {
+
+  // (Required) The output of the `input` streaming relation will be written.
+  Relation input = 1;
+
+  // The following fields directly map to API for DataStreamWriter().
+  // Consult API documentation unless explicitly documented here.
+
+  string format = 2;
+  map<string, string> options = 3;
+  repeated string partitioning_column_names = 4;
+
+  oneof trigger {
+    string processing_time_interval = 5;
+    bool available_now = 6;
+    bool one_time = 7;
+    string continuous_checkpoint_interval = 8;
+  }
+
+  string output_mode = 9;
+  string query_name = 10;
+
+  // The destination is optional. When set, it can be a path or a table name.
+  oneof sink_destination {
+    string path = 11;
+    string table_name = 12;
+  }
+}
+
+message WriteStreamOperationStartResult {
+
+  // (Required)
+  string query_id = 1;
+
+  // (Required)
+  string run_id = 2;
+
+  // An optional query name.
+  string name = 3;
+
+  // TODO: How do we indicate errors?
+  // TODO: Consider adding StreamingQueryStatusResult here.
+}
+
+// Commands for a streaming query.
+message StreamingQueryCommand {
+
+  // (Required) query id of the streaming query.
+  string query_id = 1;
+  // (Required) run id of the streaming query.
+  string run_id = 2;
+
+  // A running query is identified by both run_id and query_id.
+
+  oneof command_type {
+    // Status of the query. Used to support multiple status related API like lastProgress().
+    StatusCommand status = 3;
+    // Stops the query.
+    bool stop = 4;
+    // Waits till all the available data is processed. See processAllAvailable() API doc.
+    bool process_all_available = 5;
+    // Returns logical and physical plans.
+    ExplainCommand explain = 6;
+
+    // TODO(SPARK-42960) Add more commands: await_termination(), exception() etc.
+  }
+
+  message StatusCommand {
+    // A limit on how many progress reports to return.
+    int32 recent_progress_limit = 1;
+  }
+
+  message ExplainCommand {
+    // TODO: Consider reusing Explain from AnalyzePlanRequest message.
+    //       We can not do this right now since it base.proto imports this file.
+    bool extended = 1;
+  }
+
+}
+
+// Response for commands on a streaming query.
+message StreamingQueryCommandResult {
+  // (Required)
+  string query_id = 1;
+
+  oneof result_type {
+    StatusResult status = 2;
+    ExplainResult explain = 3;
+  }
+
+  message StatusResult {
+    // This status includes all the available to status, including progress messages.
+
+    // Fields from Scala 'StreamingQueryStatus' struct
+    string status_message = 1;
+    bool is_data_available = 2;
+    bool is_trigger_active = 3;
+
+    bool is_active = 4;
+
+    // Progress reports as an array of json strings.
+    repeated string recent_progress_json = 5;

Review Comment:
   would it be better to separate progress with status? The status of a query only contains first 3 fields



##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,118 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+// Starts write stream operation as streaming query. Query ID and Run ID of the streaming
+// query are returned.
+message WriteStreamOperationStart {
+
+  // (Required) The output of the `input` streaming relation will be written.
+  Relation input = 1;
+
+  // The following fields directly map to API for DataStreamWriter().
+  // Consult API documentation unless explicitly documented here.
+
+  string format = 2;
+  map<string, string> options = 3;
+  repeated string partitioning_column_names = 4;
+
+  oneof trigger {
+    string processing_time_interval = 5;
+    bool available_now = 6;
+    bool one_time = 7;
+    string continuous_checkpoint_interval = 8;
+  }
+
+  string output_mode = 9;
+  string query_name = 10;
+
+  // The destination is optional. When set, it can be a path or a table name.
+  oneof sink_destination {
+    string path = 11;
+    string table_name = 12;
+  }
+}
+
+message WriteStreamOperationStartResult {
+
+  // (Required)
+  string query_id = 1;
+
+  // (Required)
+  string run_id = 2;
+
+  // An optional query name.
+  string name = 3;
+
+  // TODO: How do we indicate errors?
+  // TODO: Consider adding StreamingQueryStatusResult here.
+}
+
+// Commands for a streaming query.
+message StreamingQueryCommand {
+
+  // (Required) query id of the streaming query.
+  string query_id = 1;
+  // (Required) run id of the streaming query.
+  string run_id = 2;
+
+  // A running query is identified by both run_id and query_id.
+
+  oneof command_type {
+    // Status of the query. Used to support multiple status related API like lastProgress().
+    StatusCommand status = 3;
+    // Stops the query.
+    bool stop = 4;
+    // Waits till all the available data is processed. See processAllAvailable() API doc.
+    bool process_all_available = 5;
+    // Returns logical and physical plans.
+    ExplainCommand explain = 6;
+
+    // TODO(SPARK-42960) Add more commands: await_termination(), exception() etc.
+  }
+
+  message StatusCommand {
+    // A limit on how many progress reports to return.
+    int32 recent_progress_limit = 1;
+  }
+
+  message ExplainCommand {
+    // TODO: Consider reusing Explain from AnalyzePlanRequest message.
+    //       We can not do this right now since it base.proto imports this file.
+    bool extended = 1;
+  }
+
+}
+
+// Response for commands on a streaming query.
+message StreamingQueryCommandResult {
+  // (Required)
+  string query_id = 1;

Review Comment:
   why the other response has `run_id` but this one doesn't?



##########
connector/connect/common/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -154,6 +158,9 @@ message Read {
     //
     // This is only supported by the JDBC data source.
     repeated string predicates = 5;
+
+    // (Optional) Source table name for a streaming read. Not used in batch read.
+    string streaming_table_name = 6;

Review Comment:
   I think batch can also read from a table. How does that work? Can we reuse that?



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -1969,6 +2017,142 @@ class SparkConnectPlanner(val session: SparkSession) {
     }
   }
 
+  def handleWriteStreamOperationStart(
+      writeOp: WriteStreamOperationStart,
+      sessionId: String,
+      responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
+    val plan = transformRelation(writeOp.getInput)
+    val dataset = Dataset.ofRows(session, logicalPlan = plan)
+
+    val writer = dataset.writeStream
+
+    if (writeOp.getFormat.nonEmpty) {
+      writer.format(writeOp.getFormat)
+    }
+
+    writer.options(writeOp.getOptionsMap)
+
+    if (writeOp.getPartitioningColumnNamesCount > 0) {
+      writer.partitionBy(writeOp.getPartitioningColumnNamesList.asScala.toList: _*)
+    }
+
+    writeOp.getTriggerCase match {
+      case TriggerCase.PROCESSING_TIME_INTERVAL =>
+        writer.trigger(Trigger.ProcessingTime(writeOp.getProcessingTimeInterval))
+      case TriggerCase.AVAILABLE_NOW =>
+        writer.trigger(Trigger.AvailableNow())
+      case TriggerCase.ONE_TIME =>
+        writer.trigger(Trigger.Once())
+      case TriggerCase.CONTINUOUS_CHECKPOINT_INTERVAL =>
+        writer.trigger(Trigger.Continuous(writeOp.getContinuousCheckpointInterval))
+      case TriggerCase.TRIGGER_NOT_SET =>
+    }
+
+    if (writeOp.getOutputMode.nonEmpty) {
+      writer.outputMode(writeOp.getOutputMode)
+    }
+
+    if (writeOp.getQueryName.nonEmpty) {
+      writer.queryName(writeOp.getQueryName)
+    }
+
+    val query = writeOp.getPath match {
+      case "" if writeOp.hasTableName => writer.toTable(writeOp.getTableName)
+      case "" => writer.start()
+      case path => writer.start(path)
+    }
+
+    val result = WriteStreamOperationStartResult
+      .newBuilder()
+      .setQueryId(query.id.toString)
+      .setRunId(query.runId.toString)
+      .setName(Option(query.name).getOrElse(""))
+      .build()
+
+    responseObserver.onNext(
+      ExecutePlanResponse
+        .newBuilder()
+        .setSessionId(sessionId)
+        .setWriteStreamOperationStartResult(result)
+        .build())
+  }
+
+  def handleStreamingQueryCommand(
+      command: StreamingQueryCommand,
+      sessionId: String,
+      responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
+
+    val queryId = command.getQueryId
+
+    val respBuilder = StreamingQueryCommandResult
+      .newBuilder()
+      .setQueryId(command.getQueryId)
+
+    val query = Option(session.streams.get(queryId)) match {
+      case Some(query) if query.runId.toString == command.getRunId =>
+        query
+      case Some(query) =>
+        throw new IllegalArgumentException(
+          s"Run id mismatch for query id $queryId. Run id in the request ${command.getRunId} " +
+            s"does not match one on the server ${query.runId}. The query might have restarted.")
+      case None =>
+        throw new IllegalArgumentException(s"Streaming query $queryId is not found")
+      // TODO(SPARK-42962): Handle this better. May be cache stopped queries for a few minutes.
+    }
+
+    command.getCommandTypeCase match {
+      case StreamingQueryCommand.CommandTypeCase.STATUS =>
+        val recentProgress: Seq[String] = command.getStatus.getRecentProgressLimit match {
+          case 0 => Seq.empty
+          case limit if limit < 0 =>
+            query.recentProgress.map(_.json) // All the cached progresses.
+          case limit => query.recentProgress.takeRight(limit).map(_.json) // Most recent
+        }
+
+        val queryStatus = query.status
+
+        val statusResult = StreamingQueryCommandResult.StatusResult
+          .newBuilder()
+          .setStatusMessage(queryStatus.message)
+          .setIsDataAvailable(queryStatus.isDataAvailable)
+          .setIsTriggerActive(queryStatus.isTriggerActive)
+          .setIsActive(query.isActive)
+          .addAllRecentProgressJson(recentProgress.asJava)
+          .build()
+
+        respBuilder.setStatus(statusResult)
+
+      case StreamingQueryCommand.CommandTypeCase.STOP =>
+        query.stop()
+
+      case StreamingQueryCommand.CommandTypeCase.PROCESS_ALL_AVAILABLE =>
+        query.processAllAvailable()

Review Comment:
   This method may take a long time to return if there is continually arriving data. How to handle that? Will it time out the rpc call?



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -816,6 +836,27 @@ class SparkConnectPlanner(val session: SparkSession) {
             s"Predicates are not supported for ${rel.getDataSource.getFormat} data sources.")
         }
 
+      case proto.Read.ReadTypeCase.DATA_SOURCE if rel.getIsStreaming =>
+        val streamSource = rel.getDataSource
+        val reader = session.readStream
+        if (streamSource.hasFormat) {

Review Comment:
   could `hasFormat` be false? what happens in that case?



##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,118 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+// Starts write stream operation as streaming query. Query ID and Run ID of the streaming
+// query are returned.
+message WriteStreamOperationStart {
+
+  // (Required) The output of the `input` streaming relation will be written.
+  Relation input = 1;
+
+  // The following fields directly map to API for DataStreamWriter().
+  // Consult API documentation unless explicitly documented here.
+
+  string format = 2;
+  map<string, string> options = 3;
+  repeated string partitioning_column_names = 4;
+
+  oneof trigger {
+    string processing_time_interval = 5;
+    bool available_now = 6;
+    bool one_time = 7;

Review Comment:
   why call this `one_time` instead of `once` ?



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


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

Posted by "pengzhon-db (via GitHub)" <gi...@apache.org>.
pengzhon-db commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1156260899


##########
python/pyspark/sql/dataframe.py:
##########
@@ -529,14 +529,10 @@ def writeStream(self) -> DataStreamWriter:
         --------
         >>> import tempfile
         >>> df = spark.readStream.format("rate").load()
-        >>> type(df.writeStream)

Review Comment:
   why remove those?



##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -0,0 +1,175 @@
+#
+# 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.
+#
+
+import json
+import sys
+from typing import TYPE_CHECKING, Any, cast, Dict, List, Optional
+
+from pyspark.errors import StreamingQueryException
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.streaming.query import (
+    StreamingQuery as PySparkStreamingQuery,
+)
+
+__all__ = [
+    "StreamingQuery",  # TODO(WIP): "StreamingQueryManager"
+]
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+
+
+class StreamingQuery:
+
+    def __init__(self, session: "SparkSession", queryId: str, runId: str, name: Optional[str] = None) -> None:
+        self._session = session
+        self._query_id = queryId
+        self._run_id = runId
+        self._name = name
+
+    @property
+    def id(self) -> str:
+        return self._query_id
+
+    id.__doc__ = PySparkStreamingQuery.id.__doc__
+
+    @property
+    def runId(self) -> str:
+        return self._run_id
+
+    runId.__doc__ = PySparkStreamingQuery.runId.__doc__
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    name.__doc__ = PySparkStreamingQuery.name.__doc__
+
+    @property
+    def isActive(self) -> bool:
+        return self._fetch_status().is_active
+
+    isActive.__doc__ = PySparkStreamingQuery.isActive.__doc__
+
+    def awaitTermination(self, timeout: Optional[int] = None) -> Optional[bool]:
+        raise NotImplementedError()
+
+    awaitTermination.__doc__ = PySparkStreamingQuery.awaitTermination.__doc__
+
+    @property
+    def status(self) -> Dict[str, Any]:
+        proto = self._fetch_status()
+        return {
+            "message": proto.status_message,
+            "isDataAvailable": proto.is_data_available,
+            "isTriggerActive": proto.is_trigger_active
+        }
+
+    status.__doc__ = PySparkStreamingQuery.status.__doc__
+
+    @property
+    def recentProgress(self) -> List[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=-1).recent_progress_json)

Review Comment:
   why `-1`?



##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -0,0 +1,175 @@
+#
+# 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.
+#
+
+import json
+import sys
+from typing import TYPE_CHECKING, Any, cast, Dict, List, Optional
+
+from pyspark.errors import StreamingQueryException
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.streaming.query import (
+    StreamingQuery as PySparkStreamingQuery,
+)
+
+__all__ = [
+    "StreamingQuery",  # TODO(WIP): "StreamingQueryManager"
+]
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+
+
+class StreamingQuery:
+
+    def __init__(self, session: "SparkSession", queryId: str, runId: str, name: Optional[str] = None) -> None:
+        self._session = session
+        self._query_id = queryId
+        self._run_id = runId
+        self._name = name
+
+    @property
+    def id(self) -> str:
+        return self._query_id
+
+    id.__doc__ = PySparkStreamingQuery.id.__doc__
+
+    @property
+    def runId(self) -> str:
+        return self._run_id
+
+    runId.__doc__ = PySparkStreamingQuery.runId.__doc__
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    name.__doc__ = PySparkStreamingQuery.name.__doc__
+
+    @property
+    def isActive(self) -> bool:
+        return self._fetch_status().is_active
+
+    isActive.__doc__ = PySparkStreamingQuery.isActive.__doc__
+
+    def awaitTermination(self, timeout: Optional[int] = None) -> Optional[bool]:
+        raise NotImplementedError()
+
+    awaitTermination.__doc__ = PySparkStreamingQuery.awaitTermination.__doc__
+
+    @property
+    def status(self) -> Dict[str, Any]:
+        proto = self._fetch_status()
+        return {
+            "message": proto.status_message,
+            "isDataAvailable": proto.is_data_available,
+            "isTriggerActive": proto.is_trigger_active
+        }
+
+    status.__doc__ = PySparkStreamingQuery.status.__doc__
+
+    @property
+    def recentProgress(self) -> List[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=-1).recent_progress_json)
+        return [json.loads(p) for p in progress]
+
+    recentProgress.__doc__ = PySparkStreamingQuery.recentProgress.__doc__
+
+    @property
+    def lastProgress(self) -> Optional[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=1).recent_progress_json)
+        if len(progress) > 0:
+            return json.loads(progress[-1])
+        else:
+            return None
+
+    lastProgress.__doc__ = PySparkStreamingQuery.lastProgress.__doc__
+
+    def processAllAvailable(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.process_all_available = True
+        self._execute_streaming_query_cmd(cmd)
+
+    processAllAvailable.__doc__ = PySparkStreamingQuery.processAllAvailable.__doc__
+
+    def stop(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.stop = True
+        self._execute_streaming_query_cmd(cmd)
+
+    stop.__doc__ = PySparkStreamingQuery.stop.__doc__
+
+    def explain(self, extended: bool = False) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.explain.extended = extended
+        result = self._execute_streaming_query_cmd(cmd).explain.result
+        print(result)
+
+    processAllAvailable.__doc__ = PySparkStreamingQuery.processAllAvailable.__doc__
+
+    def exception(self) -> Optional[StreamingQueryException]:
+        raise NotImplementedError()
+
+    processAllAvailable.__doc__ = PySparkStreamingQuery.processAllAvailable.__doc__

Review Comment:
   similarly, this should be exception?



##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -0,0 +1,175 @@
+#
+# 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.
+#
+
+import json
+import sys
+from typing import TYPE_CHECKING, Any, cast, Dict, List, Optional
+
+from pyspark.errors import StreamingQueryException
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.streaming.query import (
+    StreamingQuery as PySparkStreamingQuery,
+)
+
+__all__ = [
+    "StreamingQuery",  # TODO(WIP): "StreamingQueryManager"
+]
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+
+
+class StreamingQuery:
+
+    def __init__(self, session: "SparkSession", queryId: str, runId: str, name: Optional[str] = None) -> None:
+        self._session = session
+        self._query_id = queryId
+        self._run_id = runId
+        self._name = name
+
+    @property
+    def id(self) -> str:
+        return self._query_id
+
+    id.__doc__ = PySparkStreamingQuery.id.__doc__
+
+    @property
+    def runId(self) -> str:
+        return self._run_id
+
+    runId.__doc__ = PySparkStreamingQuery.runId.__doc__
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    name.__doc__ = PySparkStreamingQuery.name.__doc__
+
+    @property
+    def isActive(self) -> bool:
+        return self._fetch_status().is_active
+
+    isActive.__doc__ = PySparkStreamingQuery.isActive.__doc__
+
+    def awaitTermination(self, timeout: Optional[int] = None) -> Optional[bool]:
+        raise NotImplementedError()
+
+    awaitTermination.__doc__ = PySparkStreamingQuery.awaitTermination.__doc__
+
+    @property
+    def status(self) -> Dict[str, Any]:
+        proto = self._fetch_status()
+        return {
+            "message": proto.status_message,
+            "isDataAvailable": proto.is_data_available,
+            "isTriggerActive": proto.is_trigger_active
+        }
+
+    status.__doc__ = PySparkStreamingQuery.status.__doc__
+
+    @property
+    def recentProgress(self) -> List[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=-1).recent_progress_json)
+        return [json.loads(p) for p in progress]
+
+    recentProgress.__doc__ = PySparkStreamingQuery.recentProgress.__doc__
+
+    @property
+    def lastProgress(self) -> Optional[Dict[str, Any]]:
+        progress = list(self._fetch_status(recent_progress_limit=1).recent_progress_json)
+        if len(progress) > 0:
+            return json.loads(progress[-1])
+        else:
+            return None
+
+    lastProgress.__doc__ = PySparkStreamingQuery.lastProgress.__doc__
+
+    def processAllAvailable(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.process_all_available = True
+        self._execute_streaming_query_cmd(cmd)
+
+    processAllAvailable.__doc__ = PySparkStreamingQuery.processAllAvailable.__doc__
+
+    def stop(self) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.stop = True
+        self._execute_streaming_query_cmd(cmd)
+
+    stop.__doc__ = PySparkStreamingQuery.stop.__doc__
+
+    def explain(self, extended: bool = False) -> None:
+        cmd = pb2.StreamingQueryCommand()
+        cmd.explain.extended = extended
+        result = self._execute_streaming_query_cmd(cmd).explain.result
+        print(result)
+
+    processAllAvailable.__doc__ = PySparkStreamingQuery.processAllAvailable.__doc__

Review Comment:
   this should be explain?



##########
python/pyspark/sql/connect/streaming/readwriter.py:
##########
@@ -0,0 +1,485 @@
+#
+# 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.
+#
+
+from pyspark.sql.connect.utils import check_dependencies
+
+check_dependencies(__name__)
+
+import sys
+from typing import cast, overload, Callable, Dict, List, Optional, TYPE_CHECKING, Union
+
+from pyspark.sql.connect.plan import DataSource, LogicalPlan, WriteStreamOperation
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.connect.readwriter import OptionUtils, to_str
+from pyspark.sql.connect.streaming.query import StreamingQuery
+from pyspark.sql.streaming.readwriter import (
+    DataStreamReader as PySparkDataStreamReader,
+    DataStreamWriter as PySparkDataStreamWriter,
+)
+from pyspark.sql.types import Row, StructType
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect.session import SparkSession
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.dataframe import DataFrame
+
+__all__ = ["DataStreamReader", "DataStreamWriter"]
+
+
+class DataStreamReader(OptionUtils):
+
+    def __init__(self, client: "SparkSession") -> None:
+        self._format: Optional[str] = None
+        self._schema = ""
+        self._client = client
+        self._options: Dict[str, str] = {}
+
+    def _df(self, plan: LogicalPlan) -> "DataFrame":
+        from pyspark.sql.connect.dataframe import DataFrame
+
+        return DataFrame.withPlan(plan, self._client)
+
+    def format(self, source: str) -> "DataStreamReader":
+        self._format = source
+        return self
+
+    format.__doc__ = PySparkDataStreamReader.format.__doc__
+
+    def schema(self, schema: Union[StructType, str]) -> "DataStreamReader":
+        if isinstance(schema, StructType):
+            self._schema = schema.json()
+        elif isinstance(schema, str):
+            self._schema = schema
+        else:
+            raise TypeError("schema should be StructType or string")
+        return self
+
+    schema.__doc__ = PySparkDataStreamReader.schema.__doc__
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataStreamReader":
+        self._options[key] = str(value)
+        return self
+
+    option.__doc__ = PySparkDataStreamReader.option.__doc__
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataStreamReader":
+        for k in options:
+            self.option(k, to_str(options[k]))
+        return self
+
+    options.__doc__ = PySparkDataStreamReader.options.__doc__
+
+    def load(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> "DataFrame":
+        if format is not None:
+            self.format(format)
+        if schema is not None:
+            self.schema(schema)
+        self.options(**options)
+        if path is not None:
+            if type(path) != str or len(path.strip()) == 0:
+                raise ValueError(
+                    "If the path is provided for stream, it needs to be a "
+                    + "non-empty string. List of paths are not supported."
+                )
+
+        plan = DataSource(
+            format=self._format,
+            schema=self._schema,
+            options=self._options,
+            paths=[path] if path else None,
+            is_streaming=True
+        )
+
+        return self._df(plan)
+
+    load.__doc__ = PySparkDataStreamReader.load.__doc__
+
+    def json(
+        self,
+        path: str,
+        schema: Optional[Union[StructType, str]] = None,
+        primitivesAsString: Optional[Union[bool, str]] = None,
+        prefersDecimal: Optional[Union[bool, str]] = None,
+        allowComments: Optional[Union[bool, str]] = None,
+        allowUnquotedFieldNames: Optional[Union[bool, str]] = None,
+        allowSingleQuotes: Optional[Union[bool, str]] = None,
+        allowNumericLeadingZero: Optional[Union[bool, str]] = None,
+        allowBackslashEscapingAnyCharacter: Optional[Union[bool, str]] = None,
+        mode: Optional[str] = None,
+        columnNameOfCorruptRecord: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        multiLine: Optional[Union[bool, str]] = None,
+        allowUnquotedControlChars: Optional[Union[bool, str]] = None,
+        lineSep: Optional[str] = None,
+        locale: Optional[str] = None,
+        dropFieldIfAllNull: Optional[Union[bool, str]] = None,
+        encoding: Optional[str] = None,
+        pathGlobFilter: Optional[Union[bool, str]] = None,
+        recursiveFileLookup: Optional[Union[bool, str]] = None,
+        allowNonNumericNumbers: Optional[Union[bool, str]] = None,
+    ) -> "DataFrame":
+        self._set_opts(
+            schema=schema,
+            primitivesAsString=primitivesAsString,
+            prefersDecimal=prefersDecimal,
+            allowComments=allowComments,
+            allowUnquotedFieldNames=allowUnquotedFieldNames,
+            allowSingleQuotes=allowSingleQuotes,
+            allowNumericLeadingZero=allowNumericLeadingZero,
+            allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter,
+            mode=mode,
+            columnNameOfCorruptRecord=columnNameOfCorruptRecord,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            multiLine=multiLine,
+            allowUnquotedControlChars=allowUnquotedControlChars,
+            lineSep=lineSep,
+            locale=locale,
+            dropFieldIfAllNull=dropFieldIfAllNull,
+            encoding=encoding,
+            pathGlobFilter=pathGlobFilter,
+            recursiveFileLookup=recursiveFileLookup,
+            allowNonNumericNumbers=allowNonNumericNumbers,
+        )
+        if isinstance(path, str):
+            return self.load(path=path, format="json")
+        else:
+            raise TypeError("path can be only a single string")
+
+    json.__doc__ = PySparkDataStreamReader.json.__doc__
+
+    # def orc() TODO
+    # def parquet() TODO
+    # def text() TODO
+
+    def csv(
+        self,
+        path: str,
+        schema: Optional[Union[StructType, str]] = None,
+        sep: Optional[str] = None,
+        encoding: Optional[str] = None,
+        quote: Optional[str] = None,
+        escape: Optional[str] = None,
+        comment: Optional[str] = None,
+        header: Optional[Union[bool, str]] = None,
+        inferSchema: Optional[Union[bool, str]] = None,
+        ignoreLeadingWhiteSpace: Optional[Union[bool, str]] = None,
+        ignoreTrailingWhiteSpace: Optional[Union[bool, str]] = None,
+        nullValue: Optional[str] = None,
+        nanValue: Optional[str] = None,
+        positiveInf: Optional[str] = None,
+        negativeInf: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        maxColumns: Optional[Union[int, str]] = None,
+        maxCharsPerColumn: Optional[Union[int, str]] = None,
+        maxMalformedLogPerPartition: Optional[Union[int, str]] = None,
+        mode: Optional[str] = None,
+        columnNameOfCorruptRecord: Optional[str] = None,
+        multiLine: Optional[Union[bool, str]] = None,
+        charToEscapeQuoteEscaping: Optional[Union[bool, str]] = None,
+        enforceSchema: Optional[Union[bool, str]] = None,
+        emptyValue: Optional[str] = None,
+        locale: Optional[str] = None,
+        lineSep: Optional[str] = None,
+        pathGlobFilter: Optional[Union[bool, str]] = None,
+        recursiveFileLookup: Optional[Union[bool, str]] = None,
+        unescapedQuoteHandling: Optional[str] = None,
+    ) -> "DataFrame":
+        self._set_opts(
+            schema=schema,
+            sep=sep,
+            encoding=encoding,
+            quote=quote,
+            escape=escape,
+            comment=comment,
+            header=header,
+            inferSchema=inferSchema,
+            ignoreLeadingWhiteSpace=ignoreLeadingWhiteSpace,
+            ignoreTrailingWhiteSpace=ignoreTrailingWhiteSpace,
+            nullValue=nullValue,
+            nanValue=nanValue,
+            positiveInf=positiveInf,
+            negativeInf=negativeInf,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            maxColumns=maxColumns,
+            maxCharsPerColumn=maxCharsPerColumn,
+            maxMalformedLogPerPartition=maxMalformedLogPerPartition,
+            mode=mode,
+            columnNameOfCorruptRecord=columnNameOfCorruptRecord,
+            multiLine=multiLine,
+            charToEscapeQuoteEscaping=charToEscapeQuoteEscaping,
+            enforceSchema=enforceSchema,
+            emptyValue=emptyValue,
+            locale=locale,
+            lineSep=lineSep,
+            pathGlobFilter=pathGlobFilter,
+            recursiveFileLookup=recursiveFileLookup,
+            unescapedQuoteHandling=unescapedQuoteHandling,
+        )
+        if isinstance(path, str):
+            return self.load(path=path, format="csv")
+        else:
+            raise TypeError("path can be only a single string")
+
+    csv.__doc__ = PySparkDataStreamReader.csv.__doc__
+
+    # def table() TODO
+
+
+DataStreamReader.__doc__ = PySparkDataStreamReader.__doc__
+
+
+class DataStreamWriter:
+
+    def __init__(self, plan: "LogicalPlan", session: "SparkSession") -> None:
+        self._session = session
+        self._write_stream = WriteStreamOperation(plan)
+        self._write_proto = self._write_stream.write_op
+
+    def outputMode(self, outputMode: str) -> "DataStreamWriter":
+        self._write_proto.output_mode = outputMode
+        return self
+
+    outputMode.__doc__ = PySparkDataStreamWriter.outputMode.__doc__
+
+    def format(self, source: str) -> "DataStreamWriter":
+        self._write_proto.format = source
+        return self
+
+    format.__doc__ = PySparkDataStreamWriter.format.__doc__
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataStreamWriter":
+        self._write_proto.options[key] = to_str(value)
+        return self
+
+    option.__doc__ = PySparkDataStreamWriter.option.__doc__
+    
+    def options(self, **options: "OptionalPrimitiveType") -> "DataStreamWriter":
+        for k in options:
+            self.option(k, options[k])        
+        return self
+
+    options.__doc__ = PySparkDataStreamWriter.options.__doc__
+    
+    @overload
+    def partitionBy(self, *cols: str) -> "DataStreamWriter":
+        ...
+
+    @overload
+    def partitionBy(self, __cols: List[str]) -> "DataStreamWriter":
+        ...
+
+    def partitionBy(self, *cols: str) -> "DataStreamWriter":  # type: ignore[misc]
+        if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
+            cols = cols[0]
+        self._write_proto.partitioning_cols = cast(List[str], cols)
+        return self
+
+    partitionBy.__doc__ = PySparkDataStreamWriter.partitionBy.__doc__
+
+    def queryName(self, queryName: str) -> "DataStreamWriter":
+        self._write_proto.query_name = queryName
+        return self
+
+    queryName.__doc__ = PySparkDataStreamWriter.queryName.__doc__
+
+    @overload
+    def trigger(self, *, processingTime: str) -> "DataStreamWriter":
+        ...
+
+    @overload
+    def trigger(self, *, once: bool) -> "DataStreamWriter":
+        ...
+
+    @overload
+    def trigger(self, *, continuous: str) -> "DataStreamWriter":
+        ...
+
+    @overload
+    def trigger(self, *, availableNow: bool) -> "DataStreamWriter":
+        ...
+
+    def trigger(
+        self,
+        *,
+        processingTime: Optional[str] = None,
+        once: Optional[bool] = None,
+        continuous: Optional[str] = None,
+        availableNow: Optional[bool] = None,
+    ) -> "DataStreamWriter":
+        params = [processingTime, once, continuous, availableNow]
+
+        if params.count(None) == 4:
+            raise ValueError("No trigger provided")
+        elif params.count(None) < 3:
+            raise ValueError("Multiple triggers not allowed.")
+
+        if processingTime is not None:
+            if type(processingTime) != str or len(processingTime.strip()) == 0:
+                raise ValueError(
+                    "Value for processingTime must be a non empty string. Got: %s" % processingTime
+                )
+            self._write_proto.processing_time_interval = processingTime.strip()
+
+        elif once is not None:
+            if once is not True:
+                raise ValueError("Value for once must be True. Got: %s" % once)

Review Comment:
   why it must be True?



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


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

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on PR #40586:
URL: https://github.com/apache/spark/pull/40586#issuecomment-1498440221

   Merged to master.


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


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

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1153629770


##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,97 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+message WriteStreamOperation {

Review Comment:
   > However, the benefit of declaring fields as explicitly optional generates code that lets you easily check `hasField`.
   `// (Optional)`  comment affects the code generated? In that case we need to be even more careful in adding it. Protobuf V3 explicitly discouraged depending on `hasField()`. Unless there is a good reason, we should not depend on it. We could discuss this in slack.
   
   I will add comment to the fields, may be not '(Optional)' unless it is really required. 
   



##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,97 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+message WriteStreamOperation {
+
+  // (Required) The output of the `input` streaming relation will be written.
+  Relation input = 1;
+
+  string format = 2;
+  map<string, string> options = 3;
+  repeated string partitioning_column_names = 4;
+
+  oneof trigger {
+    string processing_time_interval = 5;
+    bool available_now = 6;
+    bool one_time = 7;
+    string continuous_checkpoint_interval = 8;
+  }
+
+  string output_mode = 9;
+  string query_name = 10;
+
+  oneof sink_destination {
+    string path = 11;
+    string table_name = 12;
+  }
+}
+
+message StreamingQueryStartResult {
+
+  string name = 1;
+
+  // (Required)
+  string id = 2;
+
+  // (Required)
+  string run_id = 3;
+
+  // TODO: How do we indicate errors?
+  // TODO: Consider adding StreamingQueryStatusResult here.
+}
+
+message StreamingQueryCommand {
+
+  // (Required)
+  string id = 1;
+
+  oneof command_type {
+      StatusCommand status = 2;
+      bool stop = 3;
+      bool process_all_available = 4;
+      ExplainCommand explain = 5;
+
+      // TODO(SPARK-42960) Add more commands: await_termination(), exception() etc.
+  }
+
+  message StatusCommand {
+    int32 recent_progress_limit = 1;
+  }
+
+  message ExplainCommand {

Review Comment:
   I tried to do this, but import heirarchy does not allow it. Existing Explain is defined inside `base.proto:AnalyzePlanRequest`. We can't import it in this file since `base.proto` already imports this file.
   Note that this `explain()` is not a Dataframe but on a Streaming query.
   I left TODO. cc: @grundprinzip about import hierarchy. 



##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,97 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+message WriteStreamOperation {
+
+  // (Required) The output of the `input` streaming relation will be written.
+  Relation input = 1;
+
+  string format = 2;
+  map<string, string> options = 3;
+  repeated string partitioning_column_names = 4;
+
+  oneof trigger {
+    string processing_time_interval = 5;
+    bool available_now = 6;
+    bool one_time = 7;
+    string continuous_checkpoint_interval = 8;
+  }
+
+  string output_mode = 9;

Review Comment:
   I thought about it, but the interface requires string. Didn't see much benefit in making it an enum here. It needs to be checked by the server anyway. 



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -1681,6 +1722,10 @@ class SparkConnectPlanner(val session: SparkSession) {
         handleCommandPlugin(command.getExtension)
       case proto.Command.CommandTypeCase.SQL_COMMAND =>
         handleSqlCommand(command.getSqlCommand, sessionId, responseObserver)
+      case proto.Command.CommandTypeCase.WRITE_STREAM_OPERATION =>
+        handleWriteStreamOperation(command.getWriteStreamOperation, sessionId, responseObserver)
+      case proto.Command.CommandTypeCase.STREAMING_QUERY_COMMAND =>
+        handleStreamingQueryCommand(command.getStreamingQueryCommand, sessionId, responseObserver)

Review Comment:
   Thank you! It was already used in `handleSqlCommand()`. 



##########
python/pyspark/sql/connect/streaming/__init__.py:
##########
@@ -0,0 +1,21 @@
+#
+# 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.
+#
+
+from pyspark.sql.connect.streaming.query import StreamingQuery  # StreamingQueryManager  noqa: F401
+from pyspark.sql.connect.streaming.readwriter import DataStreamReader, DataStreamWriter  # noqa: F401
+# from pyspark.sql.connect.streaming.listener import StreamingQueryListener  # noqa: F401

Review Comment:
   We don't have `StreamingQueryListener` yet. I can remove the line. 



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -762,22 +779,32 @@ class SparkConnectPlanner(val session: SparkSession) {
   }
 
   private def transformReadRel(rel: proto.Read): LogicalPlan = {
+
+    def parseSchema(schema: String): StructType = {
+      DataType.parseTypeWithFallback(
+        schema,
+        StructType.fromDDL,
+        fallbackParser = DataType.fromJson) match {
+        case s: StructType => s
+        case other => throw InvalidPlanInput(s"Invalid schema $other")
+      }
+    }

Review Comment:
   Adding to StructType will be a larger change. 
   I can make this a private method. 
   Just didn't feel like duplicating the code. 



##########
connector/connect/common/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -729,6 +736,18 @@ message WithColumns {
   repeated Expression.Alias aliases = 2;
 }
 
+message WithWatermark {
+
+  // (Required) The input relation
+  Relation input = 1;
+
+  // (Required)
+  string event_time = 2;
+
+  // (Required)
+  string delay_threshold = 3;

Review Comment:
   Again matching the API. It is a duration expressed in a string. E.g. "10 minutes".



##########
connector/connect/common/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -729,6 +736,18 @@ message WithColumns {
   repeated Expression.Alias aliases = 2;
 }
 
+message WithWatermark {
+
+  // (Required) The input relation
+  Relation input = 1;
+
+  // (Required)
+  string event_time = 2;

Review Comment:
   String, matching the API. It represents column name. I will add a comment. 



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -1969,6 +2014,135 @@ class SparkConnectPlanner(val session: SparkSession) {
     }
   }
 
+  def handleWriteStreamOperation(
+      writeOp: WriteStreamOperation,
+      sessionId: String,
+      responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
+    val plan = transformRelation(writeOp.getInput)
+    val dataset = Dataset.ofRows(session, logicalPlan = plan)
+
+    val writer = dataset.writeStream
+
+    if (writeOp.getFormat.nonEmpty) {
+      writer.format(writeOp.getFormat)
+    }
+
+    writer.options(writeOp.getOptionsMap)
+
+    if (writeOp.getPartitioningColumnNamesCount > 0) {
+      writer.partitionBy(writeOp.getPartitioningColumnNamesList.asScala.toList: _*)
+    }
+
+    writeOp.getTriggerCase match {
+      case TriggerCase.PROCESSING_TIME_INTERVAL =>
+        writer.trigger(Trigger.ProcessingTime(writeOp.getProcessingTimeInterval))
+      case TriggerCase.AVAILABLE_NOW =>
+        writer.trigger(Trigger.AvailableNow())
+      case TriggerCase.ONE_TIME =>
+        writer.trigger(Trigger.Once())
+      case TriggerCase.CONTINUOUS_CHECKPOINT_INTERVAL =>
+        writer.trigger(Trigger.Continuous(writeOp.getContinuousCheckpointInterval))
+      case TriggerCase.TRIGGER_NOT_SET =>
+    }
+
+    if (writeOp.getOutputMode.nonEmpty) {
+      writer.outputMode(writeOp.getOutputMode)
+    }
+
+    if (writeOp.getQueryName.nonEmpty) {
+      writer.queryName(writeOp.getQueryName)
+    }
+
+    val query = writeOp.getPath match {
+      case "" if writeOp.hasTableName => writer.toTable(writeOp.getTableName)
+      case "" => writer.start()
+      case path => writer.start(path)
+    }
+
+    val result = StreamingQueryStartResult
+      .newBuilder()
+      .setId(query.id.toString)
+      .setRunId(query.runId.toString)
+      .setName(Option(query.name).getOrElse(""))
+      .build()
+
+    responseObserver.onNext(
+      ExecutePlanResponse
+        .newBuilder()
+        .setSessionId(sessionId)
+        .setStreamingQueryStartResult(result)
+        .build())
+  }
+
+  def handleStreamingQueryCommand(
+      command: StreamingQueryCommand,
+      sessionId: String,
+      responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
+
+    val id = command.getId
+
+    val respBuilder = StreamingQueryCommandResult
+      .newBuilder()
+      .setId(command.getId)
+
+    val query = Option(session.streams.get(command.getId)).getOrElse {
+      throw new IllegalArgumentException(s"Streaming query $id is not found")
+      // TODO(SPARK-42962): Handle this better. May be cache stopped queries for a few minutes.
+    }
+
+    command.getCommandTypeCase match {
+      case StreamingQueryCommand.CommandTypeCase.STATUS =>
+        val recentProgress: Seq[String] = command.getStatus.getRecentProgressLimit match {
+          case 0 => Seq.empty
+          case limit if limit < 0 =>
+            query.recentProgress.map(_.json) // All the cached progresses.
+          case limit => query.recentProgress.takeRight(limit).map(_.json) // Most recent
+        }
+
+        val queryStatus = query.status
+
+        val statusResult = StreamingQueryCommandResult.StatusResult
+          .newBuilder()
+          .setStatusMessage(queryStatus.message)
+          .setIsDataAvailable(queryStatus.isDataAvailable)
+          .setIsTriggerActive(queryStatus.isTriggerActive)
+          .setIsActive(query.isActive)
+          .addAllRecentProgressJson(recentProgress.asJava)
+          .build()
+
+        respBuilder.setStatus(statusResult)
+
+      case StreamingQueryCommand.CommandTypeCase.STOP =>
+        query.stop()
+
+      case StreamingQueryCommand.CommandTypeCase.PROCESS_ALL_AVAILABLE =>
+        query.processAllAvailable()

Review Comment:
   Yes. 
   We might update `stop()` to return latest status. May be I should do that now. 



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


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

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1153813344


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -1969,6 +2014,136 @@ class SparkConnectPlanner(val session: SparkSession) {
     }
   }
 
+  def handleWriteStreamOperation(
+      writeOp: WriteStreamOperation,
+      sessionId: String,
+      responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
+    val plan = transformRelation(writeOp.getInput)
+    val dataset = Dataset.ofRows(session, logicalPlan = plan)
+
+    val writer = dataset.writeStream
+
+    if (writeOp.getFormat.nonEmpty) {
+      writer.format(writeOp.getFormat)
+    }
+
+    writer.options(writeOp.getOptionsMap)
+
+    if (writeOp.getPartitioningColumnNamesCount > 0) {
+      writer.partitionBy(writeOp.getPartitioningColumnNamesList.asScala.toList: _*)
+    }
+
+    writeOp.getTriggerCase match {
+      case TriggerCase.PROCESSING_TIME_INTERVAL =>
+        writer.trigger(Trigger.ProcessingTime(writeOp.getProcessingTimeInterval))
+      case TriggerCase.AVAILABLE_NOW =>
+        writer.trigger(Trigger.AvailableNow())
+      case TriggerCase.ONE_TIME =>
+        writer.trigger(Trigger.Once())
+      case TriggerCase.CONTINUOUS_CHECKPOINT_INTERVAL =>
+        writer.trigger(Trigger.Continuous(writeOp.getContinuousCheckpointInterval))
+      case TriggerCase.TRIGGER_NOT_SET =>
+    }
+
+    if (writeOp.getOutputMode.nonEmpty) {
+      writer.outputMode(writeOp.getOutputMode)
+    }
+
+    if (writeOp.getQueryName.nonEmpty) {
+      writer.queryName(writeOp.getQueryName)
+    }
+
+    val query = writeOp.getPath match {
+      case "" if writeOp.hasTableName => writer.toTable(writeOp.getTableName)
+      case "" => writer.start()
+      case path => writer.start(path)
+    }
+
+    val result = StreamingQueryStartResult
+      .newBuilder()
+      .setId(query.id.toString)
+      .setRunId(query.runId.toString)
+      .setName(Option(query.name).getOrElse(""))
+      .build()
+
+    responseObserver.onNext(
+      ExecutePlanResponse
+        .newBuilder()
+        .setSessionId(sessionId)
+        .setStreamingQueryStartResult(result)
+        .build())
+  }
+
+  def handleStreamingQueryCommand(
+      command: StreamingQueryCommand,
+      sessionId: String,
+      responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
+
+    val id = command.getId
+
+    val respBuilder = StreamingQueryCommandResult
+      .newBuilder()
+      .setId(command.getId)
+
+    val query = Option(session.streams.get(command.getId)).getOrElse {
+      throw new IllegalArgumentException(s"Streaming query $id is not found")
+      // TODO(SPARK-42962): Handle this better. May be cache stopped queries for a few minutes.
+    }
+
+    System.currentTimeMillis()
+    command.getCommandTypeCase match {
+      case StreamingQueryCommand.CommandTypeCase.STATUS =>
+        val recentProgress: Seq[String] = command.getStatus.getRecentProgressLimit match {
+          case 0 => Seq.empty
+          case limit if limit < 0 =>
+            query.recentProgress.map(_.json) // All the cached progresses.
+          case limit => query.recentProgress.takeRight(limit).map(_.json) // Most recent
+        }
+
+        val queryStatus = query.status
+
+        val statusResult = StreamingQueryCommandResult.StatusResult
+          .newBuilder()
+          .setStatusMessage(queryStatus.message)
+          .setIsDataAvailable(queryStatus.isDataAvailable)
+          .setIsTriggerActive(queryStatus.isTriggerActive)
+          .setIsActive(query.isActive)
+          .addAllRecentProgressJson(recentProgress.asJava)
+          .build()
+
+        respBuilder.setStatus(statusResult)
+
+      case StreamingQueryCommand.CommandTypeCase.STOP =>
+        query.stop()
+
+      case StreamingQueryCommand.CommandTypeCase.PROCESS_ALL_AVAILABLE =>
+        query.processAllAvailable()
+
+      case StreamingQueryCommand.CommandTypeCase.EXPLAIN =>
+        val result = query match {
+          case q: StreamExecution => q.explainInternal(command.getExplain.getExtended)

Review Comment:
   Removed. 



##########
connector/connect/common/src/main/protobuf/spark/connect/base.proto:
##########
@@ -298,6 +298,10 @@ message ExecutePlanResponse {
     // Special case for executing SQL commands.
     SqlCommandResult sql_command_result = 5;
 
+    StreamingQueryStartResult streaming_query_start_result = 8;
+
+    StreamingQueryCommandResult streaming_query_command_result = 9;

Review Comment:
   Done.



##########
connector/connect/common/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -729,6 +736,18 @@ message WithColumns {
   repeated Expression.Alias aliases = 2;
 }
 
+message WithWatermark {
+
+  // (Required) The input relation
+  Relation input = 1;
+
+  // (Required)
+  string event_time = 2;

Review Comment:
   Updated the the comment. API documentation is the full documentation. 



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -1969,6 +2014,135 @@ class SparkConnectPlanner(val session: SparkSession) {
     }
   }
 
+  def handleWriteStreamOperation(
+      writeOp: WriteStreamOperation,
+      sessionId: String,
+      responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
+    val plan = transformRelation(writeOp.getInput)
+    val dataset = Dataset.ofRows(session, logicalPlan = plan)
+
+    val writer = dataset.writeStream
+
+    if (writeOp.getFormat.nonEmpty) {
+      writer.format(writeOp.getFormat)
+    }
+
+    writer.options(writeOp.getOptionsMap)
+
+    if (writeOp.getPartitioningColumnNamesCount > 0) {
+      writer.partitionBy(writeOp.getPartitioningColumnNamesList.asScala.toList: _*)
+    }
+
+    writeOp.getTriggerCase match {
+      case TriggerCase.PROCESSING_TIME_INTERVAL =>
+        writer.trigger(Trigger.ProcessingTime(writeOp.getProcessingTimeInterval))
+      case TriggerCase.AVAILABLE_NOW =>
+        writer.trigger(Trigger.AvailableNow())
+      case TriggerCase.ONE_TIME =>
+        writer.trigger(Trigger.Once())
+      case TriggerCase.CONTINUOUS_CHECKPOINT_INTERVAL =>
+        writer.trigger(Trigger.Continuous(writeOp.getContinuousCheckpointInterval))
+      case TriggerCase.TRIGGER_NOT_SET =>
+    }
+
+    if (writeOp.getOutputMode.nonEmpty) {
+      writer.outputMode(writeOp.getOutputMode)
+    }
+
+    if (writeOp.getQueryName.nonEmpty) {
+      writer.queryName(writeOp.getQueryName)
+    }
+
+    val query = writeOp.getPath match {
+      case "" if writeOp.hasTableName => writer.toTable(writeOp.getTableName)
+      case "" => writer.start()
+      case path => writer.start(path)
+    }
+
+    val result = StreamingQueryStartResult
+      .newBuilder()
+      .setId(query.id.toString)
+      .setRunId(query.runId.toString)
+      .setName(Option(query.name).getOrElse(""))
+      .build()
+
+    responseObserver.onNext(
+      ExecutePlanResponse
+        .newBuilder()
+        .setSessionId(sessionId)
+        .setStreamingQueryStartResult(result)
+        .build())
+  }
+
+  def handleStreamingQueryCommand(
+      command: StreamingQueryCommand,
+      sessionId: String,
+      responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
+
+    val id = command.getId
+
+    val respBuilder = StreamingQueryCommandResult
+      .newBuilder()
+      .setId(command.getId)
+
+    val query = Option(session.streams.get(command.getId)).getOrElse {
+      throw new IllegalArgumentException(s"Streaming query $id is not found")
+      // TODO(SPARK-42962): Handle this better. May be cache stopped queries for a few minutes.
+    }
+
+    command.getCommandTypeCase match {
+      case StreamingQueryCommand.CommandTypeCase.STATUS =>
+        val recentProgress: Seq[String] = command.getStatus.getRecentProgressLimit match {
+          case 0 => Seq.empty
+          case limit if limit < 0 =>
+            query.recentProgress.map(_.json) // All the cached progresses.
+          case limit => query.recentProgress.takeRight(limit).map(_.json) // Most recent
+        }
+
+        val queryStatus = query.status
+
+        val statusResult = StreamingQueryCommandResult.StatusResult
+          .newBuilder()
+          .setStatusMessage(queryStatus.message)
+          .setIsDataAvailable(queryStatus.isDataAvailable)
+          .setIsTriggerActive(queryStatus.isTriggerActive)
+          .setIsActive(query.isActive)
+          .addAllRecentProgressJson(recentProgress.asJava)
+          .build()
+
+        respBuilder.setStatus(statusResult)
+
+      case StreamingQueryCommand.CommandTypeCase.STOP =>
+        query.stop()
+
+      case StreamingQueryCommand.CommandTypeCase.PROCESS_ALL_AVAILABLE =>
+        query.processAllAvailable()

Review Comment:
   Left a comment to consider returning more status in the future for stop().



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -762,22 +779,32 @@ class SparkConnectPlanner(val session: SparkSession) {
   }
 
   private def transformReadRel(rel: proto.Read): LogicalPlan = {
+
+    def parseSchema(schema: String): StructType = {
+      DataType.parseTypeWithFallback(
+        schema,
+        StructType.fromDDL,
+        fallbackParser = DataType.fromJson) match {
+        case s: StructType => s
+        case other => throw InvalidPlanInput(s"Invalid schema $other")
+      }
+    }

Review Comment:
   Moved to a private method with short scala doc.



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


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

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1153908437


##########
python/pyspark/sql/connect/session.py:
##########
@@ -489,10 +495,6 @@ def sparkContext(self) -> Any:
     def streams(self) -> Any:
         raise NotImplementedError("streams() is not implemented.")
 
-    @property
-    def readStream(self) -> Any:
-        raise NotImplementedError("readStream() is not implemented.")
-

Review Comment:
   We are keeping the file names, class names, file paths etc similar between pyspark & connect. Thought it would be nice to keep the relative position of API methods also the same. E.g. `readStream()` follows `read()`. 



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


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

Posted by "zhengruifeng (via GitHub)" <gi...@apache.org>.
zhengruifeng commented on PR #40586:
URL: https://github.com/apache/spark/pull/40586#issuecomment-1524690074

   @rangadi It seems the doctest `pyspark.sql.connect.dataframe.DataFrame.writeStream` is not stable, would you mind taking a look?
   
   https://github.com/apache/spark/actions/runs/4815479364/jobs/8574219355
   
   https://github.com/apache/spark/actions/runs/4805357601/jobs/8551705966
   
   https://github.com/apache/spark/actions/runs/4783998036/jobs/8504986700


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


[GitHub] [spark] rangadi commented on pull request #40586: [SPARK-42939] Core streaming Python API for Spark Connect

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on PR #40586:
URL: https://github.com/apache/spark/pull/40586#issuecomment-1488040738

   cc: @HyukjinKwon, @pengzhon-db, @WweiL, @grundprinzip 
   This is ready for review. I will fix the conflicts. 


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


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

Posted by "WweiL (via GitHub)" <gi...@apache.org>.
WweiL commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1152296460


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -1969,6 +2014,136 @@ class SparkConnectPlanner(val session: SparkSession) {
     }
   }
 
+  def handleWriteStreamOperation(
+      writeOp: WriteStreamOperation,
+      sessionId: String,
+      responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
+    val plan = transformRelation(writeOp.getInput)
+    val dataset = Dataset.ofRows(session, logicalPlan = plan)
+
+    val writer = dataset.writeStream
+
+    if (writeOp.getFormat.nonEmpty) {
+      writer.format(writeOp.getFormat)
+    }
+
+    writer.options(writeOp.getOptionsMap)
+
+    if (writeOp.getPartitioningColumnNamesCount > 0) {
+      writer.partitionBy(writeOp.getPartitioningColumnNamesList.asScala.toList: _*)
+    }
+
+    writeOp.getTriggerCase match {
+      case TriggerCase.PROCESSING_TIME_INTERVAL =>
+        writer.trigger(Trigger.ProcessingTime(writeOp.getProcessingTimeInterval))
+      case TriggerCase.AVAILABLE_NOW =>
+        writer.trigger(Trigger.AvailableNow())
+      case TriggerCase.ONE_TIME =>
+        writer.trigger(Trigger.Once())
+      case TriggerCase.CONTINUOUS_CHECKPOINT_INTERVAL =>
+        writer.trigger(Trigger.Continuous(writeOp.getContinuousCheckpointInterval))
+      case TriggerCase.TRIGGER_NOT_SET =>
+    }
+
+    if (writeOp.getOutputMode.nonEmpty) {
+      writer.outputMode(writeOp.getOutputMode)
+    }
+
+    if (writeOp.getQueryName.nonEmpty) {
+      writer.queryName(writeOp.getQueryName)
+    }
+
+    val query = writeOp.getPath match {
+      case "" if writeOp.hasTableName => writer.toTable(writeOp.getTableName)
+      case "" => writer.start()
+      case path => writer.start(path)
+    }
+
+    val result = StreamingQueryStartResult
+      .newBuilder()
+      .setId(query.id.toString)
+      .setRunId(query.runId.toString)
+      .setName(Option(query.name).getOrElse(""))
+      .build()
+
+    responseObserver.onNext(
+      ExecutePlanResponse
+        .newBuilder()
+        .setSessionId(sessionId)
+        .setStreamingQueryStartResult(result)
+        .build())
+  }
+
+  def handleStreamingQueryCommand(
+      command: StreamingQueryCommand,
+      sessionId: String,
+      responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
+
+    val id = command.getId
+
+    val respBuilder = StreamingQueryCommandResult
+      .newBuilder()
+      .setId(command.getId)
+
+    val query = Option(session.streams.get(command.getId)).getOrElse {
+      throw new IllegalArgumentException(s"Streaming query $id is not found")
+      // TODO(SPARK-42962): Handle this better. May be cache stopped queries for a few minutes.
+    }
+
+    System.currentTimeMillis()

Review Comment:
   Should this be deleted?



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


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

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1157037072


##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -0,0 +1,181 @@
+#
+# 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.
+#
+
+import json
+import sys
+from typing import TYPE_CHECKING, Any, cast, Dict, List, Optional
+
+from pyspark.errors import StreamingQueryException
+import pyspark.sql.connect.proto as pb2
+from pyspark.sql.streaming.query import (
+    StreamingQuery as PySparkStreamingQuery,
+)
+
+__all__ = [
+    "StreamingQuery",  # TODO(WIP): "StreamingQueryManager"

Review Comment:
   Should ideally file a JIRA for `# TODO(WIP): "StreamingQueryManager"`, and fix it like `# TODO(SPARK-XXXXX): "StreamingQueryManager"`



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


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

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1152315438


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -1969,6 +2014,136 @@ class SparkConnectPlanner(val session: SparkSession) {
     }
   }
 
+  def handleWriteStreamOperation(
+      writeOp: WriteStreamOperation,
+      sessionId: String,
+      responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
+    val plan = transformRelation(writeOp.getInput)
+    val dataset = Dataset.ofRows(session, logicalPlan = plan)
+
+    val writer = dataset.writeStream
+
+    if (writeOp.getFormat.nonEmpty) {
+      writer.format(writeOp.getFormat)
+    }
+
+    writer.options(writeOp.getOptionsMap)
+
+    if (writeOp.getPartitioningColumnNamesCount > 0) {
+      writer.partitionBy(writeOp.getPartitioningColumnNamesList.asScala.toList: _*)
+    }
+
+    writeOp.getTriggerCase match {
+      case TriggerCase.PROCESSING_TIME_INTERVAL =>
+        writer.trigger(Trigger.ProcessingTime(writeOp.getProcessingTimeInterval))
+      case TriggerCase.AVAILABLE_NOW =>
+        writer.trigger(Trigger.AvailableNow())
+      case TriggerCase.ONE_TIME =>
+        writer.trigger(Trigger.Once())
+      case TriggerCase.CONTINUOUS_CHECKPOINT_INTERVAL =>
+        writer.trigger(Trigger.Continuous(writeOp.getContinuousCheckpointInterval))
+      case TriggerCase.TRIGGER_NOT_SET =>
+    }
+
+    if (writeOp.getOutputMode.nonEmpty) {
+      writer.outputMode(writeOp.getOutputMode)
+    }
+
+    if (writeOp.getQueryName.nonEmpty) {
+      writer.queryName(writeOp.getQueryName)
+    }
+
+    val query = writeOp.getPath match {
+      case "" if writeOp.hasTableName => writer.toTable(writeOp.getTableName)
+      case "" => writer.start()
+      case path => writer.start(path)
+    }
+
+    val result = StreamingQueryStartResult
+      .newBuilder()
+      .setId(query.id.toString)
+      .setRunId(query.runId.toString)
+      .setName(Option(query.name).getOrElse(""))
+      .build()
+
+    responseObserver.onNext(
+      ExecutePlanResponse
+        .newBuilder()
+        .setSessionId(sessionId)
+        .setStreamingQueryStartResult(result)
+        .build())
+  }
+
+  def handleStreamingQueryCommand(
+      command: StreamingQueryCommand,
+      sessionId: String,
+      responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
+
+    val id = command.getId
+
+    val respBuilder = StreamingQueryCommandResult
+      .newBuilder()
+      .setId(command.getId)
+
+    val query = Option(session.streams.get(command.getId)).getOrElse {
+      throw new IllegalArgumentException(s"Streaming query $id is not found")
+      // TODO(SPARK-42962): Handle this better. May be cache stopped queries for a few minutes.
+    }
+
+    System.currentTimeMillis()

Review Comment:
   Yep. Removed. 



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -1969,6 +2014,136 @@ class SparkConnectPlanner(val session: SparkSession) {
     }
   }
 
+  def handleWriteStreamOperation(
+      writeOp: WriteStreamOperation,
+      sessionId: String,
+      responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
+    val plan = transformRelation(writeOp.getInput)
+    val dataset = Dataset.ofRows(session, logicalPlan = plan)
+
+    val writer = dataset.writeStream
+
+    if (writeOp.getFormat.nonEmpty) {
+      writer.format(writeOp.getFormat)
+    }
+
+    writer.options(writeOp.getOptionsMap)
+
+    if (writeOp.getPartitioningColumnNamesCount > 0) {
+      writer.partitionBy(writeOp.getPartitioningColumnNamesList.asScala.toList: _*)
+    }
+
+    writeOp.getTriggerCase match {
+      case TriggerCase.PROCESSING_TIME_INTERVAL =>
+        writer.trigger(Trigger.ProcessingTime(writeOp.getProcessingTimeInterval))
+      case TriggerCase.AVAILABLE_NOW =>
+        writer.trigger(Trigger.AvailableNow())
+      case TriggerCase.ONE_TIME =>
+        writer.trigger(Trigger.Once())
+      case TriggerCase.CONTINUOUS_CHECKPOINT_INTERVAL =>
+        writer.trigger(Trigger.Continuous(writeOp.getContinuousCheckpointInterval))
+      case TriggerCase.TRIGGER_NOT_SET =>
+    }
+
+    if (writeOp.getOutputMode.nonEmpty) {
+      writer.outputMode(writeOp.getOutputMode)
+    }
+
+    if (writeOp.getQueryName.nonEmpty) {
+      writer.queryName(writeOp.getQueryName)
+    }
+
+    val query = writeOp.getPath match {
+      case "" if writeOp.hasTableName => writer.toTable(writeOp.getTableName)
+      case "" => writer.start()
+      case path => writer.start(path)
+    }
+
+    val result = StreamingQueryStartResult
+      .newBuilder()
+      .setId(query.id.toString)
+      .setRunId(query.runId.toString)
+      .setName(Option(query.name).getOrElse(""))
+      .build()
+
+    responseObserver.onNext(
+      ExecutePlanResponse
+        .newBuilder()
+        .setSessionId(sessionId)
+        .setStreamingQueryStartResult(result)
+        .build())
+  }
+
+  def handleStreamingQueryCommand(
+      command: StreamingQueryCommand,
+      sessionId: String,
+      responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
+
+    val id = command.getId
+
+    val respBuilder = StreamingQueryCommandResult
+      .newBuilder()
+      .setId(command.getId)
+
+    val query = Option(session.streams.get(command.getId)).getOrElse {
+      throw new IllegalArgumentException(s"Streaming query $id is not found")
+      // TODO(SPARK-42962): Handle this better. May be cache stopped queries for a few minutes.
+    }
+
+    System.currentTimeMillis()
+    command.getCommandTypeCase match {
+      case StreamingQueryCommand.CommandTypeCase.STATUS =>
+        val recentProgress: Seq[String] = command.getStatus.getRecentProgressLimit match {
+          case 0 => Seq.empty
+          case limit if limit < 0 =>
+            query.recentProgress.map(_.json) // All the cached progresses.
+          case limit => query.recentProgress.takeRight(limit).map(_.json) // Most recent
+        }
+
+        val queryStatus = query.status
+
+        val statusResult = StreamingQueryCommandResult.StatusResult
+          .newBuilder()
+          .setStatusMessage(queryStatus.message)
+          .setIsDataAvailable(queryStatus.isDataAvailable)
+          .setIsTriggerActive(queryStatus.isTriggerActive)
+          .setIsActive(query.isActive)
+          .addAllRecentProgressJson(recentProgress.asJava)
+          .build()
+
+        respBuilder.setStatus(statusResult)
+
+      case StreamingQueryCommand.CommandTypeCase.STOP =>
+        query.stop()
+
+      case StreamingQueryCommand.CommandTypeCase.PROCESS_ALL_AVAILABLE =>
+        query.processAllAvailable()
+
+      case StreamingQueryCommand.CommandTypeCase.EXPLAIN =>
+        val result = query match {
+          case q: StreamExecution => q.explainInternal(command.getExplain.getExtended)

Review Comment:
   Yeah, we could. I was wondering about it too.



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


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

Posted by "WweiL (via GitHub)" <gi...@apache.org>.
WweiL commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1152307065


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -1969,6 +2014,136 @@ class SparkConnectPlanner(val session: SparkSession) {
     }
   }
 
+  def handleWriteStreamOperation(
+      writeOp: WriteStreamOperation,
+      sessionId: String,
+      responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
+    val plan = transformRelation(writeOp.getInput)
+    val dataset = Dataset.ofRows(session, logicalPlan = plan)
+
+    val writer = dataset.writeStream
+
+    if (writeOp.getFormat.nonEmpty) {
+      writer.format(writeOp.getFormat)
+    }
+
+    writer.options(writeOp.getOptionsMap)
+
+    if (writeOp.getPartitioningColumnNamesCount > 0) {
+      writer.partitionBy(writeOp.getPartitioningColumnNamesList.asScala.toList: _*)
+    }
+
+    writeOp.getTriggerCase match {
+      case TriggerCase.PROCESSING_TIME_INTERVAL =>
+        writer.trigger(Trigger.ProcessingTime(writeOp.getProcessingTimeInterval))
+      case TriggerCase.AVAILABLE_NOW =>
+        writer.trigger(Trigger.AvailableNow())
+      case TriggerCase.ONE_TIME =>
+        writer.trigger(Trigger.Once())
+      case TriggerCase.CONTINUOUS_CHECKPOINT_INTERVAL =>
+        writer.trigger(Trigger.Continuous(writeOp.getContinuousCheckpointInterval))
+      case TriggerCase.TRIGGER_NOT_SET =>
+    }
+
+    if (writeOp.getOutputMode.nonEmpty) {
+      writer.outputMode(writeOp.getOutputMode)
+    }
+
+    if (writeOp.getQueryName.nonEmpty) {
+      writer.queryName(writeOp.getQueryName)
+    }
+
+    val query = writeOp.getPath match {
+      case "" if writeOp.hasTableName => writer.toTable(writeOp.getTableName)
+      case "" => writer.start()
+      case path => writer.start(path)
+    }
+
+    val result = StreamingQueryStartResult
+      .newBuilder()
+      .setId(query.id.toString)
+      .setRunId(query.runId.toString)
+      .setName(Option(query.name).getOrElse(""))
+      .build()
+
+    responseObserver.onNext(
+      ExecutePlanResponse
+        .newBuilder()
+        .setSessionId(sessionId)
+        .setStreamingQueryStartResult(result)
+        .build())
+  }
+
+  def handleStreamingQueryCommand(
+      command: StreamingQueryCommand,
+      sessionId: String,
+      responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
+
+    val id = command.getId
+
+    val respBuilder = StreamingQueryCommandResult
+      .newBuilder()
+      .setId(command.getId)
+
+    val query = Option(session.streams.get(command.getId)).getOrElse {
+      throw new IllegalArgumentException(s"Streaming query $id is not found")
+      // TODO(SPARK-42962): Handle this better. May be cache stopped queries for a few minutes.
+    }
+
+    System.currentTimeMillis()
+    command.getCommandTypeCase match {
+      case StreamingQueryCommand.CommandTypeCase.STATUS =>
+        val recentProgress: Seq[String] = command.getStatus.getRecentProgressLimit match {
+          case 0 => Seq.empty
+          case limit if limit < 0 =>
+            query.recentProgress.map(_.json) // All the cached progresses.
+          case limit => query.recentProgress.takeRight(limit).map(_.json) // Most recent
+        }
+
+        val queryStatus = query.status
+
+        val statusResult = StreamingQueryCommandResult.StatusResult
+          .newBuilder()
+          .setStatusMessage(queryStatus.message)
+          .setIsDataAvailable(queryStatus.isDataAvailable)
+          .setIsTriggerActive(queryStatus.isTriggerActive)
+          .setIsActive(query.isActive)
+          .addAllRecentProgressJson(recentProgress.asJava)
+          .build()
+
+        respBuilder.setStatus(statusResult)
+
+      case StreamingQueryCommand.CommandTypeCase.STOP =>
+        query.stop()
+
+      case StreamingQueryCommand.CommandTypeCase.PROCESS_ALL_AVAILABLE =>
+        query.processAllAvailable()
+
+      case StreamingQueryCommand.CommandTypeCase.EXPLAIN =>
+        val result = query match {
+          case q: StreamExecution => q.explainInternal(command.getExplain.getExtended)

Review Comment:
   IIUC the query created is always a `StreamingQueryWrapper`, maybe this case could be deleted?
   
   https://github.com/apache/spark/blob/85abf73462b59573f34690366b6e631880d872ff/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala#L252



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


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

Posted by "amaliujia (via GitHub)" <gi...@apache.org>.
amaliujia commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1152388067


##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,97 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+message WriteStreamOperation {
+
+  // (Required) The output of the `input` streaming relation will be written.
+  Relation input = 1;
+
+  string format = 2;
+  map<string, string> options = 3;
+  repeated string partitioning_column_names = 4;
+
+  oneof trigger {
+    string processing_time_interval = 5;
+    bool available_now = 6;
+    bool one_time = 7;
+    string continuous_checkpoint_interval = 8;
+  }
+
+  string output_mode = 9;
+  string query_name = 10;
+
+  oneof sink_destination {
+    string path = 11;
+    string table_name = 12;
+  }
+}
+
+message StreamingQueryStartResult {
+
+  string name = 1;
+
+  // (Required)
+  string id = 2;
+
+  // (Required)
+  string run_id = 3;
+
+  // TODO: How do we indicate errors?
+  // TODO: Consider adding StreamingQueryStatusResult here.
+}
+
+message StreamingQueryCommand {
+
+  // (Required)
+  string id = 1;
+
+  oneof command_type {
+      StatusCommand status = 2;
+      bool stop = 3;
+      bool process_all_available = 4;
+      ExplainCommand explain = 5;
+
+      // TODO(SPARK-42960) Add more commands: await_termination(), exception() etc.
+  }
+
+  message StatusCommand {
+    int32 recent_progress_limit = 1;
+  }
+
+  message ExplainCommand {

Review Comment:
   Can we re-use the existing explain command?



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


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

Posted by "WweiL (via GitHub)" <gi...@apache.org>.
WweiL commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1153899173


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -2120,7 +2130,6 @@ class SparkConnectPlanner(val session: SparkSession) {
 
       case StreamingQueryCommand.CommandTypeCase.EXPLAIN =>
         val result = query match {
-          case q: StreamExecution => q.explainInternal(command.getExplain.getExtended)

Review Comment:
   Sorry double checked and found that the query was returned using `StreamingQueryManager`'s `get` method, which is StreamingQuery https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala#L106
   
   Following existing code maybe we could change this to 
   `case q: StreamingQuery` and do a cast like:
   https://github.com/apache/spark/blob/35503a535771d257b517e7ddf2adfaefefd97dad/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala#L249



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


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

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1153908962


##########
python/pyspark/sql/connect/session.py:
##########
@@ -14,6 +14,7 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
+

Review Comment:
   Done.



##########
python/pyspark/sql/connect/streaming/__init__.py:
##########
@@ -0,0 +1,21 @@
+#
+# 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.
+#
+
+from pyspark.sql.connect.streaming.query import StreamingQuery  # StreamingQueryManager  noqa: F401
+from pyspark.sql.connect.streaming.readwriter import DataStreamReader, DataStreamWriter  # noqa: F401
+# from pyspark.sql.connect.streaming.listener import StreamingQueryListener  # noqa: F401

Review Comment:
   Removed. 



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


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

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1156583318


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -1969,6 +2017,142 @@ class SparkConnectPlanner(val session: SparkSession) {
     }
   }
 
+  def handleWriteStreamOperationStart(
+      writeOp: WriteStreamOperationStart,
+      sessionId: String,
+      responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
+    val plan = transformRelation(writeOp.getInput)
+    val dataset = Dataset.ofRows(session, logicalPlan = plan)
+
+    val writer = dataset.writeStream
+
+    if (writeOp.getFormat.nonEmpty) {
+      writer.format(writeOp.getFormat)
+    }
+
+    writer.options(writeOp.getOptionsMap)
+
+    if (writeOp.getPartitioningColumnNamesCount > 0) {
+      writer.partitionBy(writeOp.getPartitioningColumnNamesList.asScala.toList: _*)
+    }
+
+    writeOp.getTriggerCase match {
+      case TriggerCase.PROCESSING_TIME_INTERVAL =>
+        writer.trigger(Trigger.ProcessingTime(writeOp.getProcessingTimeInterval))
+      case TriggerCase.AVAILABLE_NOW =>
+        writer.trigger(Trigger.AvailableNow())
+      case TriggerCase.ONE_TIME =>
+        writer.trigger(Trigger.Once())
+      case TriggerCase.CONTINUOUS_CHECKPOINT_INTERVAL =>
+        writer.trigger(Trigger.Continuous(writeOp.getContinuousCheckpointInterval))
+      case TriggerCase.TRIGGER_NOT_SET =>
+    }
+
+    if (writeOp.getOutputMode.nonEmpty) {
+      writer.outputMode(writeOp.getOutputMode)
+    }
+
+    if (writeOp.getQueryName.nonEmpty) {
+      writer.queryName(writeOp.getQueryName)
+    }
+
+    val query = writeOp.getPath match {
+      case "" if writeOp.hasTableName => writer.toTable(writeOp.getTableName)
+      case "" => writer.start()
+      case path => writer.start(path)
+    }
+
+    val result = WriteStreamOperationStartResult
+      .newBuilder()
+      .setQueryId(query.id.toString)
+      .setRunId(query.runId.toString)
+      .setName(Option(query.name).getOrElse(""))
+      .build()
+
+    responseObserver.onNext(
+      ExecutePlanResponse
+        .newBuilder()
+        .setSessionId(sessionId)
+        .setWriteStreamOperationStartResult(result)
+        .build())
+  }
+
+  def handleStreamingQueryCommand(
+      command: StreamingQueryCommand,
+      sessionId: String,
+      responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
+
+    val queryId = command.getQueryId
+
+    val respBuilder = StreamingQueryCommandResult
+      .newBuilder()
+      .setQueryId(command.getQueryId)
+
+    val query = Option(session.streams.get(queryId)) match {
+      case Some(query) if query.runId.toString == command.getRunId =>
+        query
+      case Some(query) =>
+        throw new IllegalArgumentException(
+          s"Run id mismatch for query id $queryId. Run id in the request ${command.getRunId} " +
+            s"does not match one on the server ${query.runId}. The query might have restarted.")
+      case None =>
+        throw new IllegalArgumentException(s"Streaming query $queryId is not found")
+      // TODO(SPARK-42962): Handle this better. May be cache stopped queries for a few minutes.
+    }
+
+    command.getCommandTypeCase match {
+      case StreamingQueryCommand.CommandTypeCase.STATUS =>
+        val recentProgress: Seq[String] = command.getStatus.getRecentProgressLimit match {
+          case 0 => Seq.empty
+          case limit if limit < 0 =>
+            query.recentProgress.map(_.json) // All the cached progresses.
+          case limit => query.recentProgress.takeRight(limit).map(_.json) // Most recent
+        }
+
+        val queryStatus = query.status
+
+        val statusResult = StreamingQueryCommandResult.StatusResult
+          .newBuilder()
+          .setStatusMessage(queryStatus.message)
+          .setIsDataAvailable(queryStatus.isDataAvailable)
+          .setIsTriggerActive(queryStatus.isTriggerActive)
+          .setIsActive(query.isActive)
+          .addAllRecentProgressJson(recentProgress.asJava)
+          .build()
+
+        respBuilder.setStatus(statusResult)
+
+      case StreamingQueryCommand.CommandTypeCase.STOP =>
+        query.stop()
+
+      case StreamingQueryCommand.CommandTypeCase.PROCESS_ALL_AVAILABLE =>
+        query.processAllAvailable()

Review Comment:
   Left a comment. We need to handle this better as part of session management improvements. 



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


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

Posted by "amaliujia (via GitHub)" <gi...@apache.org>.
amaliujia commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1157644345


##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,126 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+// Starts write stream operation as streaming query. Query ID and Run ID of the streaming
+// query are returned.
+message WriteStreamOperationStart {
+
+  // (Required) The output of the `input` streaming relation will be written.
+  Relation input = 1;
+
+  // The following fields directly map to API for DataStreamWriter().
+  // Consult API documentation unless explicitly documented here.
+
+  string format = 2;
+  map<string, string> options = 3;
+  repeated string partitioning_column_names = 4;
+
+  oneof trigger {
+    string processing_time_interval = 5;
+    bool available_now = 6;
+    bool once = 7;
+    string continuous_checkpoint_interval = 8;
+  }
+
+  string output_mode = 9;
+  string query_name = 10;
+
+  // The destination is optional. When set, it can be a path or a table name.
+  oneof sink_destination {
+    string path = 11;
+    string table_name = 12;
+  }
+}
+
+message WriteStreamOperationStartResult {
+
+  // (Required) Query instance. See `StreamingQueryInstanceId`.
+  StreamingQueryInstanceId query_id = 1;
+
+  // An optional query name.
+  string name = 2;
+
+  // TODO: How do we indicate errors?
+  // TODO: Consider adding status, last progress etc here.
+}
+
+// A tuple that uniquely identifies an instance of streaming query run. It consists of `id` that
+// persists across the streaming runs and `run_id` that changes between each run of the
+// streaming query that resumes from the checkpoint.
+message StreamingQueryInstanceId {
+

Review Comment:
   Thank you!



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


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

Posted by "grundprinzip (via GitHub)" <gi...@apache.org>.
grundprinzip commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1156252321


##########
connector/connect/common/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -154,6 +158,9 @@ message Read {
     //
     // This is only supported by the JDBC data source.
     repeated string predicates = 5;
+
+    // (Optional) Source table name for a streaming read. Not used in batch read.
+    string streaming_table_name = 6;

Review Comment:
   I would like to avoid merging protos that are immediately removed. If the options are sufficient, let's try to adopt them now please.



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


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

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #40586:
URL: https://github.com/apache/spark/pull/40586#discussion_r1157031272


##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -177,3 +179,118 @@ message WriteOperationV2 {
   // (Optional) A condition for overwrite saving mode
   Expression overwrite_condition = 8;
 }
+
+// Starts write stream operation as streaming query. Query ID and Run ID of the streaming
+// query are returned.
+message WriteStreamOperationStart {
+
+  // (Required) The output of the `input` streaming relation will be written.
+  Relation input = 1;
+
+  // The following fields directly map to API for DataStreamWriter().
+  // Consult API documentation unless explicitly documented here.
+
+  string format = 2;
+  map<string, string> options = 3;
+  repeated string partitioning_column_names = 4;
+
+  oneof trigger {
+    string processing_time_interval = 5;
+    bool available_now = 6;
+    bool one_time = 7;

Review Comment:
   yeah let's match



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