[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...

2018-10-17 Thread asfgit
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...

2018-10-12 Thread cloud-fan
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...

2018-10-12 Thread brkyvz
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...

2018-10-10 Thread cloud-fan
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...

2018-10-10 Thread brkyvz
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...

2018-10-10 Thread cloud-fan
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 

[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...

2018-10-10 Thread cloud-fan
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...

2018-10-10 Thread cloud-fan
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...

2018-10-10 Thread brkyvz
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 

[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...

2018-10-10 Thread brkyvz
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...

2018-10-10 Thread brkyvz
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...

2018-10-09 Thread cloud-fan
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 

[GitHub] spark pull request #22674: [SPARK-25680][SQL] SQL execution listener shouldn...

2018-10-09 Thread cloud-fan
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...

2018-10-09 Thread cloud-fan
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...

2018-10-09 Thread cloud-fan
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...

2018-10-09 Thread cloud-fan
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...

2018-10-09 Thread hvanhovell
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...

2018-10-09 Thread hvanhovell
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...

2018-10-09 Thread hvanhovell
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...

2018-10-09 Thread hvanhovell
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...

2018-10-09 Thread jiangxb1987
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...

2018-10-08 Thread dongjoon-hyun
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...

2018-10-08 Thread cloud-fan
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...

2018-10-08 Thread cloud-fan
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...

2018-10-08 Thread cloud-fan
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