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-24479][SS] Added config for registering streamingQueryListeners
## What changes were proposed in this pull request? Currently a "StreamingQueryListener" can only be registered programatically. We could have a new config "spark.sql.streamingQueryListeners" similar to "spark.sql.queryExecutionListeners" and "spark.extraListeners" for users to register custom streaming listeners. ## How was this patch tested? New unit test and running example programs. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Arun Mahadevan <[email protected]> Closes apache#21504 from arunmahadevan/SPARK-24480.
- Loading branch information
1 parent
4c388bc
commit 7703b46
Showing
3 changed files
with
89 additions
and
0 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
66 changes: 66 additions & 0 deletions
66
...core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.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,66 @@ | ||
/* | ||
* 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 scala.language.reflectiveCalls | ||
|
||
import org.scalatest.BeforeAndAfter | ||
|
||
import org.apache.spark.SparkConf | ||
import org.apache.spark.sql.execution.streaming._ | ||
import org.apache.spark.sql.streaming.StreamingQueryListener._ | ||
|
||
|
||
class StreamingQueryListenersConfSuite extends StreamTest with BeforeAndAfter { | ||
|
||
import testImplicits._ | ||
|
||
|
||
override protected def sparkConf: SparkConf = | ||
super.sparkConf.set("spark.sql.streaming.streamingQueryListeners", | ||
"org.apache.spark.sql.streaming.TestListener") | ||
|
||
test("test if the configured query lister is loaded") { | ||
testStream(MemoryStream[Int].toDS)( | ||
StartStream(), | ||
StopStream | ||
) | ||
|
||
assert(TestListener.queryStartedEvent != null) | ||
assert(TestListener.queryTerminatedEvent != null) | ||
} | ||
|
||
} | ||
|
||
object TestListener { | ||
@volatile var queryStartedEvent: QueryStartedEvent = null | ||
@volatile var queryTerminatedEvent: QueryTerminatedEvent = null | ||
} | ||
|
||
class TestListener(sparkConf: SparkConf) extends StreamingQueryListener { | ||
|
||
override def onQueryStarted(event: QueryStartedEvent): Unit = { | ||
TestListener.queryStartedEvent = event | ||
} | ||
|
||
override def onQueryProgress(event: QueryProgressEvent): Unit = {} | ||
|
||
override def onQueryTerminated(event: QueryTerminatedEvent): Unit = { | ||
TestListener.queryTerminatedEvent = event | ||
} | ||
} |