Skip to content

Commit

Permalink
[SPARK-43129] Scala core API for streaming Spark Connect
Browse files Browse the repository at this point in the history
### What changes were proposed in this pull request?
Implements core streaming API in Scala for running streaming queries over Spark Connect.
This is functionally equivalent to Python side PR apache#40586

There are no server side changes here since it was done earlier in Python PR.

We can run most streaming queries.
Notably, queries using `foreachBatch()` are not yet supported.

### Why are the changes needed?
This adds structured streaming support in Scala for Spark connect.

### Does this PR introduce _any_ user-facing change?

Adds more streaming API to Scala Spark Connect client.

### How was this patch tested?

  - Unit test
  - Manual testing

Closes apache#40783 from rangadi/scala-m1.

Authored-by: Raghu Angadi <[email protected]>
Signed-off-by: Hyukjin Kwon <[email protected]>
  • Loading branch information
Raghu Angadi authored and HyukjinKwon committed Apr 19, 2023
1 parent b9400c7 commit 3814d15
Show file tree
Hide file tree
Showing 12 changed files with 1,345 additions and 6 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,180 @@
/*
* 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.concurrent.TimeUnit;

import scala.concurrent.duration.Duration;

import org.apache.spark.annotation.Evolving;
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;

/**
* Policy used to indicate how often results should be produced by a [[StreamingQuery]].
*
* @since 3.5.0
*/
@Evolving
public class Trigger {
// This is a copy of the same class in sql/core/.../streaming/Trigger.java

/**
* A trigger policy that runs a query periodically based on an interval in processing time.
* If `interval` is 0, the query will run as fast as possible.
*
* @since 3.5.0
*/
public static Trigger ProcessingTime(long intervalMs) {
return ProcessingTimeTrigger.create(intervalMs, TimeUnit.MILLISECONDS);
}

/**
* (Java-friendly)
* A trigger policy that runs a query periodically based on an interval in processing time.
* If `interval` is 0, the query will run as fast as possible.
*
* {{{
* import java.util.concurrent.TimeUnit
* df.writeStream().trigger(Trigger.ProcessingTime(10, TimeUnit.SECONDS))
* }}}
*
* @since 3.5.0
*/
public static Trigger ProcessingTime(long interval, TimeUnit timeUnit) {
return ProcessingTimeTrigger.create(interval, timeUnit);
}

/**
* (Scala-friendly)
* A trigger policy that runs a query periodically based on an interval in processing time.
* If `duration` is 0, the query will run as fast as possible.
*
* {{{
* import scala.concurrent.duration._
* df.writeStream.trigger(Trigger.ProcessingTime(10.seconds))
* }}}
* @since 3.5.0
*/
public static Trigger ProcessingTime(Duration interval) {
return ProcessingTimeTrigger.apply(interval);
}

/**
* A trigger policy that runs a query periodically based on an interval in processing time.
* If `interval` is effectively 0, the query will run as fast as possible.
*
* {{{
* df.writeStream.trigger(Trigger.ProcessingTime("10 seconds"))
* }}}
* @since 3.5.0
*/
public static Trigger ProcessingTime(String interval) {
return ProcessingTimeTrigger.apply(interval);
}

/**
* A trigger that processes all available data in a single batch then terminates the query.
*
* @since 3.5.0
* @deprecated This is deprecated as of Spark 3.4.0. Use {@link #AvailableNow()} to leverage
* better guarantee of processing, fine-grained scale of batches, and better gradual
* processing of watermark advancement including no-data batch.
* See the NOTES in {@link #AvailableNow()} for details.
*/
@Deprecated
public static Trigger Once() {
return OneTimeTrigger$.MODULE$;
}

/**
* A trigger that processes all available data at the start of the query in one or multiple
* batches, then terminates the query.
*
* Users are encouraged to set the source options to control the size of the batch as similar as
* controlling the size of the batch in {@link #ProcessingTime(long)} trigger.
*
* NOTES:
* - This trigger provides a strong guarantee of processing: regardless of how many batches were
* left over in previous run, it ensures all available data at the time of execution gets
* processed before termination. All uncommitted batches will be processed first.
* - Watermark gets advanced per each batch, and no-data batch gets executed before termination
* if the last batch advances the watermark. This helps to maintain smaller and predictable
* state size and smaller latency on the output of stateful operators.
*
* @since 3.5.0
*/
public static Trigger AvailableNow() {
return AvailableNowTrigger$.MODULE$;
}

/**
* A trigger that continuously processes streaming data, asynchronously checkpointing at
* the specified interval.
*
* @since 3.5.0
*/
public static Trigger Continuous(long intervalMs) {
return ContinuousTrigger.apply(intervalMs);
}

/**
* A trigger that continuously processes streaming data, asynchronously checkpointing at
* the specified interval.
*
* {{{
* import java.util.concurrent.TimeUnit
* df.writeStream.trigger(Trigger.Continuous(10, TimeUnit.SECONDS))
* }}}
*
* @since 3.5.0
*/
public static Trigger Continuous(long interval, TimeUnit timeUnit) {
return ContinuousTrigger.create(interval, timeUnit);
}

/**
* (Scala-friendly)
* A trigger that continuously processes streaming data, asynchronously checkpointing at
* the specified interval.
*
* {{{
* import scala.concurrent.duration._
* df.writeStream.trigger(Trigger.Continuous(10.seconds))
* }}}
* @since 3.5.0
*/
public static Trigger Continuous(Duration interval) {
return ContinuousTrigger.apply(interval);
}

/**
* A trigger that continuously processes streaming data, asynchronously checkpointing at
* the specified interval.
*
* {{{
* df.writeStream.trigger(Trigger.Continuous("10 seconds"))
* }}}
* @since 3.5.0
*/
public static Trigger Continuous(String interval) {
return ContinuousTrigger.apply(interval);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import org.apache.spark.sql.connect.client.{SparkResult, UdfUtils}
import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, StorageLevelProtoConverter}
import org.apache.spark.sql.expressions.ScalarUserDefinedFunction
import org.apache.spark.sql.functions.{struct, to_json}
import org.apache.spark.sql.streaming.DataStreamWriter
import org.apache.spark.sql.types.{Metadata, StructType}
import org.apache.spark.storage.StorageLevel
import org.apache.spark.util.Utils
Expand Down Expand Up @@ -2935,6 +2936,16 @@ class Dataset[T] private[sql] (
new DataFrameWriterV2[T](table, this)
}

/**
* Interface for saving the content of the streaming Dataset out into external storage.
*
* @group basic
* @since 3.5.0
*/
def writeStream: DataStreamWriter[T] = {
new DataStreamWriter[T](this)
}

/**
* Persist this Dataset with the default storage level (`MEMORY_AND_DISK`).
*
Expand Down Expand Up @@ -3017,8 +3028,37 @@ class Dataset[T] private[sql] (
.getStorageLevel)
}

/**
* Defines an event time watermark for this [[Dataset]]. A watermark tracks a point in time
* before which we assume no more late data is going to arrive.
*
* Spark will use this watermark for several purposes: <ul> <li>To know when a given time window
* aggregation can be finalized and thus can be emitted when using output modes that do not
* allow updates.</li> <li>To minimize the amount of state that we need to keep for on-going
* aggregations, `mapGroupsWithState` and `dropDuplicates` operators.</li> </ul> The current
* watermark is computed by looking at the `MAX(eventTime)` seen across all of the partitions in
* the query minus a user specified `delayThreshold`. Due to the cost of coordinating this value
* across partitions, the actual watermark used is only guaranteed to be at least
* `delayThreshold` behind the actual event time. In some cases we may still process records
* that arrive more than `delayThreshold` late.
*
* @param eventTime
* the name of the column that contains the event time of the row.
* @param delayThreshold
* the minimum delay to wait to data to arrive late, relative to the latest record that has
* been processed in the form of an interval (e.g. "1 minute" or "5 hours"). NOTE: This should
* not be negative.
*
* @group streaming
* @since 3.5.0
*/
def withWatermark(eventTime: String, delayThreshold: String): Dataset[T] = {
throw new UnsupportedOperationException("withWatermark is not implemented.")
sparkSession.newDataset(encoder) { builder =>
builder.getWithWatermarkBuilder
.setInput(plan.getRoot)
.setEventTime(eventTime)
.setDelayThreshold(delayThreshold)
}
}

def observe(name: String, expr: Column, exprs: Column*): Dataset[T] = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ import org.apache.arrow.memory.RootAllocator

import org.apache.spark.annotation.{DeveloperApi, Experimental}
import org.apache.spark.connect.proto
import org.apache.spark.connect.proto.ExecutePlanResponse
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalog.Catalog
import org.apache.spark.sql.catalyst.{JavaTypeInference, ScalaReflection}
Expand All @@ -37,6 +38,7 @@ import org.apache.spark.sql.connect.client.{ClassFinder, SparkConnectClient, Spa
import org.apache.spark.sql.connect.client.util.{Cleaner, ConvertToArrow}
import org.apache.spark.sql.connect.common.LiteralValueProtoConverter.toLiteralProto
import org.apache.spark.sql.internal.CatalogImpl
import org.apache.spark.sql.streaming.DataStreamReader
import org.apache.spark.sql.types.StructType

/**
Expand Down Expand Up @@ -287,6 +289,17 @@ class SparkSession private[sql] (
*/
def read: DataFrameReader = new DataFrameReader(this)

/**
* Returns a `DataStreamReader` that can be used to read streaming data in as a `DataFrame`.
* {{{
* sparkSession.readStream.parquet("/path/to/directory/of/parquet/files")
* sparkSession.readStream.schema(schema).json("/path/to/directory/of/json/files")
* }}}
*
* @since 3.5.0
*/
def readStream: DataStreamReader = new DataStreamReader(this)

/**
* Interface through which the user may create, drop, alter or query underlying databases,
* tables, functions etc.
Expand Down Expand Up @@ -453,9 +466,9 @@ class SparkSession private[sql] (
client.execute(plan).asScala.foreach(_ => ())
}

private[sql] def execute(command: proto.Command): Unit = {
private[sql] def execute(command: proto.Command): Seq[ExecutePlanResponse] = {
val plan = proto.Plan.newBuilder().setCommand(command).build()
client.execute(plan).asScala.foreach(_ => ())
client.execute(plan).asScala.toSeq
}

@DeveloperApi
Expand Down
Loading

0 comments on commit 3814d15

Please sign in to comment.