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


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala:
##########
@@ -0,0 +1,299 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.streaming
+
+import java.util.Locale
+import java.util.concurrent.TimeoutException
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.Evolving
+import org.apache.spark.api.java.function.VoidFunction2
+import org.apache.spark.connect.proto.Command
+import org.apache.spark.connect.proto.WriteStreamOperationStart
+import org.apache.spark.sql.Dataset
+import org.apache.spark.sql.execution.streaming.AvailableNowTrigger
+import org.apache.spark.sql.execution.streaming.ContinuousTrigger
+import org.apache.spark.sql.execution.streaming.OneTimeTrigger
+import org.apache.spark.sql.execution.streaming.ProcessingTimeTrigger
+
+/**
+ * Interface used to write a streaming `Dataset` to external storage systems 
(e.g. file systems,
+ * key-value stores, etc). Use `Dataset.writeStream` to access this.
+ *
+ * @since 3.5.0
+ */
+@Evolving
+final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
+
+  /**
+   * Specifies how data of a streaming DataFrame/Dataset is written to a 
streaming sink.
+   * <ul>
+   * <li> `OutputMode.Append()`: only the new rows in the streaming 
DataFrame/Dataset will be
+   * written to the sink.</li>
+   * <li> `OutputMode.Complete()`: all the rows in the streaming 
DataFrame/Dataset will be written
+   * to the sink every time there are some updates.</li>
+   * <li> `OutputMode.Update()`: only the rows that were updated in the 
streaming
+   * DataFrame/Dataset will be written to the sink every time there are some 
updates.
+   * If the query doesn't contain aggregations, it will be equivalent to
+   * `OutputMode.Append()` mode.</li>
+   * </ul>
+   *
+   * @since 3.5.0
+   */
+  def outputMode(outputMode: OutputMode): DataStreamWriter[T] = {
+    sinkBuilder.setOutputMode(outputMode.toString.toLowerCase(Locale.ROOT))
+    this
+  }
+
+  /**
+   * Specifies how data of a streaming DataFrame/Dataset is written to a 
streaming sink.
+   * <ul>
+   * <li> `append`: only the new rows in the streaming DataFrame/Dataset will 
be written to
+   * the sink.</li>
+   * <li> `complete`: all the rows in the streaming DataFrame/Dataset will be 
written to the sink
+   * every time there are some updates.</li>
+   * <li> `update`: only the rows that were updated in the streaming 
DataFrame/Dataset will
+   * be written to the sink every time there are some updates. If the query 
doesn't
+   * contain aggregations, it will be equivalent to `append` mode.</li>
+   * </ul>
+   *
+   * @since 3.5.0
+   */
+  def outputMode(outputMode: String): DataStreamWriter[T] = {
+    sinkBuilder.setOutputMode(outputMode)
+    this
+  }
+
+  /**
+   * Set the trigger for the stream query. The default value is 
`ProcessingTime(0)` and it will run
+   * the query as fast as possible.
+   *
+   * Scala Example:
+   * {{{
+   *   df.writeStream.trigger(ProcessingTime("10 seconds"))
+   *
+   *   import scala.concurrent.duration._
+   *   df.writeStream.trigger(ProcessingTime(10.seconds))
+   * }}}
+   *
+   * Java Example:
+   * {{{
+   *   df.writeStream().trigger(ProcessingTime.create("10 seconds"))
+   *
+   *   import java.util.concurrent.TimeUnit
+   *   df.writeStream().trigger(ProcessingTime.create(10, TimeUnit.SECONDS))
+   * }}}
+   *
+   * @since 3.5.0
+   */
+  def trigger(trigger: Trigger): DataStreamWriter[T] = {
+    trigger match {
+      case ProcessingTimeTrigger(intervalMs) =>
+        sinkBuilder.setProcessingTimeInterval(s"$intervalMs milliseconds")
+      case AvailableNowTrigger =>
+        sinkBuilder.setAvailableNow(true)
+      case OneTimeTrigger =>
+        sinkBuilder.setOnce(true)
+      case ContinuousTrigger(intervalMs) =>
+        sinkBuilder.setContinuousCheckpointInterval(s"$intervalMs 
milliseconds")
+    }
+    this
+  }
+
+  /**
+   * Specifies the name of the [[StreamingQuery]] that can be started with 
`start()`.
+   * This name must be unique among all the currently active queries in the 
associated SQLContext.
+   *
+   * @since 3.5.0
+   */
+  def queryName(queryName: String): DataStreamWriter[T] = {
+    sinkBuilder.setQueryName(queryName)
+    this
+  }
+
+  /**
+   * Specifies the underlying output data source.
+   *
+   * @since 3.5.0
+   */
+  def format(source: String): DataStreamWriter[T] = {
+    sinkBuilder.setFormat(source)
+    this
+  }
+
+  /**
+   * Partitions the output by the given columns on the file system. If 
specified, the output is
+   * laid out on the file system similar to Hive's partitioning scheme. As an 
example, when we
+   * partition a dataset by year and then month, the directory layout would 
look like:
+   *
+   * <ul>
+   * <li> year=2016/month=01/</li>
+   * <li> year=2016/month=02/</li>
+   * </ul>
+   *
+   * Partitioning is one of the most widely used techniques to optimize 
physical data layout.
+   * It provides a coarse-grained index for skipping unnecessary data reads 
when queries have
+   * predicates on the partitioned columns. In order for partitioning to work 
well, the number
+   * of distinct values in each column should typically be less than tens of 
thousands.
+   *
+   * @since 3.5.0
+   */
+  @scala.annotation.varargs
+  def partitionBy(colNames: String*): DataStreamWriter[T] = {
+    sinkBuilder.clearPartitioningColumnNames()
+    sinkBuilder.addAllPartitioningColumnNames(colNames.asJava)
+    this
+  }
+
+  /**
+   * Adds an output option for the underlying data source.
+   *
+   * @since 3.5.0
+   */
+  def option(key: String, value: String): DataStreamWriter[T] = {
+    sinkBuilder.putOptions(key, value)
+    this
+  }
+
+  /**
+   * Adds an output option for the underlying data source.
+   *
+   * @since 3.5.0
+   */
+  def option(key: String, value: Boolean): DataStreamWriter[T] = option(key, 
value.toString)
+
+  /**
+   * Adds an output option for the underlying data source.
+   *
+   * @since 3.5.0
+   */
+  def option(key: String, value: Long): DataStreamWriter[T] = option(key, 
value.toString)
+
+  /**
+   * Adds an output option for the underlying data source.
+   *
+   * @since 3.5.0
+   */
+  def option(key: String, value: Double): DataStreamWriter[T] = option(key, 
value.toString)
+
+  /**
+   * (Scala-specific) Adds output options for the underlying data source.
+   *
+   * @since 3.5.0
+   */
+  def options(options: scala.collection.Map[String, String]): 
DataStreamWriter[T] = {
+    this.options(options.asJava)
+    this
+  }
+
+  /**
+   * Adds output options for the underlying data source.
+   *
+   * @since 3.5.0
+   */
+  def options(options: java.util.Map[String, String]): DataStreamWriter[T] = {
+    this.options(options.asScala)
+    this
+  }
+
+  /**
+   * Starts the execution of the streaming query, which will continually 
output results to the given
+   * path as new data arrives. The returned [[StreamingQuery]] object can be 
used to interact with
+   * the stream.
+   *
+   * @since 3.5.0
+   */
+  def start(path: String): StreamingQuery = {
+    sinkBuilder.setPath(path)
+    start()
+  }
+
+  /**
+   * Starts the execution of the streaming query, which will continually 
output results to the given
+   * path as new data arrives. The returned [[StreamingQuery]] object can be 
used to interact with
+   * the stream. Throws a `TimeoutException` if the following conditions are 
met:
+   *  - Another run of the same streaming query, that is a streaming query
+   *    sharing the same checkpoint location, is already active on the same
+   *    Spark Driver
+   *  - The SQL configuration `spark.sql.streaming.stopActiveRunOnRestart`
+   *    is enabled
+   *  - The active run cannot be stopped within the timeout controlled by
+   *    the SQL configuration `spark.sql.streaming.stopTimeout`
+   *
+   * @since 3.5.0
+   */
+  @throws[TimeoutException]
+  def start(): StreamingQuery = {
+    val startCmd = Command
+      .newBuilder()
+      .setWriteStreamOperationStart(sinkBuilder.build())
+      .build()
+
+    val resp = ds.sparkSession.execute(startCmd).head
+    RemoteStreamingQuery.fromStartCommandResponse(ds.sparkSession, resp)
+  }
+
+  /**
+   * TODO(SPARK-43133) Add foreach() API in Scala.
+   */
+  // def foreach(writer: ForeachWriter[T]): DataStreamWriter[T] = {
+
+
+  /**
+   * :: Experimental ::
+   *
+   * (Scala-specific) Sets the output of the streaming query to be processed 
using the provided
+   * function. This is supported only in the micro-batch execution modes (that 
is, when the
+   * trigger is not continuous). In every micro-batch, the provided function 
will be called in
+   * every micro-batch with (i) the output rows as a Dataset and (ii) the 
batch identifier.
+   * The batchId can be used to deduplicate and transactionally write the 
output
+   * (that is, the provided Dataset) to external systems. The output Dataset 
is guaranteed
+   * to be exactly the same for the same batchId (assuming all operations are 
deterministic
+   * in the query).
+   *
+   * @since 3.5.0
+   */
+  @Evolving
+  def foreachBatch(function: (Dataset[T], Long) => Unit): DataStreamWriter[T] 
= {

Review Comment:
   Made a few updates CheckConnectJvmClientCompatibility. Will update more. 



##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala:
##########
@@ -0,0 +1,306 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.streaming
+
+import java.util.UUID
+import java.util.concurrent.TimeoutException
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.annotation.Evolving
+import org.apache.spark.connect.proto.Command
+import org.apache.spark.connect.proto.ExecutePlanResponse
+import org.apache.spark.connect.proto.StreamingQueryCommand
+import org.apache.spark.connect.proto.StreamingQueryCommandResult
+import org.apache.spark.sql.SparkSession
+
+/**
+ * A handle to a query that is executing continuously in the background as new 
data arrives.
+ * All these methods are thread-safe.
+ * @since 3.5.0
+ */
+@Evolving
+trait StreamingQuery {
+  // This is a copy of StreamingQuery in 
sql/core/.../streaming/StreamingQuery.scala
+
+  /**
+   * Returns the user-specified name of the query, or null if not specified.
+   * This name can be specified in the 
`org.apache.spark.sql.streaming.DataStreamWriter`
+   * as `dataframe.writeStream.queryName("query").start()`.
+   * This name, if set, must be unique across all active queries.
+   *
+   * @since 3.5.0
+   */
+  def name: String
+
+  /**
+   * Returns 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. Also 
see [[runId]].
+   *
+   * @since 3.5.0
+   */
+  def id: UUID
+
+  /**
+   * Returns the unique id of this run of the query. That is, every 
start/restart of a query will
+   * generate a unique runId. Therefore, every time a query is restarted from
+   * checkpoint, it will have the same [[id]] but different [[runId]]s.
+   */
+  def runId: UUID
+
+  /**
+   * Returns the `SparkSession` associated with `this`.
+   *
+   * @since 3.5.0
+   */
+  def sparkSession: SparkSession
+
+  /**
+   * Returns `true` if this query is actively running.
+   *
+   * @since 3.5.0
+   */
+  def isActive: Boolean
+
+  /**
+   * Returns the [[StreamingQueryException]] if the query was terminated by an 
exception.
+   * @since 3.5.0
+   */
+  def exception: Option[StreamingQueryException]
+
+  /**
+   * Returns the current status of the query.
+   *
+   * @since 3.5.0
+   */
+  def status: StreamingQueryStatus
+
+  /**
+   * Returns an array of the most recent [[StreamingQueryProgress]] updates 
for this query.
+   * The number of progress updates retained for each stream is configured by 
Spark session
+   * configuration `spark.sql.streaming.numRecentProgressUpdates`.
+   *
+   * @since 3.5.0
+   */
+  def recentProgress: Array[StreamingQueryProgress]
+
+  /**
+   * Returns the most recent [[StreamingQueryProgress]] update of this 
streaming query.
+   *
+   * @since 3.5.0
+   */
+  def lastProgress: StreamingQueryProgress
+
+  /**
+   * Waits for the termination of `this` query, either by `query.stop()` or by 
an exception.
+   * If the query has terminated with an exception, then the exception will be 
thrown.
+   *
+   * If the query has terminated, then all subsequent calls to this method 
will either return
+   * immediately (if the query was terminated by `stop()`), or throw the 
exception
+   * immediately (if the query has terminated with exception).
+   *
+   * @throws StreamingQueryException if the query has terminated with an 
exception.
+   *
+   * @since 3.5.0
+   */
+  @throws[StreamingQueryException]
+  def awaitTermination(): Unit
+
+  /**
+   * Waits for the termination of `this` query, either by `query.stop()` or by 
an exception.
+   * If the query has terminated with an exception, then the exception will be 
thrown.
+   * Otherwise, it returns whether the query has terminated or not within the 
`timeoutMs`
+   * milliseconds.
+   *
+   * If the query has terminated, then all subsequent calls to this method 
will either return
+   * `true` immediately (if the query was terminated by `stop()`), or throw 
the exception
+   * immediately (if the query has terminated with exception).
+   *
+   * @throws StreamingQueryException if the query has terminated with an 
exception
+   *
+   * @since 3.5.0
+   */
+  @throws[StreamingQueryException]
+  def awaitTermination(timeoutMs: Long): Boolean
+
+  /**
+   * Blocks until all available data in the source has been processed and 
committed to the sink.
+   * This method is intended for testing. Note that in the case of continually 
arriving data, this
+   * method may block forever. Additionally, this method is only guaranteed to 
block until data that
+   * has been synchronously appended data to a 
`org.apache.spark.sql.execution.streaming.Source`
+   * prior to invocation. (i.e. `getOffset` must immediately reflect the 
addition).
+   * @since 3.5.0
+   */
+  def processAllAvailable(): Unit
+
+  /**
+   * Stops the execution of this query if it is running. This waits until the 
termination of the
+   * query execution threads or until a timeout is hit.
+   *
+   * By default stop will block indefinitely. You can configure a timeout by 
the configuration
+   * `spark.sql.streaming.stopTimeout`. A timeout of 0 (or negative) 
milliseconds will block
+   * indefinitely. If a `TimeoutException` is thrown, users can retry stopping 
the stream. If the
+   * issue persists, it is advisable to kill the Spark application.
+   *
+   * @since 3.5.0
+   */
+  @throws[TimeoutException]
+  def stop(): Unit
+
+  /**
+   * Prints the physical plan to the console for debugging purposes.
+   * @since 3.5.0
+   */
+  def explain(): Unit
+
+  /**
+   * Prints the physical plan to the console for debugging purposes.
+   *
+   * @param extended whether to do extended explain or not
+   * @since 3.5.0
+   */
+  def explain(extended: Boolean): Unit
+}
+
+class RemoteStreamingQuery(
+  override val id: UUID,
+  override val runId: UUID,
+  override val name: String,
+  override val sparkSession: SparkSession) extends StreamingQuery {
+
+  override def isActive: Boolean = {
+    executeQueryCmd(_.setStatus(true)).getStatus.getIsActive
+  }
+
+  override def exception: Option[StreamingQueryException] = {
+    throw new UnsupportedOperationException("exception is not yet implemented")
+  }
+
+  override def status: StreamingQueryStatus = {
+    val statusResp = executeQueryCmd(_.setStatus(true)).getStatus
+    new StreamingQueryStatus(
+      message = statusResp.getStatusMessage,
+      isDataAvailable = statusResp.getIsDataAvailable,
+      isTriggerActive = statusResp.getIsTriggerActive
+    )
+  }
+
+  override def recentProgress: Array[StreamingQueryProgress] = {
+    executeQueryCmd(_.setRecentProgress(true))
+      .getRecentProgress
+      .getRecentProgressJsonList
+      .asScala
+      .map(json => new StreamingQueryProgress(json))
+      .toArray
+  }
+
+  override def lastProgress: StreamingQueryProgress = {
+    executeQueryCmd(_.setLastProgress(true))
+      .getRecentProgress
+      .getRecentProgressJsonList
+      .asScala
+      .headOption
+      .map(json => new StreamingQueryProgress(json))
+      .orNull
+  }
+
+  override def awaitTermination(): Unit = {
+    throw new UnsupportedOperationException("awaitTermination() is not yet 
implemented")
+  }
+
+  override def awaitTermination(timeoutMs: Long): Boolean = {
+    throw new UnsupportedOperationException("awaitTermination() is not yet 
implemented")
+  }
+
+  override def processAllAvailable(): Unit = {
+    executeQueryCmd(_.setProcessAllAvailable(true))
+  }
+
+  override def stop(): Unit = {
+    executeQueryCmd(_.setStop(true))
+  }
+
+  override def explain(): Unit = {
+    explain(extended = false)
+  }
+
+  override def explain(extended: Boolean): Unit = {
+    val explainCmd = StreamingQueryCommand
+      .ExplainCommand.newBuilder()
+      .setExtended(extended)
+      .build()
+
+    val explain = executeQueryCmd(_.setExplain(explainCmd))
+      .getExplain
+      .getResult
+
+    // scalastyle:off println
+    println(explain)
+    // scalastyle:on println
+  }
+
+  private def executeQueryCmd(
+    setCmdFn: StreamingQueryCommand.Builder => Unit // Sets the command field, 
like stop().
+  ): StreamingQueryCommandResult = {
+
+    val cmdBuilder = Command.newBuilder()
+    val queryCmdBuilder = cmdBuilder.getStreamingQueryCommandBuilder
+
+    // Set queryId.
+    queryCmdBuilder
+      .getQueryIdBuilder
+      .setId(id.toString)
+      .setRunId(runId.toString)

Review Comment:
   True, but this is not any noticeable overhead. `toString` is done only when 
we are sending an RPC. 



##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/progress.scala:
##########
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.streaming
+
+class StreamingQueryProgress private[sql](
+  val json: String
+) {
+  // TODO: (Implement full object by parsing from json).

Review Comment:
   Forgot to add SPARK-43128. Updated.
   Will be done in a followup. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to