[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...
Github user asfgit closed the pull request at: https://github.com/apache/spark/pull/22674 --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/22674#discussion_r224756886 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala --- @@ -39,7 +39,22 @@ case class SparkListenerSQLExecutionStart( @DeveloperApi case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long) - extends SparkListenerEvent + extends SparkListenerEvent { + + // The name of the execution, e.g. `df.collect` will trigger a SQL execution with name "collect". + @JsonIgnore private[sql] var executionName: Option[String] = None + + // The following 3 fields are only accessed when `executionName` is defined. + + // The duration of the SQL execution, in nanoseconds. + @JsonIgnore private[sql] var duration: Long = 0L --- End diff -- There is a test to verify it: https://github.com/apache/spark/pull/22674/files#diff-6fa1d00d1cb20554dda238f2a3bc3ecbR55 I also used `@JsonIgnoreProperties` before, when these fields are case class fields. It seems we don't need `@JsonIgnoreProperties` when they are private `var`s. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...
Github user brkyvz commented on a diff in the pull request: https://github.com/apache/spark/pull/22674#discussion_r224755348 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala --- @@ -39,7 +39,22 @@ case class SparkListenerSQLExecutionStart( @DeveloperApi case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long) - extends SparkListenerEvent + extends SparkListenerEvent { + + // The name of the execution, e.g. `df.collect` will trigger a SQL execution with name "collect". + @JsonIgnore private[sql] var executionName: Option[String] = None + + // The following 3 fields are only accessed when `executionName` is defined. + + // The duration of the SQL execution, in nanoseconds. + @JsonIgnore private[sql] var duration: Long = 0L --- End diff -- did you verify that the JsonIgnore annotation actually works? For some reason, I actually needed to annotate the class as ```scala @JsonIgnoreProperties(Array("a", b", "c")) class SomeClass { @JsonProperty("a") val a: ... @JsonProperty("b") val a: ... } ``` the reason being Json4s understands that API better. I believe we use Json4s for all of these events --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/22674#discussion_r224071116 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala --- @@ -3356,21 +3356,11 @@ class Dataset[T] private[sql]( * user-registered callback functions. */ private def withAction[U](name: String, qe: QueryExecution)(action: SparkPlan => U) = { -try { - qe.executedPlan.foreach { plan => -plan.resetMetrics() - } - val start = System.nanoTime() - val result = SQLExecution.withNewExecutionId(sparkSession, qe) { -action(qe.executedPlan) - } - val end = System.nanoTime() - sparkSession.listenerManager.onSuccess(name, qe, end - start) - result -} catch { - case e: Exception => -sparkSession.listenerManager.onFailure(name, qe, e) -throw e +qe.executedPlan.foreach { plan => --- End diff -- ah i see your point here --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...
Github user brkyvz commented on a diff in the pull request: https://github.com/apache/spark/pull/22674#discussion_r224013755 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala --- @@ -3356,21 +3356,11 @@ class Dataset[T] private[sql]( * user-registered callback functions. */ private def withAction[U](name: String, qe: QueryExecution)(action: SparkPlan => U) = { -try { - qe.executedPlan.foreach { plan => -plan.resetMetrics() - } - val start = System.nanoTime() - val result = SQLExecution.withNewExecutionId(sparkSession, qe) { -action(qe.executedPlan) - } - val end = System.nanoTime() - sparkSession.listenerManager.onSuccess(name, qe, end - start) - result -} catch { - case e: Exception => -sparkSession.listenerManager.onFailure(name, qe, e) -throw e +qe.executedPlan.foreach { plan => --- End diff -- can't executedPlan throw an exception? I thought it can if the original spark plan failed? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/22674#discussion_r224012183 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala --- @@ -75,95 +76,74 @@ trait QueryExecutionListener { */ @Experimental @InterfaceStability.Evolving -class ExecutionListenerManager private extends Logging { - - private[sql] def this(conf: SparkConf) = { -this() +// The `session` is used to indicate which session carries this listener manager, and we only +// catch SQL executions which are launched by the same session. +// The `loadExtensions` flag is used to indicate whether we should load the pre-defined, +// user-specified listeners during construction. We should not do it when cloning this listener +// manager, as we will copy all listeners to the cloned listener manager. +class ExecutionListenerManager private[sql](session: SparkSession, loadExtensions: Boolean) + extends SparkListener with Logging { + + private[this] val listeners = new CopyOnWriteArrayList[QueryExecutionListener] + + if (loadExtensions) { +val conf = session.sparkContext.conf conf.get(QUERY_EXECUTION_LISTENERS).foreach { classNames => Utils.loadExtensions(classOf[QueryExecutionListener], classNames, conf).foreach(register) } } + session.sparkContext.listenerBus.addToSharedQueue(this) + /** * Registers the specified [[QueryExecutionListener]]. */ @DeveloperApi - def register(listener: QueryExecutionListener): Unit = writeLock { -listeners += listener + def register(listener: QueryExecutionListener): Unit = { +listeners.add(listener) } /** * Unregisters the specified [[QueryExecutionListener]]. */ @DeveloperApi - def unregister(listener: QueryExecutionListener): Unit = writeLock { -listeners -= listener + def unregister(listener: QueryExecutionListener): Unit = { +listeners.remove(listener) } /** * Removes all the registered [[QueryExecutionListener]]. */ @DeveloperApi - def clear(): Unit = writeLock { + def clear(): Unit = { listeners.clear() } /** * Get an identical copy of this listener manager. */ - @DeveloperApi - override def clone(): ExecutionListenerManager = writeLock { -val newListenerManager = new ExecutionListenerManager -listeners.foreach(newListenerManager.register) + private[sql] def clone(session: SparkSession): ExecutionListenerManager = { +val newListenerManager = new ExecutionListenerManager(session, loadExtensions = false) +listeners.iterator().asScala.foreach(newListenerManager.register) newListenerManager } - private[sql] def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { -readLock { - withErrorHandling { listener => -listener.onSuccess(funcName, qe, duration) + override def onOtherEvent(event: SparkListenerEvent): Unit = event match { +case e: SparkListenerSQLExecutionEnd if shouldCatchEvent(e) => + val funcName = e.executionName.get + e.executionFailure match { +case Some(ex) => + listeners.iterator().asScala.foreach(_.onFailure(funcName, e.qe, ex)) +case _ => + listeners.iterator().asScala.foreach(_.onSuccess(funcName, e.qe, e.duration)) } -} - } - private[sql] def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit = { -readLock { - withErrorHandling { listener => -listener.onFailure(funcName, qe, exception) - } -} +case _ => // Ignore } - private[this] val listeners = ListBuffer.empty[QueryExecutionListener] - - /** A lock to prevent updating the list of listeners while we are traversing through them. */ - private[this] val lock = new ReentrantReadWriteLock() - - private def withErrorHandling(f: QueryExecutionListener => Unit): Unit = { -for (listener <- listeners) { - try { -f(listener) - } catch { -case NonFatal(e) => logWarning("Error executing query execution listener", e) - } -} - } - - /** Acquires a read lock on the cache for the duration of `f`. */ - private def readLock[A](f: => A): A = { -val rl = lock.readLock() -rl.lock() -try f finally { - rl.unlock() -} - } - - /** Acquires a write lock on the cache for the duration of `f`. */ - private def writeLock[A](f: => A): A = { -val wl = lock.writeLock() -wl.lock() -try f
[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/22674#discussion_r224008348 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala --- @@ -71,14 +72,35 @@ object SQLExecution { val callSite = sc.getCallSite() withSQLConfPropagated(sparkSession) { -sc.listenerBus.post(SparkListenerSQLExecutionStart( - executionId, callSite.shortForm, callSite.longForm, queryExecution.toString, - SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan), System.currentTimeMillis())) +var ex: Option[Exception] = None +val startTime = System.currentTimeMillis() try { + sc.listenerBus.post(SparkListenerSQLExecutionStart( +executionId = executionId, +description = callSite.shortForm, +details = callSite.longForm, +physicalPlanDescription = queryExecution.toString, +// `queryExecution.executedPlan` triggers query planning. If it fails, the exception +// will be caught and reported in the `SparkListenerSQLExecutionEnd` +sparkPlanInfo = SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan), +time = startTime)) body +} catch { + case e: Exception => +ex = Some(e) +throw e } finally { - sc.listenerBus.post(SparkListenerSQLExecutionEnd( -executionId, System.currentTimeMillis())) + val endTime = System.currentTimeMillis() + val event = SparkListenerSQLExecutionEnd(executionId, endTime) + // Currently only `Dataset.withAction` and `DataFrameWriter.runCommand` specify the `name` + // parameter. The `ExecutionListenerManager` only watches SQL executions with name. We + // can specify the execution name in more places in the future, so that + // `QueryExecutionListener` can track more cases. + event.executionName = name + event.duration = endTime - startTime --- End diff -- ah good catch! --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/22674#discussion_r224007732 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala --- @@ -3356,21 +3356,11 @@ class Dataset[T] private[sql]( * user-registered callback functions. */ private def withAction[U](name: String, qe: QueryExecution)(action: SparkPlan => U) = { -try { - qe.executedPlan.foreach { plan => -plan.resetMetrics() - } - val start = System.nanoTime() - val result = SQLExecution.withNewExecutionId(sparkSession, qe) { -action(qe.executedPlan) - } - val end = System.nanoTime() - sparkSession.listenerManager.onSuccess(name, qe, end - start) - result -} catch { - case e: Exception => -sparkSession.listenerManager.onFailure(name, qe, e) -throw e +qe.executedPlan.foreach { plan => --- End diff -- I don't think `resetMetrics` can throw exception... --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...
Github user brkyvz commented on a diff in the pull request: https://github.com/apache/spark/pull/22674#discussion_r224006828 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala --- @@ -75,95 +76,74 @@ trait QueryExecutionListener { */ @Experimental @InterfaceStability.Evolving -class ExecutionListenerManager private extends Logging { - - private[sql] def this(conf: SparkConf) = { -this() +// The `session` is used to indicate which session carries this listener manager, and we only +// catch SQL executions which are launched by the same session. +// The `loadExtensions` flag is used to indicate whether we should load the pre-defined, +// user-specified listeners during construction. We should not do it when cloning this listener +// manager, as we will copy all listeners to the cloned listener manager. +class ExecutionListenerManager private[sql](session: SparkSession, loadExtensions: Boolean) + extends SparkListener with Logging { + + private[this] val listeners = new CopyOnWriteArrayList[QueryExecutionListener] + + if (loadExtensions) { +val conf = session.sparkContext.conf conf.get(QUERY_EXECUTION_LISTENERS).foreach { classNames => Utils.loadExtensions(classOf[QueryExecutionListener], classNames, conf).foreach(register) } } + session.sparkContext.listenerBus.addToSharedQueue(this) + /** * Registers the specified [[QueryExecutionListener]]. */ @DeveloperApi - def register(listener: QueryExecutionListener): Unit = writeLock { -listeners += listener + def register(listener: QueryExecutionListener): Unit = { +listeners.add(listener) } /** * Unregisters the specified [[QueryExecutionListener]]. */ @DeveloperApi - def unregister(listener: QueryExecutionListener): Unit = writeLock { -listeners -= listener + def unregister(listener: QueryExecutionListener): Unit = { +listeners.remove(listener) } /** * Removes all the registered [[QueryExecutionListener]]. */ @DeveloperApi - def clear(): Unit = writeLock { + def clear(): Unit = { listeners.clear() } /** * Get an identical copy of this listener manager. */ - @DeveloperApi - override def clone(): ExecutionListenerManager = writeLock { -val newListenerManager = new ExecutionListenerManager -listeners.foreach(newListenerManager.register) + private[sql] def clone(session: SparkSession): ExecutionListenerManager = { +val newListenerManager = new ExecutionListenerManager(session, loadExtensions = false) +listeners.iterator().asScala.foreach(newListenerManager.register) newListenerManager } - private[sql] def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { -readLock { - withErrorHandling { listener => -listener.onSuccess(funcName, qe, duration) + override def onOtherEvent(event: SparkListenerEvent): Unit = event match { +case e: SparkListenerSQLExecutionEnd if shouldCatchEvent(e) => + val funcName = e.executionName.get + e.executionFailure match { +case Some(ex) => + listeners.iterator().asScala.foreach(_.onFailure(funcName, e.qe, ex)) +case _ => + listeners.iterator().asScala.foreach(_.onSuccess(funcName, e.qe, e.duration)) } -} - } - private[sql] def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit = { -readLock { - withErrorHandling { listener => -listener.onFailure(funcName, qe, exception) - } -} +case _ => // Ignore } - private[this] val listeners = ListBuffer.empty[QueryExecutionListener] - - /** A lock to prevent updating the list of listeners while we are traversing through them. */ - private[this] val lock = new ReentrantReadWriteLock() - - private def withErrorHandling(f: QueryExecutionListener => Unit): Unit = { -for (listener <- listeners) { - try { -f(listener) - } catch { -case NonFatal(e) => logWarning("Error executing query execution listener", e) - } -} - } - - /** Acquires a read lock on the cache for the duration of `f`. */ - private def readLock[A](f: => A): A = { -val rl = lock.readLock() -rl.lock() -try f finally { - rl.unlock() -} - } - - /** Acquires a write lock on the cache for the duration of `f`. */ - private def writeLock[A](f: => A): A = { -val wl = lock.writeLock() -wl.lock() -try f fi
[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...
Github user brkyvz commented on a diff in the pull request: https://github.com/apache/spark/pull/22674#discussion_r224000145 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala --- @@ -3356,21 +3356,11 @@ class Dataset[T] private[sql]( * user-registered callback functions. */ private def withAction[U](name: String, qe: QueryExecution)(action: SparkPlan => U) = { -try { - qe.executedPlan.foreach { plan => -plan.resetMetrics() - } - val start = System.nanoTime() - val result = SQLExecution.withNewExecutionId(sparkSession, qe) { -action(qe.executedPlan) - } - val end = System.nanoTime() - sparkSession.listenerManager.onSuccess(name, qe, end - start) - result -} catch { - case e: Exception => -sparkSession.listenerManager.onFailure(name, qe, e) -throw e +qe.executedPlan.foreach { plan => --- End diff -- can this throw an exception? Imagine if `df.count()` threw an exception, and then you run it again. Won't this be a behavior change in that case? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...
Github user brkyvz commented on a diff in the pull request: https://github.com/apache/spark/pull/22674#discussion_r224000809 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala --- @@ -71,14 +72,35 @@ object SQLExecution { val callSite = sc.getCallSite() withSQLConfPropagated(sparkSession) { -sc.listenerBus.post(SparkListenerSQLExecutionStart( - executionId, callSite.shortForm, callSite.longForm, queryExecution.toString, - SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan), System.currentTimeMillis())) +var ex: Option[Exception] = None +val startTime = System.currentTimeMillis() try { + sc.listenerBus.post(SparkListenerSQLExecutionStart( +executionId = executionId, +description = callSite.shortForm, +details = callSite.longForm, +physicalPlanDescription = queryExecution.toString, +// `queryExecution.executedPlan` triggers query planning. If it fails, the exception +// will be caught and reported in the `SparkListenerSQLExecutionEnd` +sparkPlanInfo = SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan), +time = startTime)) body +} catch { + case e: Exception => +ex = Some(e) +throw e } finally { - sc.listenerBus.post(SparkListenerSQLExecutionEnd( -executionId, System.currentTimeMillis())) + val endTime = System.currentTimeMillis() + val event = SparkListenerSQLExecutionEnd(executionId, endTime) + // Currently only `Dataset.withAction` and `DataFrameWriter.runCommand` specify the `name` + // parameter. The `ExecutionListenerManager` only watches SQL executions with name. We + // can specify the execution name in more places in the future, so that + // `QueryExecutionListener` can track more cases. + event.executionName = name + event.duration = endTime - startTime --- End diff -- duration used to be reported in nanos. Now it's millis. I would still report it as nanos if possible. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/22674#discussion_r223911185 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala --- @@ -75,95 +76,74 @@ trait QueryExecutionListener { */ @Experimental @InterfaceStability.Evolving -class ExecutionListenerManager private extends Logging { - - private[sql] def this(conf: SparkConf) = { -this() +// The `session` is used to indicate which session carries this listener manager, and we only +// catch SQL executions which are launched by the same session. +// The `loadExtensions` flag is used to indicate whether we should load the pre-defined, +// user-specified listeners during construction. We should not do it when cloning this listener +// manager, as we will copy all listeners to the cloned listener manager. +class ExecutionListenerManager private[sql](session: SparkSession, loadExtensions: Boolean) + extends SparkListener with Logging { + + private[this] val listeners = new CopyOnWriteArrayList[QueryExecutionListener] + + if (loadExtensions) { +val conf = session.sparkContext.conf conf.get(QUERY_EXECUTION_LISTENERS).foreach { classNames => Utils.loadExtensions(classOf[QueryExecutionListener], classNames, conf).foreach(register) } } + session.sparkContext.listenerBus.addToSharedQueue(this) + /** * Registers the specified [[QueryExecutionListener]]. */ @DeveloperApi - def register(listener: QueryExecutionListener): Unit = writeLock { -listeners += listener + def register(listener: QueryExecutionListener): Unit = { +listeners.add(listener) } /** * Unregisters the specified [[QueryExecutionListener]]. */ @DeveloperApi - def unregister(listener: QueryExecutionListener): Unit = writeLock { -listeners -= listener + def unregister(listener: QueryExecutionListener): Unit = { +listeners.remove(listener) } /** * Removes all the registered [[QueryExecutionListener]]. */ @DeveloperApi - def clear(): Unit = writeLock { + def clear(): Unit = { listeners.clear() } /** * Get an identical copy of this listener manager. */ - @DeveloperApi - override def clone(): ExecutionListenerManager = writeLock { -val newListenerManager = new ExecutionListenerManager -listeners.foreach(newListenerManager.register) + private[sql] def clone(session: SparkSession): ExecutionListenerManager = { +val newListenerManager = new ExecutionListenerManager(session, loadExtensions = false) +listeners.iterator().asScala.foreach(newListenerManager.register) newListenerManager } - private[sql] def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { -readLock { - withErrorHandling { listener => -listener.onSuccess(funcName, qe, duration) + override def onOtherEvent(event: SparkListenerEvent): Unit = event match { +case e: SparkListenerSQLExecutionEnd if shouldCatchEvent(e) => + val funcName = e.executionName.get + e.executionFailure match { +case Some(ex) => + listeners.iterator().asScala.foreach(_.onFailure(funcName, e.qe, ex)) +case _ => + listeners.iterator().asScala.foreach(_.onSuccess(funcName, e.qe, e.duration)) } -} - } - private[sql] def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit = { -readLock { - withErrorHandling { listener => -listener.onFailure(funcName, qe, exception) - } -} +case _ => // Ignore } - private[this] val listeners = ListBuffer.empty[QueryExecutionListener] - - /** A lock to prevent updating the list of listeners while we are traversing through them. */ - private[this] val lock = new ReentrantReadWriteLock() - - private def withErrorHandling(f: QueryExecutionListener => Unit): Unit = { -for (listener <- listeners) { - try { -f(listener) - } catch { -case NonFatal(e) => logWarning("Error executing query execution listener", e) - } -} - } - - /** Acquires a read lock on the cache for the duration of `f`. */ - private def readLock[A](f: => A): A = { -val rl = lock.readLock() -rl.lock() -try f finally { - rl.unlock() -} - } - - /** Acquires a write lock on the cache for the duration of `f`. */ - private def writeLock[A](f: => A): A = { -val wl = lock.writeLock() -wl.lock() -try f
[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/22674#discussion_r223910477 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala --- @@ -75,95 +76,74 @@ trait QueryExecutionListener { */ @Experimental @InterfaceStability.Evolving -class ExecutionListenerManager private extends Logging { - - private[sql] def this(conf: SparkConf) = { -this() +// The `session` is used to indicate which session carries this listener manager, and we only +// catch SQL executions which are launched by the same session. +// The `loadExtensions` flag is used to indicate whether we should load the pre-defined, +// user-specified listeners during construction. We should not do it when cloning this listener +// manager, as we will copy all listeners to the cloned listener manager. +class ExecutionListenerManager private[sql](session: SparkSession, loadExtensions: Boolean) + extends SparkListener with Logging { + + private[this] val listeners = new CopyOnWriteArrayList[QueryExecutionListener] + + if (loadExtensions) { +val conf = session.sparkContext.conf conf.get(QUERY_EXECUTION_LISTENERS).foreach { classNames => Utils.loadExtensions(classOf[QueryExecutionListener], classNames, conf).foreach(register) } } + session.sparkContext.listenerBus.addToSharedQueue(this) + /** * Registers the specified [[QueryExecutionListener]]. */ @DeveloperApi - def register(listener: QueryExecutionListener): Unit = writeLock { -listeners += listener + def register(listener: QueryExecutionListener): Unit = { +listeners.add(listener) } /** * Unregisters the specified [[QueryExecutionListener]]. */ @DeveloperApi - def unregister(listener: QueryExecutionListener): Unit = writeLock { -listeners -= listener + def unregister(listener: QueryExecutionListener): Unit = { +listeners.remove(listener) } /** * Removes all the registered [[QueryExecutionListener]]. */ @DeveloperApi - def clear(): Unit = writeLock { + def clear(): Unit = { listeners.clear() } /** * Get an identical copy of this listener manager. */ - @DeveloperApi - override def clone(): ExecutionListenerManager = writeLock { -val newListenerManager = new ExecutionListenerManager -listeners.foreach(newListenerManager.register) + private[sql] def clone(session: SparkSession): ExecutionListenerManager = { +val newListenerManager = new ExecutionListenerManager(session, loadExtensions = false) +listeners.iterator().asScala.foreach(newListenerManager.register) newListenerManager } - private[sql] def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { -readLock { - withErrorHandling { listener => -listener.onSuccess(funcName, qe, duration) + override def onOtherEvent(event: SparkListenerEvent): Unit = event match { +case e: SparkListenerSQLExecutionEnd if shouldCatchEvent(e) => + val funcName = e.executionName.get + e.executionFailure match { +case Some(ex) => + listeners.iterator().asScala.foreach(_.onFailure(funcName, e.qe, ex)) --- End diff -- `ExecutionListenerManager` is already a listener, which is running in a separated thread, receiving events from `LiveListenerBus` --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/22674#discussion_r223910145 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala --- @@ -75,95 +76,74 @@ trait QueryExecutionListener { */ @Experimental @InterfaceStability.Evolving -class ExecutionListenerManager private extends Logging { - - private[sql] def this(conf: SparkConf) = { -this() +// The `session` is used to indicate which session carries this listener manager, and we only --- End diff -- The constructor is private, so we should not make it visible in the class doc --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/22674#discussion_r223910028 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala --- @@ -39,7 +39,14 @@ case class SparkListenerSQLExecutionStart( @DeveloperApi case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long) - extends SparkListenerEvent + extends SparkListenerEvent { + + @JsonIgnore private[sql] var executionName: Option[String] = None --- End diff -- It's a developer api, which is public. The backward compatibility is not that strong, compared to end-user public APIs, but we should still keep them unchanged if not too hard. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/22674#discussion_r223910082 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala --- @@ -39,7 +39,14 @@ case class SparkListenerSQLExecutionStart( @DeveloperApi case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long) - extends SparkListenerEvent + extends SparkListenerEvent { + + @JsonIgnore private[sql] var executionName: Option[String] = None --- End diff -- that said, a developer can write a spark listener and catch this event. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...
Github user hvanhovell commented on a diff in the pull request: https://github.com/apache/spark/pull/22674#discussion_r223886858 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala --- @@ -75,95 +76,74 @@ trait QueryExecutionListener { */ @Experimental @InterfaceStability.Evolving -class ExecutionListenerManager private extends Logging { - - private[sql] def this(conf: SparkConf) = { -this() +// The `session` is used to indicate which session carries this listener manager, and we only +// catch SQL executions which are launched by the same session. +// The `loadExtensions` flag is used to indicate whether we should load the pre-defined, +// user-specified listeners during construction. We should not do it when cloning this listener +// manager, as we will copy all listeners to the cloned listener manager. +class ExecutionListenerManager private[sql](session: SparkSession, loadExtensions: Boolean) + extends SparkListener with Logging { + + private[this] val listeners = new CopyOnWriteArrayList[QueryExecutionListener] + + if (loadExtensions) { +val conf = session.sparkContext.conf conf.get(QUERY_EXECUTION_LISTENERS).foreach { classNames => Utils.loadExtensions(classOf[QueryExecutionListener], classNames, conf).foreach(register) } } + session.sparkContext.listenerBus.addToSharedQueue(this) + /** * Registers the specified [[QueryExecutionListener]]. */ @DeveloperApi - def register(listener: QueryExecutionListener): Unit = writeLock { -listeners += listener + def register(listener: QueryExecutionListener): Unit = { +listeners.add(listener) } /** * Unregisters the specified [[QueryExecutionListener]]. */ @DeveloperApi - def unregister(listener: QueryExecutionListener): Unit = writeLock { -listeners -= listener + def unregister(listener: QueryExecutionListener): Unit = { +listeners.remove(listener) } /** * Removes all the registered [[QueryExecutionListener]]. */ @DeveloperApi - def clear(): Unit = writeLock { + def clear(): Unit = { listeners.clear() } /** * Get an identical copy of this listener manager. */ - @DeveloperApi - override def clone(): ExecutionListenerManager = writeLock { -val newListenerManager = new ExecutionListenerManager -listeners.foreach(newListenerManager.register) + private[sql] def clone(session: SparkSession): ExecutionListenerManager = { +val newListenerManager = new ExecutionListenerManager(session, loadExtensions = false) +listeners.iterator().asScala.foreach(newListenerManager.register) newListenerManager } - private[sql] def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { -readLock { - withErrorHandling { listener => -listener.onSuccess(funcName, qe, duration) + override def onOtherEvent(event: SparkListenerEvent): Unit = event match { +case e: SparkListenerSQLExecutionEnd if shouldCatchEvent(e) => + val funcName = e.executionName.get + e.executionFailure match { +case Some(ex) => + listeners.iterator().asScala.foreach(_.onFailure(funcName, e.qe, ex)) --- End diff -- This is a bit of high level thought, you could consider making the calling event queue responsible for the dispatch of these events. That way you can leverage any improvement to the underlying event bus. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...
Github user hvanhovell commented on a diff in the pull request: https://github.com/apache/spark/pull/22674#discussion_r223885742 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala --- @@ -75,95 +76,74 @@ trait QueryExecutionListener { */ @Experimental @InterfaceStability.Evolving -class ExecutionListenerManager private extends Logging { - - private[sql] def this(conf: SparkConf) = { -this() +// The `session` is used to indicate which session carries this listener manager, and we only +// catch SQL executions which are launched by the same session. +// The `loadExtensions` flag is used to indicate whether we should load the pre-defined, +// user-specified listeners during construction. We should not do it when cloning this listener +// manager, as we will copy all listeners to the cloned listener manager. +class ExecutionListenerManager private[sql](session: SparkSession, loadExtensions: Boolean) + extends SparkListener with Logging { + + private[this] val listeners = new CopyOnWriteArrayList[QueryExecutionListener] + + if (loadExtensions) { +val conf = session.sparkContext.conf conf.get(QUERY_EXECUTION_LISTENERS).foreach { classNames => Utils.loadExtensions(classOf[QueryExecutionListener], classNames, conf).foreach(register) } } + session.sparkContext.listenerBus.addToSharedQueue(this) + /** * Registers the specified [[QueryExecutionListener]]. */ @DeveloperApi - def register(listener: QueryExecutionListener): Unit = writeLock { -listeners += listener + def register(listener: QueryExecutionListener): Unit = { +listeners.add(listener) } /** * Unregisters the specified [[QueryExecutionListener]]. */ @DeveloperApi - def unregister(listener: QueryExecutionListener): Unit = writeLock { -listeners -= listener + def unregister(listener: QueryExecutionListener): Unit = { +listeners.remove(listener) } /** * Removes all the registered [[QueryExecutionListener]]. */ @DeveloperApi - def clear(): Unit = writeLock { + def clear(): Unit = { listeners.clear() } /** * Get an identical copy of this listener manager. */ - @DeveloperApi - override def clone(): ExecutionListenerManager = writeLock { -val newListenerManager = new ExecutionListenerManager -listeners.foreach(newListenerManager.register) + private[sql] def clone(session: SparkSession): ExecutionListenerManager = { +val newListenerManager = new ExecutionListenerManager(session, loadExtensions = false) +listeners.iterator().asScala.foreach(newListenerManager.register) newListenerManager } - private[sql] def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { -readLock { - withErrorHandling { listener => -listener.onSuccess(funcName, qe, duration) + override def onOtherEvent(event: SparkListenerEvent): Unit = event match { +case e: SparkListenerSQLExecutionEnd if shouldCatchEvent(e) => + val funcName = e.executionName.get + e.executionFailure match { +case Some(ex) => + listeners.iterator().asScala.foreach(_.onFailure(funcName, e.qe, ex)) +case _ => + listeners.iterator().asScala.foreach(_.onSuccess(funcName, e.qe, e.duration)) } -} - } - private[sql] def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit = { -readLock { - withErrorHandling { listener => -listener.onFailure(funcName, qe, exception) - } -} +case _ => // Ignore } - private[this] val listeners = ListBuffer.empty[QueryExecutionListener] - - /** A lock to prevent updating the list of listeners while we are traversing through them. */ - private[this] val lock = new ReentrantReadWriteLock() - - private def withErrorHandling(f: QueryExecutionListener => Unit): Unit = { -for (listener <- listeners) { - try { -f(listener) - } catch { -case NonFatal(e) => logWarning("Error executing query execution listener", e) - } -} - } - - /** Acquires a read lock on the cache for the duration of `f`. */ - private def readLock[A](f: => A): A = { -val rl = lock.readLock() -rl.lock() -try f finally { - rl.unlock() -} - } - - /** Acquires a write lock on the cache for the duration of `f`. */ - private def writeLock[A](f: => A): A = { -val wl = lock.writeLock() -wl.lock() -try
[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...
Github user hvanhovell commented on a diff in the pull request: https://github.com/apache/spark/pull/22674#discussion_r223873662 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala --- @@ -75,95 +76,74 @@ trait QueryExecutionListener { */ @Experimental @InterfaceStability.Evolving -class ExecutionListenerManager private extends Logging { - - private[sql] def this(conf: SparkConf) = { -this() +// The `session` is used to indicate which session carries this listener manager, and we only --- End diff -- Why is this not a class doc? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...
Github user hvanhovell commented on a diff in the pull request: https://github.com/apache/spark/pull/22674#discussion_r223873406 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala --- @@ -39,7 +39,14 @@ case class SparkListenerSQLExecutionStart( @DeveloperApi case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long) - extends SparkListenerEvent + extends SparkListenerEvent { + + @JsonIgnore private[sql] var executionName: Option[String] = None --- End diff -- Why do we want to be backwards compatible here? SHS? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...
Github user jiangxb1987 commented on a diff in the pull request: https://github.com/apache/spark/pull/22674#discussion_r223729445 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala --- @@ -75,95 +76,69 @@ trait QueryExecutionListener { */ @Experimental @InterfaceStability.Evolving -class ExecutionListenerManager private extends Logging { +class ExecutionListenerManager private[sql](session: SparkSession, loadExtensions: Boolean) --- End diff -- nit: we shall add param comments. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...
Github user dongjoon-hyun commented on a diff in the pull request: https://github.com/apache/spark/pull/22674#discussion_r223450058 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala --- @@ -75,95 +76,70 @@ trait QueryExecutionListener { */ @Experimental @InterfaceStability.Evolving -class ExecutionListenerManager private extends Logging { +class ExecutionListenerManager private[sql](session: SparkSession, loadExtensions: Boolean) + extends SparkListener with Logging { - private[sql] def this(conf: SparkConf) = { -this() + private[this] val listeners = new CopyOnWriteArrayList[QueryExecutionListener] + + if (loadExtensions) { +val conf = session.sparkContext.conf conf.get(QUERY_EXECUTION_LISTENERS).foreach { classNames => Utils.loadExtensions(classOf[QueryExecutionListener], classNames, conf).foreach(register) } } + session.sparkContext.listenerBus.addToSharedQueue(this) + /** * Registers the specified [[QueryExecutionListener]]. */ @DeveloperApi - def register(listener: QueryExecutionListener): Unit = writeLock { -listeners += listener + def register(listener: QueryExecutionListener): Unit = { +listeners.add(listener) } /** * Unregisters the specified [[QueryExecutionListener]]. */ @DeveloperApi - def unregister(listener: QueryExecutionListener): Unit = writeLock { -listeners -= listener + def unregister(listener: QueryExecutionListener): Unit = { +listeners.remove(listener) } /** * Removes all the registered [[QueryExecutionListener]]. */ @DeveloperApi - def clear(): Unit = writeLock { + def clear(): Unit = { listeners.clear() } /** * Get an identical copy of this listener manager. */ @DeveloperApi - override def clone(): ExecutionListenerManager = writeLock { -val newListenerManager = new ExecutionListenerManager -listeners.foreach(newListenerManager.register) + def clone(session: SparkSession): ExecutionListenerManager = { --- End diff -- Could you add MiMa exclusion rule? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/22674#discussion_r223443736 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala --- @@ -75,95 +76,70 @@ trait QueryExecutionListener { */ @Experimental @InterfaceStability.Evolving -class ExecutionListenerManager private extends Logging { +class ExecutionListenerManager private[sql](session: SparkSession, loadExtensions: Boolean) + extends SparkListener with Logging { - private[sql] def this(conf: SparkConf) = { -this() + private[this] val listeners = new CopyOnWriteArrayList[QueryExecutionListener] + + if (loadExtensions) { +val conf = session.sparkContext.conf conf.get(QUERY_EXECUTION_LISTENERS).foreach { classNames => Utils.loadExtensions(classOf[QueryExecutionListener], classNames, conf).foreach(register) } } + session.sparkContext.listenerBus.addToSharedQueue(this) + /** * Registers the specified [[QueryExecutionListener]]. */ @DeveloperApi - def register(listener: QueryExecutionListener): Unit = writeLock { -listeners += listener + def register(listener: QueryExecutionListener): Unit = { +listeners.add(listener) } /** * Unregisters the specified [[QueryExecutionListener]]. */ @DeveloperApi - def unregister(listener: QueryExecutionListener): Unit = writeLock { -listeners -= listener + def unregister(listener: QueryExecutionListener): Unit = { +listeners.remove(listener) } /** * Removes all the registered [[QueryExecutionListener]]. */ @DeveloperApi - def clear(): Unit = writeLock { + def clear(): Unit = { listeners.clear() } /** * Get an identical copy of this listener manager. */ @DeveloperApi - override def clone(): ExecutionListenerManager = writeLock { -val newListenerManager = new ExecutionListenerManager -listeners.foreach(newListenerManager.register) + def clone(session: SparkSession): ExecutionListenerManager = { --- End diff -- I don't know why this method is public at the first place... I have to break it here. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/22674#discussion_r223443327 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala --- @@ -39,7 +39,14 @@ case class SparkListenerSQLExecutionStart( @DeveloperApi case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long) - extends SparkListenerEvent + extends SparkListenerEvent { + + @JsonIgnore private[sql] var executionName: Option[String] = None --- End diff -- For backward compatibility, I make these new fields `var`. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...
GitHub user cloud-fan opened a pull request: https://github.com/apache/spark/pull/22674 [SPARK-25680][SQL] SQL execution listener shouldn't happen on execution thread ## What changes were proposed in this pull request? The SQL execution listener framework was created from scratch(see https://github.com/apache/spark/pull/9078). It didn't leverage what we already have in the spark listener framework, and one major problem is, the listener runs on the spark execution thread, which means a bad listener can block spark's query processing. This PR re-implements the SQL execution listener framework. Now `ExecutionListenerManager` is just a normal spark listener, which watches the `SparkListenerSQLExecutionEnd` events and post events to the SQL execution listeners. ## How was this patch tested? existing tests. You can merge this pull request into a Git repository by running: $ git pull https://github.com/cloud-fan/spark listener Alternatively you can review and apply these changes as the patch at: https://github.com/apache/spark/pull/22674.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #22674 commit 1701f3b9df78babef60d3f0cad12a332e3f24ec8 Author: Wenchen Fan Date: 2018-10-08T17:18:44Z SQL execution listener shouldn't happen on execution thread --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org