forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[SPARK-48048][CONNECT][SS] Added client side listener support for Scala
### What changes were proposed in this pull request? Added client side Streaming Listener support for Scala ### Why are the changes needed? Support Streaming Listener on client side for Spark Connect which has better user experience (no breaking change compared to legacy mode) compared to previous server side listener. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added unit test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes apache#46287 from bogao007/client-listener. Authored-by: bogao007 <bo.gao@databricks.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
- Loading branch information
1 parent
7b433c5
commit 62ce0c3
Showing
7 changed files
with
396 additions
and
78 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
152 changes: 152 additions & 0 deletions
152
.../client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListenerBus.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,152 @@ | ||
/* | ||
* 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.CopyOnWriteArrayList | ||
|
||
import scala.jdk.CollectionConverters._ | ||
|
||
import org.apache.spark.connect.proto.{Command, ExecutePlanResponse, Plan, StreamingQueryEventType} | ||
import org.apache.spark.internal.Logging | ||
import org.apache.spark.sql.SparkSession | ||
import org.apache.spark.sql.connect.client.CloseableIterator | ||
import org.apache.spark.sql.streaming.StreamingQueryListener.{Event, QueryIdleEvent, QueryProgressEvent, QueryStartedEvent, QueryTerminatedEvent} | ||
|
||
class StreamingQueryListenerBus(sparkSession: SparkSession) extends Logging { | ||
private val listeners = new CopyOnWriteArrayList[StreamingQueryListener]() | ||
private var executionThread: Option[Thread] = Option.empty | ||
|
||
val lock = new Object() | ||
|
||
def close(): Unit = { | ||
listeners.forEach(remove(_)) | ||
} | ||
|
||
def append(listener: StreamingQueryListener): Unit = lock.synchronized { | ||
listeners.add(listener) | ||
|
||
if (listeners.size() == 1) { | ||
var iter: Option[CloseableIterator[ExecutePlanResponse]] = Option.empty | ||
try { | ||
iter = Some(registerServerSideListener()) | ||
} catch { | ||
case e: Exception => | ||
logWarning("Failed to add the listener, please add it again.", e) | ||
listeners.remove(listener) | ||
return | ||
} | ||
executionThread = Some(new Thread(new Runnable { | ||
def run(): Unit = { | ||
queryEventHandler(iter.get) | ||
} | ||
})) | ||
// Start the thread | ||
executionThread.get.start() | ||
} | ||
} | ||
|
||
def remove(listener: StreamingQueryListener): Unit = lock.synchronized { | ||
if (listeners.size() == 1) { | ||
val cmdBuilder = Command.newBuilder() | ||
cmdBuilder.getStreamingQueryListenerBusCommandBuilder | ||
.setRemoveListenerBusListener(true) | ||
try { | ||
sparkSession.execute(cmdBuilder.build()) | ||
} catch { | ||
case e: Exception => | ||
logWarning("Failed to remove the listener, please remove it again.", e) | ||
return | ||
} | ||
if (executionThread.isDefined) { | ||
executionThread.get.interrupt() | ||
executionThread = Option.empty | ||
} | ||
} | ||
listeners.remove(listener) | ||
} | ||
|
||
def list(): Array[StreamingQueryListener] = lock.synchronized { | ||
listeners.asScala.toArray | ||
} | ||
|
||
def registerServerSideListener(): CloseableIterator[ExecutePlanResponse] = { | ||
val cmdBuilder = Command.newBuilder() | ||
cmdBuilder.getStreamingQueryListenerBusCommandBuilder | ||
.setAddListenerBusListener(true) | ||
|
||
val plan = Plan.newBuilder().setCommand(cmdBuilder.build()).build() | ||
val iterator = sparkSession.client.execute(plan) | ||
while (iterator.hasNext) { | ||
val response = iterator.next() | ||
if (response.getStreamingQueryListenerEventsResult.hasListenerBusListenerAdded && | ||
response.getStreamingQueryListenerEventsResult.getListenerBusListenerAdded) { | ||
return iterator | ||
} | ||
} | ||
iterator | ||
} | ||
|
||
def queryEventHandler(iter: CloseableIterator[ExecutePlanResponse]): Unit = { | ||
try { | ||
while (iter.hasNext) { | ||
val response = iter.next() | ||
val listenerEvents = response.getStreamingQueryListenerEventsResult.getEventsList | ||
listenerEvents.forEach(event => { | ||
event.getEventType match { | ||
case StreamingQueryEventType.QUERY_PROGRESS_EVENT => | ||
postToAll(QueryProgressEvent.fromJson(event.getEventJson)) | ||
case StreamingQueryEventType.QUERY_IDLE_EVENT => | ||
postToAll(QueryIdleEvent.fromJson(event.getEventJson)) | ||
case StreamingQueryEventType.QUERY_TERMINATED_EVENT => | ||
postToAll(QueryTerminatedEvent.fromJson(event.getEventJson)) | ||
case _ => | ||
logWarning(s"Unknown StreamingQueryListener event: $event") | ||
} | ||
}) | ||
} | ||
} catch { | ||
case e: Exception => | ||
logWarning("StreamingQueryListenerBus Handler thread received exception, all client" + | ||
" side listeners are removed and handler thread is terminated.", e) | ||
lock.synchronized { | ||
executionThread = Option.empty | ||
listeners.forEach(remove(_)) | ||
} | ||
} | ||
} | ||
|
||
def postToAll(event: Event): Unit = lock.synchronized { | ||
listeners.forEach(listener => | ||
try { | ||
event match { | ||
case t: QueryStartedEvent => | ||
listener.onQueryStarted(t) | ||
case t: QueryProgressEvent => | ||
listener.onQueryProgress(t) | ||
case t: QueryIdleEvent => | ||
listener.onQueryIdle(t) | ||
case t: QueryTerminatedEvent => | ||
listener.onQueryTerminated(t) | ||
case _ => logWarning(s"Unknown StreamingQueryListener event: $event") | ||
} | ||
} catch { | ||
case e: Exception => | ||
logWarning(s"Listener $listener threw an exception", e) | ||
}) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.