Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -96,6 +96,14 @@ object StaticSQLConf {
.toSequence
.createOptional

val STREAMING_QUERY_LISTENERS = buildStaticConf("spark.sql.streaming.streamingQueryListeners")
.doc("List of class names implementing StreamingQueryListener that will be automatically " +
"added to newly created sessions. The classes should have either a no-arg constructor, " +
"or a constructor that expects a SparkConf argument.")
.stringConf
.toSequence
.createOptional

val UI_RETAINED_EXECUTIONS =
buildStaticConf("spark.sql.ui.retainedExecutions")
.doc("Number of executions to retain in the Spark UI.")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import scala.collection.mutable

import org.apache.hadoop.fs.Path

import org.apache.spark.SparkException
import org.apache.spark.annotation.InterfaceStability
import org.apache.spark.internal.Logging
import org.apache.spark.sql.{AnalysisException, DataFrame, SparkSession}
Expand All @@ -32,6 +33,7 @@ import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.execution.streaming.continuous.{ContinuousExecution, ContinuousTrigger}
import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.StaticSQLConf.STREAMING_QUERY_LISTENERS
import org.apache.spark.sql.sources.v2.StreamWriteSupport
import org.apache.spark.util.{Clock, SystemClock, Utils}

Expand All @@ -55,6 +57,19 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo
@GuardedBy("awaitTerminationLock")
private var lastTerminatedQuery: StreamingQuery = null

try {
sparkSession.sparkContext.conf.get(STREAMING_QUERY_LISTENERS).foreach { classNames =>
Utils.loadExtensions(classOf[StreamingQueryListener], classNames,
sparkSession.sparkContext.conf).foreach(listener => {
addListener(listener)
logInfo(s"Registered listener ${listener.getClass.getName}")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would do this at debug level ..

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Either debug or info is fine for me, since it would add just couple of log lines only once.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since its only once and provides information to user I guess info is fine. Similar pattern here https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkContext.scala#L2359

})
}
} catch {
case e: Exception =>
throw new SparkException("Exception when registering StreamingQueryListener", e)
}

/**
* Returns a list of active queries associated with this SQLContext
*
Expand Down
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
}
}