From e965c5f5ac99fe825b1d6e4907f9876f76735ea0 Mon Sep 17 00:00:00 2001 From: shahid Date: Sun, 3 Nov 2019 13:14:13 +0530 Subject: [PATCH 01/24] kvstore thrift --- .../deploy/history/FsHistoryProvider.scala | 3 +- sql/hive-thriftserver/pom.xml | 5 + ...apache.spark.status.AppHistoryServerPlugin | 1 + .../hive/thriftserver/HiveThriftServer2.scala | 199 +--------- ...HiveThriftServer2HistoryServerPlugin.scala | 39 ++ .../HiveThriftServer2Listener.scala | 353 ++++++++++++++++++ .../SparkExecuteStatementOperation.scala | 37 +- .../SparkGetCatalogsOperation.scala | 21 +- .../SparkGetColumnsOperation.scala | 21 +- .../SparkGetFunctionsOperation.scala | 21 +- .../SparkGetSchemasOperation.scala | 21 +- .../SparkGetTableTypesOperation.scala | 21 +- .../SparkGetTablesOperation.scala | 21 +- .../SparkGetTypeInfoOperation.scala | 21 +- .../thriftserver/SparkSQLSessionManager.scala | 11 +- .../ui/HiveThriftServer2AppStatusStore.scala | 117 ++++++ .../thriftserver/ui/ThriftServerPage.scala | 22 +- .../ui/ThriftServerSessionPage.scala | 12 +- .../thriftserver/ui/ThriftServerTab.scala | 16 +- .../HiveThriftServer2ListenerSuite.scala | 94 +++++ 20 files changed, 773 insertions(+), 283 deletions(-) create mode 100644 sql/hive-thriftserver/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2HistoryServerPlugin.scala create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala create mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 0b7e695efa870..dcef5f456e9a5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -353,7 +353,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) val ui = SparkUI.create(None, new HistoryAppStatusStore(conf, kvstore), conf, secManager, app.info.name, HistoryServer.getAttemptURI(appId, attempt.info.attemptId), attempt.info.startTime.getTime(), attempt.info.appSparkVersion) - loadPlugins().foreach(_.setupUI(ui)) + // To make SQL tab first, then JDBC/ODBC tab. + loadPlugins().toSeq.reverse.foreach(_.setupUI(ui)) val loadedUI = LoadedAppUI(ui) diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 9dd927084298a..bb2f5a8ac809b 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -125,6 +125,11 @@ net.sf.jpam jpam + + org.mockito + mockito-core + test + target/scala-${scala.binary.version}/classes diff --git a/sql/hive-thriftserver/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin b/sql/hive-thriftserver/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin new file mode 100644 index 0000000000000..c301debfb5206 --- /dev/null +++ b/sql/hive-thriftserver/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin @@ -0,0 +1 @@ +org.apache.spark.sql.hive.thriftserver.HiveThriftServer2HistoryServerPlugin diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index 9517a599be633..3e0233f36dc40 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -20,24 +20,21 @@ package org.apache.spark.sql.hive.thriftserver import java.util.Locale import java.util.concurrent.atomic.AtomicBoolean -import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer - import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hive.service.cli.thrift.{ThriftBinaryCLIService, ThriftHttpCLIService} import org.apache.hive.service.server.HiveServer2 -import org.apache.spark.SparkContext +import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging import org.apache.spark.internal.config.UI.UI_ENABLED -import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd, SparkListenerJobStart} import org.apache.spark.sql.SQLContext import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ -import org.apache.spark.sql.hive.thriftserver.ui.ThriftServerTab -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.hive.thriftserver.ui.{HiveThriftServer2AppStatusStore, ThriftServerTab} +import org.apache.spark.status.ElementTrackingStore +import org.apache.spark.ui.SparkUI import org.apache.spark.util.{ShutdownHookManager, Utils} /** @@ -62,16 +59,25 @@ object HiveThriftServer2 extends Logging { server.init(executionHive.conf) server.start() - listener = new HiveThriftServer2Listener(server, sqlContext.conf) + val kvStore = SparkSQLEnv.sqlContext.sparkContext + .statusStore.store.asInstanceOf[ElementTrackingStore] + listener = new HiveThriftServer2Listener(kvStore, Some(server), Some(sqlContext)) sqlContext.sparkContext.addSparkListener(listener) uiTab = if (sqlContext.sparkContext.getConf.get(UI_ENABLED)) { - Some(new ThriftServerTab(sqlContext.sparkContext)) + Some(new ThriftServerTab(new HiveThriftServer2AppStatusStore(kvStore, Some(listener)), + getSparkUI(sqlContext.sparkContext))) } else { None } server } + def getSparkUI(sparkContext: SparkContext): SparkUI = { + sparkContext.ui.getOrElse { + throw new SparkException("Parent SparkUI to attach this tab to not found!") + } + } + def main(args: Array[String]): Unit = { // If the arguments contains "-h" or "--help", print out the usage and exit. if (args.contains("-h") || args.contains("--help")) { @@ -101,10 +107,12 @@ object HiveThriftServer2 extends Logging { server.init(executionHive.conf) server.start() logInfo("HiveThriftServer2 started") - listener = new HiveThriftServer2Listener(server, SparkSQLEnv.sqlContext.conf) + val kvStore = SparkSQLEnv.sparkContext.statusStore.store.asInstanceOf[ElementTrackingStore] + listener = new HiveThriftServer2Listener(kvStore, Some(server), Some(SparkSQLEnv.sqlContext)) SparkSQLEnv.sparkContext.addSparkListener(listener) uiTab = if (SparkSQLEnv.sparkContext.getConf.get(UI_ENABLED)) { - Some(new ThriftServerTab(SparkSQLEnv.sparkContext)) + Some(new ThriftServerTab(new HiveThriftServer2AppStatusStore(kvStore, Some(listener)), + getSparkUI(SparkSQLEnv.sparkContext))) } else { None } @@ -121,179 +129,10 @@ object HiveThriftServer2 extends Logging { } } - private[thriftserver] class SessionInfo( - val sessionId: String, - val startTimestamp: Long, - val ip: String, - val userName: String) { - var finishTimestamp: Long = 0L - var totalExecution: Int = 0 - def totalTime: Long = { - if (finishTimestamp == 0L) { - System.currentTimeMillis - startTimestamp - } else { - finishTimestamp - startTimestamp - } - } - } - private[thriftserver] object ExecutionState extends Enumeration { val STARTED, COMPILED, CANCELED, FAILED, FINISHED, CLOSED = Value type ExecutionState = Value } - - private[thriftserver] class ExecutionInfo( - val statement: String, - val sessionId: String, - val startTimestamp: Long, - val userName: String) { - var finishTimestamp: Long = 0L - var closeTimestamp: Long = 0L - var executePlan: String = "" - var detail: String = "" - var state: ExecutionState.Value = ExecutionState.STARTED - val jobId: ArrayBuffer[String] = ArrayBuffer[String]() - var groupId: String = "" - def totalTime(endTime: Long): Long = { - if (endTime == 0L) { - System.currentTimeMillis - startTimestamp - } else { - endTime - startTimestamp - } - } - } - - - /** - * An inner sparkListener called in sc.stop to clean up the HiveThriftServer2 - */ - private[thriftserver] class HiveThriftServer2Listener( - val server: HiveServer2, - val conf: SQLConf) extends SparkListener { - - override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = { - server.stop() - } - private val sessionList = new mutable.LinkedHashMap[String, SessionInfo] - private val executionList = new mutable.LinkedHashMap[String, ExecutionInfo] - private val retainedStatements = conf.getConf(SQLConf.THRIFTSERVER_UI_STATEMENT_LIMIT) - private val retainedSessions = conf.getConf(SQLConf.THRIFTSERVER_UI_SESSION_LIMIT) - - def getOnlineSessionNum: Int = synchronized { - sessionList.count(_._2.finishTimestamp == 0) - } - - def isExecutionActive(execInfo: ExecutionInfo): Boolean = { - !(execInfo.state == ExecutionState.FAILED || - execInfo.state == ExecutionState.CANCELED || - execInfo.state == ExecutionState.CLOSED) - } - - /** - * When an error or a cancellation occurs, we set the finishTimestamp of the statement. - * Therefore, when we count the number of running statements, we need to exclude errors and - * cancellations and count all statements that have not been closed so far. - */ - def getTotalRunning: Int = synchronized { - executionList.count { - case (_, v) => isExecutionActive(v) - } - } - - def getSessionList: Seq[SessionInfo] = synchronized { sessionList.values.toSeq } - - def getSession(sessionId: String): Option[SessionInfo] = synchronized { - sessionList.get(sessionId) - } - - def getExecutionList: Seq[ExecutionInfo] = synchronized { executionList.values.toSeq } - - override def onJobStart(jobStart: SparkListenerJobStart): Unit = synchronized { - for { - props <- Option(jobStart.properties) - groupId <- Option(props.getProperty(SparkContext.SPARK_JOB_GROUP_ID)) - (_, info) <- executionList if info.groupId == groupId - } { - info.jobId += jobStart.jobId.toString - info.groupId = groupId - } - } - - def onSessionCreated(ip: String, sessionId: String, userName: String = "UNKNOWN"): Unit = { - synchronized { - val info = new SessionInfo(sessionId, System.currentTimeMillis, ip, userName) - sessionList.put(sessionId, info) - trimSessionIfNecessary() - } - } - - def onSessionClosed(sessionId: String): Unit = synchronized { - sessionList(sessionId).finishTimestamp = System.currentTimeMillis - trimSessionIfNecessary() - } - - def onStatementStart( - id: String, - sessionId: String, - statement: String, - groupId: String, - userName: String = "UNKNOWN"): Unit = synchronized { - val info = new ExecutionInfo(statement, sessionId, System.currentTimeMillis, userName) - info.state = ExecutionState.STARTED - executionList.put(id, info) - trimExecutionIfNecessary() - sessionList(sessionId).totalExecution += 1 - executionList(id).groupId = groupId - } - - def onStatementParsed(id: String, executionPlan: String): Unit = synchronized { - executionList(id).executePlan = executionPlan - executionList(id).state = ExecutionState.COMPILED - } - - def onStatementCanceled(id: String): Unit = synchronized { - executionList(id).finishTimestamp = System.currentTimeMillis - executionList(id).state = ExecutionState.CANCELED - trimExecutionIfNecessary() - } - - def onStatementError(id: String, errorMsg: String, errorTrace: String): Unit = synchronized { - executionList(id).finishTimestamp = System.currentTimeMillis - executionList(id).detail = errorMsg - executionList(id).state = ExecutionState.FAILED - trimExecutionIfNecessary() - } - - def onStatementFinish(id: String): Unit = synchronized { - executionList(id).finishTimestamp = System.currentTimeMillis - executionList(id).state = ExecutionState.FINISHED - trimExecutionIfNecessary() - } - - def onOperationClosed(id: String): Unit = synchronized { - executionList(id).closeTimestamp = System.currentTimeMillis - executionList(id).state = ExecutionState.CLOSED - } - - private def trimExecutionIfNecessary() = { - if (executionList.size > retainedStatements) { - val toRemove = math.max(retainedStatements / 10, 1) - executionList.filter(_._2.finishTimestamp != 0).take(toRemove).foreach { s => - executionList.remove(s._1) - } - } - } - - private def trimSessionIfNecessary() = { - if (sessionList.size > retainedSessions) { - val toRemove = math.max(retainedSessions / 10, 1) - sessionList.filter(_._2.finishTimestamp != 0).take(toRemove).foreach { s => - sessionList.remove(s._1) - } - } - - } - } } private[hive] class HiveThriftServer2(sqlContext: SQLContext) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2HistoryServerPlugin.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2HistoryServerPlugin.scala new file mode 100644 index 0000000000000..6010daaef6b04 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2HistoryServerPlugin.scala @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import org.apache.spark.SparkConf +import org.apache.spark.scheduler.SparkListener +import org.apache.spark.sql.hive.thriftserver.ui.{HiveThriftServer2AppStatusStore, ThriftServerTab} +import org.apache.spark.status.{AppHistoryServerPlugin, ElementTrackingStore} +import org.apache.spark.ui.SparkUI + +class HiveThriftServer2HistoryServerPlugin extends AppHistoryServerPlugin { + + override def createListeners(conf: SparkConf, store: ElementTrackingStore): Seq[SparkListener] = { + Seq(new HiveThriftServer2Listener(store, None, None, Some(conf), false)) + } + + override def setupUI(ui: SparkUI): Unit = { + val store = new HiveThriftServer2AppStatusStore(ui.store.store) + if (store.getSessionCount() > 0) { + new ThriftServerTab(store, ui) + } + } +} + diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala new file mode 100644 index 0000000000000..44018d158dfe2 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala @@ -0,0 +1,353 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import java.util.concurrent.ConcurrentHashMap + +import org.apache.hive.service.server.HiveServer2 +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd, SparkListenerEvent, SparkListenerJobStart} +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.ExecutionState +import org.apache.spark.sql.hive.thriftserver.ui.{ExecutionInfo, SessionInfo} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.status.{ElementTrackingStore, KVUtils, LiveEntity} + +/** + * An inner sparkListener called in sc.stop to clean up the HiveThriftServer2 + */ +private[thriftserver] class HiveThriftServer2Listener( + kvstore: ElementTrackingStore, + server: Option[HiveServer2], + sqlContext: Option[SQLContext], + sparkConf: Option[SparkConf] = None, + live: Boolean = true) extends SparkListener { + + private val sessionList = new ConcurrentHashMap[String, LiveSessionData]() + private val executionList = new ConcurrentHashMap[String, LiveExecutionData]() + + private val (retainedStatements: Int, retainedSessions: Int) = { + if (live) { + val conf = sqlContext.get.conf + (conf.getConf(SQLConf.THRIFTSERVER_UI_STATEMENT_LIMIT), + conf.getConf(SQLConf.THRIFTSERVER_UI_SESSION_LIMIT)) + } else { + val conf = sparkConf.get + (conf.get(SQLConf.THRIFTSERVER_UI_STATEMENT_LIMIT), + conf.get(SQLConf.THRIFTSERVER_UI_SESSION_LIMIT)) + } + } + + private val sc: Option[SparkContext] = if (live) { + Some(sqlContext.get.sparkContext) + } else { + None + } + + kvstore.addTrigger(classOf[SessionInfo], retainedSessions) { count => + cleanupSession(count) + } + + kvstore.addTrigger(classOf[ExecutionInfo], retainedStatements) { count => + cleanupExecutions(count) + } + + kvstore.onFlush { + if (!live) { + val now = System.nanoTime() + flush(update(_, now)) + executionList.keys().asScala.foreach( + key => executionList.remove(key) + ) + sessionList.keys().asScala.foreach( + key => sessionList.remove(key) + ) + } + } + + def postLiveListenerBus(event: SparkListenerEvent): Unit = { + if (live) { + sc.get.listenerBus.post(event) + } + } + + override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = { + if (live && server.isDefined) { + server.get.stop() + } + } + + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { + if (jobStart.properties != null) { + val groupId = jobStart.properties.getProperty(SparkContext.SPARK_JOB_GROUP_ID) + if (groupId != null) { + executionList.values().asScala.filter(_.groupId == groupId).foreach( + exec => { + exec.jobId += jobStart.jobId.toString + exec.groupId = groupId + updateLiveStore(exec) + } + ) + } + } + } + + override def onOtherEvent(event: SparkListenerEvent): Unit = { + event match { + case e: SparkListenerSessionCreated => onSessionCreated(e) + case e: SparkListenerSessionClosed => onSessionClosed(e) + case e: SparkListenerStatementStart => onStatementStart(e) + case e: SparkListenerStatementParsed => onStatementParsed(e) + case e: SparkListenerStatementCanceled => onStatementCanceled(e) + case e: SparkListenerStatementError => onStatementError(e) + case e: SparkListenerStatementFinish => onStatementFinish(e) + case e: SparkListenerOperationClosed => onOperationClosed(e) + case _ => // Ignore + } + } + + def onSessionCreated(e: SparkListenerSessionCreated): Unit = { + val session = getOrCreateSession(e.sessionId, e.startTime, e.ip, e.userName) + sessionList.put(e.sessionId, session) + updateLiveStore(session) + } + + def onSessionClosed(e: SparkListenerSessionClosed): Unit = { + val session = sessionList.get(e.sessionId) + session.finishTimestamp = e.finishTime + updateLiveStore(session) + if (live) { + sessionList.remove(e.sessionId) + } + } + + def onStatementStart( e: SparkListenerStatementStart): Unit = { + val info = + getOrCreateExecution( + e.id, + e.statement, + e.sessionId, + e.startTime, + e.userName) + + info.state = ExecutionState.STARTED + executionList.put(e.id, info) + sessionList.get(e.sessionId).totalExecution += 1 + executionList.get(e.id).groupId = e.groupId + updateLiveStore(executionList.get(e.id)) + updateLiveStore(sessionList.get(e.sessionId)) + } + + def onStatementParsed(e: SparkListenerStatementParsed): Unit = { + executionList.get(e.id).executePlan = e.executionPlan + executionList.get(e.id).state = ExecutionState.COMPILED + updateLiveStore(executionList.get(e.id)) + } + + def onStatementCanceled(e: SparkListenerStatementCanceled): Unit = { + executionList.get(e.id).finishTimestamp = e.finishTime + executionList.get(e.id).state = ExecutionState.CANCELED + updateLiveStore(executionList.get(e.id)) + } + + def onStatementError(e: SparkListenerStatementError): Unit = { + executionList.get(e.id).finishTimestamp = e.finishTime + executionList.get(e.id).detail = e.errorMsg + executionList.get(e.id).state = ExecutionState.FAILED + updateLiveStore(executionList.get(e.id)) + } + + def onStatementFinish(e: SparkListenerStatementFinish): Unit = { + executionList.get(e.id).finishTimestamp = e.finishTime + executionList.get(e.id).state = ExecutionState.FINISHED + updateLiveStore(executionList.get(e.id)) + } + + def onOperationClosed(e: SparkListenerOperationClosed): Unit = { + executionList.get(e.id).closeTimestamp = e.closeTime + executionList.get(e.id).state = ExecutionState.CLOSED + updateLiveStore(executionList.get(e.id)) + if (live) { + executionList.remove(e.id) + } + } + + /** Go through all `LiveEntity`s and use `entityFlushFunc(entity)` to flush them. */ + private def flush(entityFlushFunc: LiveEntity => Unit): Unit = { + sessionList.values.asScala.foreach(entityFlushFunc) + executionList.values.asScala.foreach(entityFlushFunc) + } + + private def update(entity: LiveEntity, now: Long): Unit = { + entity.write(kvstore, now) + } + + def updateLiveStore(session: LiveEntity): Unit = { + if (live) { + session.write(kvstore, System.nanoTime()) + } + } + + private def getOrCreateSession( + sessionId: String, + startTime: Long, + ip: String, + username: String): LiveSessionData = { + sessionList.computeIfAbsent(sessionId, + (_: String) => new LiveSessionData(sessionId, startTime, ip, username)) + } + + private def getOrCreateExecution( + execId: String, statement: String, + sessionId: String, startTimestamp: Long, + userName: String): LiveExecutionData = { + executionList.computeIfAbsent(execId, + (_: String) => new LiveExecutionData(execId, statement, sessionId, startTimestamp, userName)) + } + + private def cleanupExecutions(count: Long): Unit = { + val countToDelete = calculateNumberToRemove(count, retainedStatements) + if (countToDelete <= 0L) { + return + } + val view = kvstore.view(classOf[ExecutionInfo]).index("execId").first(0L) + val toDelete = KVUtils.viewToSeq(view, countToDelete.toInt) { j => + j.finishTimestamp != 0 + } + toDelete.foreach { j => kvstore.delete(j.getClass(), j.execId) } + } + + private def cleanupSession(count: Long): Unit = { + val countToDelete = calculateNumberToRemove(count, retainedSessions) + if (countToDelete <= 0L) { + return + } + val view = kvstore.view(classOf[SessionInfo]).index("sessionId").first(0L) + val toDelete = KVUtils.viewToSeq(view, countToDelete.toInt) { j => + j.finishTimestamp != 0L + } + toDelete.foreach { j => kvstore.delete(j.getClass(), j.sessionId) } + } + + /** + * Remove at least (retainedSize / 10) items to reduce friction. Because tracking may be done + * asynchronously, this method may return 0 in case enough items have been deleted already. + */ + private def calculateNumberToRemove(dataSize: Long, retainedSize: Long): Long = { + if (dataSize > retainedSize) { + math.max(retainedSize / 10L, dataSize - retainedSize) + } else { + 0L + } + } +} + +private[thriftserver] class LiveExecutionData( + val execId: String, + val statement: String, + val sessionId: String, + val startTimestamp: Long, + val userName: String) extends LiveEntity { + + var finishTimestamp: Long = 0L + var closeTimestamp: Long = 0L + var executePlan: String = "" + var detail: String = "" + var state: ExecutionState.Value = ExecutionState.STARTED + val jobId: ArrayBuffer[String] = ArrayBuffer[String]() + var groupId: String = "" + + override protected def doUpdate(): Any = { + new ExecutionInfo( + execId, + statement, + sessionId, + startTimestamp, + userName, + finishTimestamp, + closeTimestamp, + executePlan, + detail, + state, + jobId, + groupId) + } +} + + +private[thriftserver] class LiveSessionData( + val sessionId: String, + val startTimeStamp: Long, + val ip: String, + val username: String) extends LiveEntity { + + var finishTimestamp: Long = 0L + var totalExecution: Int = 0 + + override protected def doUpdate(): Any = { + new SessionInfo( + sessionId, + startTimeStamp, + ip, + username, + finishTimestamp, + totalExecution) + } + +} + +private[thriftserver] case class SparkListenerSessionCreated( + ip: String, + sessionId: String, + userName: String, + startTime: Long) extends SparkListenerEvent + +private[thriftserver] case class SparkListenerSessionClosed( + sessionId: String, finishTime: Long) extends SparkListenerEvent + +private[thriftserver] case class SparkListenerStatementStart( + id: String, + sessionId: String, + statement: String, + groupId: String, + startTime: Long, + userName: String = "UNKNOWN") extends SparkListenerEvent + +private[thriftserver] case class SparkListenerStatementParsed( + id: String, + executionPlan: String) extends SparkListenerEvent + +private[thriftserver] case class SparkListenerStatementCanceled( + id: String, finishTime: Long) extends SparkListenerEvent + +private[thriftserver] case class SparkListenerStatementError( + id: String, + errorMsg: String, + errorTrace: String, + finishTime: Long) extends SparkListenerEvent + +private[thriftserver] case class SparkListenerStatementFinish(id: String, finishTime: Long) + extends SparkListenerEvent + +private[thriftserver] case class SparkListenerOperationClosed(id: String, closeTime: Long) + extends SparkListenerEvent + + diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index 68197a9de8566..8b9a347711feb 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -77,7 +77,8 @@ private[hive] class SparkExecuteStatementOperation( // RDDs will be cleaned automatically upon garbage collection. logInfo(s"Close statement with $statementId") cleanup(OperationState.CLOSED) - HiveThriftServer2.listener.onOperationClosed(statementId) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerOperationClosed(statementId, + System.currentTimeMillis())) } def addNonNullColumnValue(from: SparkRow, to: ArrayBuffer[Any], ordinal: Int): Unit = { @@ -195,12 +196,13 @@ private[hive] class SparkExecuteStatementOperation( setState(OperationState.PENDING) statementId = UUID.randomUUID().toString logInfo(s"Submitting query '$statement' with $statementId") - HiveThriftServer2.listener.onStatementStart( + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, statement, statementId, - parentSession.getUsername) + System.currentTimeMillis(), + parentSession.getUsername)) setHasResultSet(true) // avoid no resultset for async run if (!runInBackground) { @@ -245,15 +247,16 @@ private[hive] class SparkExecuteStatementOperation( case rejected: RejectedExecutionException => logError("Error submitting query in background, query rejected", rejected) setState(OperationState.ERROR) - HiveThriftServer2.listener.onStatementError( - statementId, rejected.getMessage, SparkUtils.exceptionString(rejected)) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( + statementId, rejected.getMessage, SparkUtils.exceptionString(rejected), + System.currentTimeMillis())) throw new HiveSQLException("The background threadpool cannot accept" + " new task for execution, please retry the operation", rejected) case NonFatal(e) => logError(s"Error executing query in background", e) setState(OperationState.ERROR) - HiveThriftServer2.listener.onStatementError( - statementId, e.getMessage, SparkUtils.exceptionString(e)) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( + statementId, e.getMessage, SparkUtils.exceptionString(e), System.currentTimeMillis())) throw new HiveSQLException(e) } } @@ -284,7 +287,8 @@ private[hive] class SparkExecuteStatementOperation( "in this session.") case _ => } - HiveThriftServer2.listener.onStatementParsed(statementId, result.queryExecution.toString()) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementParsed(statementId, + result.queryExecution.toString())) iter = { if (sqlContext.getConf(SQLConf.THRIFTSERVER_INCREMENTAL_COLLECT.key).toBoolean) { resultList = None @@ -315,13 +319,15 @@ private[hive] class SparkExecuteStatementOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.onStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException), + System.currentTimeMillis())) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.onStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root)) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root), + System.currentTimeMillis())) throw new HiveSQLException("Error running query: " + root.toString, root) } } @@ -329,7 +335,8 @@ private[hive] class SparkExecuteStatementOperation( synchronized { if (!getStatus.getState.isTerminal) { setState(OperationState.FINISHED) - HiveThriftServer2.listener.onStatementFinish(statementId) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementFinish(statementId, + System.currentTimeMillis())) } } sqlContext.sparkContext.clearJobGroup() @@ -341,7 +348,9 @@ private[hive] class SparkExecuteStatementOperation( if (!getStatus.getState.isTerminal) { logInfo(s"Cancel query with $statementId") cleanup(OperationState.CANCELED) - HiveThriftServer2.listener.onStatementCanceled(statementId) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementCanceled( + statementId, + System.currentTimeMillis())) } } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala index 6c8a5b00992da..6d3e6a22d07ba 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala @@ -44,7 +44,8 @@ private[hive] class SparkGetCatalogsOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.onOperationClosed(statementId) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerOperationClosed(statementId, + System.currentTimeMillis())) } override def runInternal(): Unit = { @@ -56,12 +57,13 @@ private[hive] class SparkGetCatalogsOperation( val executionHiveClassLoader = sqlContext.sharedState.jarClassLoader Thread.currentThread().setContextClassLoader(executionHiveClassLoader) - HiveThriftServer2.listener.onStatementStart( + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, statementId, - parentSession.getUsername) + System.currentTimeMillis(), + parentSession.getUsername)) try { if (isAuthV2Enabled) { @@ -74,16 +76,19 @@ private[hive] class SparkGetCatalogsOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.onStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException), + System.currentTimeMillis())) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.onStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root)) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root), + System.currentTimeMillis())) throw new HiveSQLException("Error getting catalogs: " + root.toString, root) } } - HiveThriftServer2.listener.onStatementFinish(statementId) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementFinish(statementId, + System.currentTimeMillis())) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala index f845a2285b9a3..64d99896b1598 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala @@ -63,7 +63,8 @@ private[hive] class SparkGetColumnsOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.onOperationClosed(statementId) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerOperationClosed(statementId, + System.currentTimeMillis())) } override def runInternal(): Unit = { @@ -78,12 +79,13 @@ private[hive] class SparkGetColumnsOperation( val executionHiveClassLoader = sqlContext.sharedState.jarClassLoader Thread.currentThread().setContextClassLoader(executionHiveClassLoader) - HiveThriftServer2.listener.onStatementStart( + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, statementId, - parentSession.getUsername) + System.currentTimeMillis(), + parentSession.getUsername)) val schemaPattern = convertSchemaPattern(schemaName) val tablePattern = convertIdentifierPattern(tableName, true) @@ -135,17 +137,20 @@ private[hive] class SparkGetColumnsOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.onStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException), + System.currentTimeMillis())) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.onStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root)) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root), + System.currentTimeMillis())) throw new HiveSQLException("Error getting columns: " + root.toString, root) } } - HiveThriftServer2.listener.onStatementFinish(statementId) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementFinish(statementId, + System.currentTimeMillis())) } private def addToRowSet( diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala index 1cdd8918421bb..46c13b1ad88da 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala @@ -54,7 +54,8 @@ private[hive] class SparkGetFunctionsOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.onOperationClosed(statementId) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerOperationClosed(statementId, + System.currentTimeMillis())) } override def runInternal(): Unit = { @@ -81,12 +82,13 @@ private[hive] class SparkGetFunctionsOperation( authorizeMetaGets(HiveOperationType.GET_FUNCTIONS, privObjs, cmdStr) } - HiveThriftServer2.listener.onStatementStart( + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, statementId, - parentSession.getUsername) + System.currentTimeMillis(), + parentSession.getUsername)) try { matchingDbs.foreach { db => @@ -110,16 +112,19 @@ private[hive] class SparkGetFunctionsOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.onStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException), + System.currentTimeMillis())) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.onStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root)) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root), + System.currentTimeMillis())) throw new HiveSQLException("Error getting functions: " + root.toString, root) } } - HiveThriftServer2.listener.onStatementFinish(statementId) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementFinish(statementId, + System.currentTimeMillis())) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala index 928610a6bcff9..21def2dd5303b 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala @@ -50,7 +50,8 @@ private[hive] class SparkGetSchemasOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.onOperationClosed(statementId) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerOperationClosed(statementId, + System.currentTimeMillis())) } override def runInternal(): Unit = { @@ -68,12 +69,13 @@ private[hive] class SparkGetSchemasOperation( authorizeMetaGets(HiveOperationType.GET_TABLES, null, cmdStr) } - HiveThriftServer2.listener.onStatementStart( + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, statementId, - parentSession.getUsername) + System.currentTimeMillis(), + parentSession.getUsername)) try { val schemaPattern = convertSchemaPattern(schemaName) @@ -93,16 +95,19 @@ private[hive] class SparkGetSchemasOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.onStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException), + System.currentTimeMillis())) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.onStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root)) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root), + System.currentTimeMillis())) throw new HiveSQLException("Error getting schemas: " + root.toString, root) } } - HiveThriftServer2.listener.onStatementFinish(statementId) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementFinish(statementId, + System.currentTimeMillis())) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala index ec03f1e148e69..845026424a51b 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala @@ -45,7 +45,8 @@ private[hive] class SparkGetTableTypesOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.onOperationClosed(statementId) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerOperationClosed(statementId, + System.currentTimeMillis())) } override def runInternal(): Unit = { @@ -61,12 +62,13 @@ private[hive] class SparkGetTableTypesOperation( authorizeMetaGets(HiveOperationType.GET_TABLETYPES, null) } - HiveThriftServer2.listener.onStatementStart( + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, statementId, - parentSession.getUsername) + System.currentTimeMillis(), + parentSession.getUsername)) try { val tableTypes = CatalogTableType.tableTypes.map(tableTypeString).toSet @@ -80,16 +82,19 @@ private[hive] class SparkGetTableTypesOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.onStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException), + System.currentTimeMillis())) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.onStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root)) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root), + System.currentTimeMillis())) throw new HiveSQLException("Error getting table types: " + root.toString, root) } } - HiveThriftServer2.listener.onStatementFinish(statementId) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementFinish(statementId, + System.currentTimeMillis())) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala index bf9cf7ad46d95..96724abdf63bd 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala @@ -59,7 +59,8 @@ private[hive] class SparkGetTablesOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.onOperationClosed(statementId) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerOperationClosed(statementId, + System.currentTimeMillis())) } override def runInternal(): Unit = { @@ -85,12 +86,13 @@ private[hive] class SparkGetTablesOperation( authorizeMetaGets(HiveOperationType.GET_TABLES, privObjs, cmdStr) } - HiveThriftServer2.listener.onStatementStart( + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, statementId, - parentSession.getUsername) + System.currentTimeMillis(), + parentSession.getUsername)) try { // Tables and views @@ -124,17 +126,20 @@ private[hive] class SparkGetTablesOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.onStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException), + System.currentTimeMillis())) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.onStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root)) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root), + System.currentTimeMillis())) throw new HiveSQLException("Error getting tables: " + root.toString, root) } } - HiveThriftServer2.listener.onStatementFinish(statementId) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementFinish(statementId, + System.currentTimeMillis())) } private def addToRowSet( diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala index 0d263b09d57d3..971832b2870c8 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala @@ -44,7 +44,8 @@ private[hive] class SparkGetTypeInfoOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.onOperationClosed(statementId) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerOperationClosed(statementId, + System.currentTimeMillis())) } override def runInternal(): Unit = { @@ -60,12 +61,13 @@ private[hive] class SparkGetTypeInfoOperation( authorizeMetaGets(HiveOperationType.GET_TYPEINFO, null) } - HiveThriftServer2.listener.onStatementStart( + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, statementId, - parentSession.getUsername) + System.currentTimeMillis(), + parentSession.getUsername)) try { ThriftserverShimUtils.supportedType().foreach(typeInfo => { @@ -98,16 +100,19 @@ private[hive] class SparkGetTypeInfoOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.onStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException), + System.currentTimeMillis())) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.onStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root)) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root), + System.currentTimeMillis())) throw new HiveSQLException("Error getting type info: " + root.toString, root) } } - HiveThriftServer2.listener.onStatementFinish(statementId) + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementFinish(statementId, + System.currentTimeMillis())) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala index c4248bfde38cc..e9b1fbd924ec6 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala @@ -55,8 +55,12 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, sqlContext: super.openSession(protocol, username, passwd, ipAddress, sessionConf, withImpersonation, delegationToken) val session = super.getSession(sessionHandle) - HiveThriftServer2.listener.onSessionCreated( - session.getIpAddress, sessionHandle.getSessionId.toString, session.getUsername) + + HiveThriftServer2.listener.postLiveListenerBus(SparkListenerSessionCreated( + session.getIpAddress, + sessionHandle.getSessionId.toString, + session.getUsername, + System.currentTimeMillis())) val ctx = if (sqlContext.conf.hiveThriftServerSingleSession) { sqlContext } else { @@ -74,7 +78,8 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, sqlContext: } override def closeSession(sessionHandle: SessionHandle): Unit = { - HiveThriftServer2.listener.onSessionClosed(sessionHandle.getSessionId.toString) + HiveThriftServer2.listener.postLiveListenerBus( + SparkListenerSessionClosed(sessionHandle.getSessionId.toString, System.currentTimeMillis())) super.closeSession(sessionHandle) sparkSqlOperationManager.sessionToActivePool.remove(sessionHandle) sparkSqlOperationManager.sessionToContexts.remove(sessionHandle) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala new file mode 100644 index 0000000000000..b1a8080cadb82 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver.ui + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.ExecutionState +import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2Listener +import org.apache.spark.status.KVUtils.KVIndexParam +import org.apache.spark.util.kvstore.KVStore + +/** + * Provides a view of a KVStore with methods that make it easy to query SQL-specific state. There's + * no state kept in this class, so it's ok to have multiple instances of it in an application. + */ +class HiveThriftServer2AppStatusStore( + store: KVStore, + val listener: Option[HiveThriftServer2Listener] = None) { + + def getSessionList: Seq[SessionInfo] = { + store.view(classOf[SessionInfo]).asScala.toSeq + } + + def getExecutionList: Seq[ExecutionInfo] = { + store.view(classOf[ExecutionInfo]).asScala.toSeq + } + + def getOnlineSessionNum: Int = { + store.view(classOf[SessionInfo]).asScala.count(_.finishTimestamp == 0) + } + + def getSession(sessionId: String): Option[SessionInfo] = { + try { + Some(store.read(classOf[SessionInfo], sessionId)) + } catch { + case _: NoSuchElementException => None + } + } + /** + * When an error or a cancellation occurs, we set the finishTimestamp of the statement. + * Therefore, when we count the number of running statements, we need to exclude errors and + * cancellations and count all statements that have not been closed so far. + */ + def getTotalRunning: Int = { + store.view(classOf[ExecutionInfo]).asScala.count(isExecutionActive) + } + + def isExecutionActive(execInfo: ExecutionInfo): Boolean = { + !(execInfo.state == ExecutionState.FAILED || + execInfo.state == ExecutionState.CANCELED || + execInfo.state == ExecutionState.CLOSED) + } + + def getSessionCount(): Long = { + store.count(classOf[SessionInfo]) + } + + def getExecutionCount(): Long = { + store.count(classOf[ExecutionInfo]) + } +} + +private[thriftserver] class SessionInfo( + @KVIndexParam val sessionId: String, + val startTimestamp: Long, + val ip: String, + val userName: String, + val finishTimestamp: Long, + val totalExecution: Long) { + + def totalTime: Long = { + if (finishTimestamp == 0L) { + System.currentTimeMillis - startTimestamp + } else { + finishTimestamp - startTimestamp + } + } +} + +private[thriftserver] class ExecutionInfo( + @KVIndexParam val execId: String, + val statement: String, + val sessionId: String, + val startTimestamp: Long, + val userName: String, + val finishTimestamp: Long, + val closeTimestamp: Long, + val executePlan: String, + val detail: String, + val state: ExecutionState.Value, + val jobId: ArrayBuffer[String], + val groupId: String) { + + def totalTime(endTime: Long): Long = { + if (endTime == 0L) { + System.currentTimeMillis - startTimestamp + } else { + endTime - startTimestamp + } + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala index d224157098604..1500112b7d621 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala @@ -28,7 +28,6 @@ import scala.xml.{Node, Unparsed} import org.apache.commons.text.StringEscapeUtils import org.apache.spark.internal.Logging -import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.{ExecutionInfo, SessionInfo} import org.apache.spark.sql.hive.thriftserver.ui.ToolTips._ import org.apache.spark.ui._ import org.apache.spark.ui.UIUtils._ @@ -36,23 +35,20 @@ import org.apache.spark.util.Utils /** Page for Spark Web UI that shows statistics of the thrift server */ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("") with Logging { - - private val listener = parent.listener - private val startTime = Calendar.getInstance().getTime() + private val store = parent.store + private val startTime = parent.startTime /** Render the page */ def render(request: HttpServletRequest): Seq[Node] = { - val content = - listener.synchronized { // make sure all parts in this page are consistent + val content = // make sure all parts in this page are consistent generateBasicStats() ++
++

- {listener.getOnlineSessionNum} session(s) are online, - running {listener.getTotalRunning} SQL statement(s) + {store.getOnlineSessionNum} session(s) are online, + running {store.getTotalRunning} SQL statement(s)

++ generateSessionStatsTable(request) ++ generateSQLStatsTable(request) - } UIUtils.headerSparkPage(request, "JDBC/ODBC Server", content, parent) } @@ -72,7 +68,7 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" /** Generate stats of batch statements of the thrift server program */ private def generateSQLStatsTable(request: HttpServletRequest): Seq[Node] = { - val numStatement = listener.getExecutionList.size + val numStatement = store.getExecutionList.size val table = if (numStatement > 0) { @@ -103,7 +99,7 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" Some(new SqlStatsPagedTable( request, parent, - listener.getExecutionList, + store.getExecutionList, "sqlserver", UIUtils.prependBaseUri(request, parent.basePath), parameterOtherTable, @@ -138,7 +134,7 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" /** Generate stats of batch sessions of the thrift server program */ private def generateSessionStatsTable(request: HttpServletRequest): Seq[Node] = { - val numSessions = listener.getSessionList.size + val numSessions = store.getSessionList.size val table = if (numSessions > 0) { val sessionTableTag = "sessionstat" @@ -168,7 +164,7 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" Some(new SessionStatsPagedTable( request, parent, - listener.getSessionList, + store.getSessionList, "sqlserver", UIUtils.prependBaseUri(request, parent.basePath), parameterOtherTable, diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala index 8b275f8f7be0d..8dca771d0eb25 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala @@ -31,18 +31,16 @@ import org.apache.spark.util.Utils /** Page for Spark Web UI that shows statistics of jobs running in the thrift server */ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) extends WebUIPage("session") with Logging { - - private val listener = parent.listener - private val startTime = Calendar.getInstance().getTime() + val store = parent.store + private val startTime = parent.startTime /** Render the page */ def render(request: HttpServletRequest): Seq[Node] = { val parameterId = request.getParameter("id") require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") - val content = - listener.synchronized { // make sure all parts in this page are consistent - val sessionStat = listener.getSession(parameterId).getOrElse(null) + val content = { // make sure all parts in this page are consistent + val sessionStat = store.getSession(parameterId).getOrElse(null) require(sessionStat != null, "Invalid sessionID[" + parameterId + "]") generateBasicStats() ++ @@ -73,7 +71,7 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) /** Generate stats of batch statements of the thrift server program */ private def generateSQLStatsTable(request: HttpServletRequest, sessionID: String): Seq[Node] = { - val executionList = listener.getExecutionList + val executionList = store.getExecutionList .filter(_.sessionId == sessionID) val numStatement = executionList.size val table = if (numStatement > 0) { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala index 8efb2c3311cfe..a3fb7959cd1a7 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala @@ -19,28 +19,26 @@ package org.apache.spark.sql.hive.thriftserver.ui import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.internal.Logging -import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2 -import org.apache.spark.sql.hive.thriftserver.ui.ThriftServerTab._ +import org.apache.spark.sql.hive.thriftserver.{HiveThriftServer2, HiveThriftServer2Listener} import org.apache.spark.ui.{SparkUI, SparkUITab} /** * Spark Web UI tab that shows statistics of jobs running in the thrift server. * This assumes the given SparkContext has enabled its SparkUI. */ -private[thriftserver] class ThriftServerTab(sparkContext: SparkContext) - extends SparkUITab(getSparkUI(sparkContext), "sqlserver") with Logging { - +private[thriftserver] class ThriftServerTab( + val store: HiveThriftServer2AppStatusStore, + sparkUI: SparkUI) extends SparkUITab(sparkUI, "sqlserver") with Logging { override val name = "JDBC/ODBC Server" - val parent = getSparkUI(sparkContext) - val listener = HiveThriftServer2.listener + val parent = sparkUI + val startTime = sparkUI.store.applicationInfo().attempts.head.startTime attachPage(new ThriftServerPage(this)) attachPage(new ThriftServerSessionPage(this)) parent.attachTab(this) - def detach(): Unit = { - getSparkUI(sparkContext).detachTab(this) + sparkUI.detachTab(this) } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala new file mode 100644 index 0000000000000..b684178058857 --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import java.util.Properties + +import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} +import org.scalatest.BeforeAndAfter + +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.scheduler.SparkListenerJobStart +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2AppStatusStore +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.status.ElementTrackingStore +import org.apache.spark.util.kvstore.InMemoryStore + +class HiveThriftServer2ListenerSuite extends SparkFunSuite + with BeforeAndAfter { + + private var kvstore: ElementTrackingStore = _ + + after { + if (kvstore != null) { + kvstore.close() + kvstore = null + } + } + + private def createProperties: Properties = { + val properties = new Properties() + properties.setProperty(SparkContext.SPARK_JOB_GROUP_ID, "groupId") + properties + } + + private def createStatusStore(): HiveThriftServer2AppStatusStore = { + kvstore = new ElementTrackingStore(new InMemoryStore, new SparkConf()) + val server = mock(classOf[HiveThriftServer2], RETURNS_SMART_NULLS) + val sqlContext = mock(classOf[SQLContext]) + val sc = mock(classOf[SparkContext]) + val sqlConf = new SQLConf + when(sqlContext.conf).thenReturn(sqlConf) + when(sqlContext.sparkContext).thenReturn(sc) + val listener = new HiveThriftServer2Listener(kvstore, Some(server), Some(sqlContext)) + new HiveThriftServer2AppStatusStore(kvstore, Some(listener)) + } + + test("listener events should store successfully") { + val statusStore = createStatusStore() + val listener = statusStore.listener.get + + listener.onSessionCreated(SparkListenerSessionCreated("localhost", "sessionId", "user", + System.currentTimeMillis())) + listener.onStatementStart(SparkListenerStatementStart("id", "sessionId", "dummy query", + "groupId", System.currentTimeMillis(), "user")) + listener.onStatementParsed(SparkListenerStatementParsed("id", "dummy plan")) + listener.onJobStart(SparkListenerJobStart( + 0, + System.currentTimeMillis(), + Nil, + createProperties)) + listener.onStatementFinish(SparkListenerStatementFinish("id", System.currentTimeMillis())) + listener.onOperationClosed(SparkListenerOperationClosed("id", System.currentTimeMillis())) + + assert(statusStore.getOnlineSessionNum == 1) + + listener.onSessionClosed(SparkListenerSessionClosed("sessionId", System.currentTimeMillis())) + + assert(statusStore.getOnlineSessionNum == 0) + assert(statusStore.getExecutionList.size == 1) + + val storeExecData = statusStore.getExecutionList.head + + assert(storeExecData.execId == "id") + assert(storeExecData.sessionId == "sessionId") + assert(storeExecData.executePlan == "dummy plan") + assert(storeExecData.jobId == Seq("0")) + } +} From a93ca4eb428e93e67cd85219ba38deb479e384e6 Mon Sep 17 00:00:00 2001 From: shahid Date: Wed, 6 Nov 2019 01:01:18 +0530 Subject: [PATCH 02/24] add --- .../hive/thriftserver/HiveThriftServer2.scala | 18 ++++++++++-------- .../HiveThriftServer2HistoryServerPlugin.scala | 2 +- .../HiveThriftServer2Listener.scala | 7 +------ .../HiveThriftServer2ListenerSuite.scala | 2 +- 4 files changed, 13 insertions(+), 16 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index 3e0233f36dc40..35afe98cb3953 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -59,11 +59,11 @@ object HiveThriftServer2 extends Logging { server.init(executionHive.conf) server.start() - val kvStore = SparkSQLEnv.sqlContext.sparkContext - .statusStore.store.asInstanceOf[ElementTrackingStore] - listener = new HiveThriftServer2Listener(kvStore, Some(server), Some(sqlContext)) - sqlContext.sparkContext.addSparkListener(listener) - uiTab = if (sqlContext.sparkContext.getConf.get(UI_ENABLED)) { + val sc = sqlContext.sparkContext + val kvStore = sc.statusStore.store.asInstanceOf[ElementTrackingStore] + listener = new HiveThriftServer2Listener(kvStore, Some(server), Some(sqlContext), Some(sc)) + sc.addSparkListener(listener) + uiTab = if (sc.getConf.get(UI_ENABLED)) { Some(new ThriftServerTab(new HiveThriftServer2AppStatusStore(kvStore, Some(listener)), getSparkUI(sqlContext.sparkContext))) } else { @@ -108,9 +108,11 @@ object HiveThriftServer2 extends Logging { server.start() logInfo("HiveThriftServer2 started") val kvStore = SparkSQLEnv.sparkContext.statusStore.store.asInstanceOf[ElementTrackingStore] - listener = new HiveThriftServer2Listener(kvStore, Some(server), Some(SparkSQLEnv.sqlContext)) - SparkSQLEnv.sparkContext.addSparkListener(listener) - uiTab = if (SparkSQLEnv.sparkContext.getConf.get(UI_ENABLED)) { + val sc = SparkSQLEnv.sparkContext + listener = new HiveThriftServer2Listener(kvStore, Some(server), + Some(SparkSQLEnv.sqlContext), Some(sc)) + sc.addSparkListener(listener) + uiTab = if (sc.getConf.get(UI_ENABLED)) { Some(new ThriftServerTab(new HiveThriftServer2AppStatusStore(kvStore, Some(listener)), getSparkUI(SparkSQLEnv.sparkContext))) } else { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2HistoryServerPlugin.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2HistoryServerPlugin.scala index 6010daaef6b04..1dcb3ee4f2f8a 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2HistoryServerPlugin.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2HistoryServerPlugin.scala @@ -26,7 +26,7 @@ import org.apache.spark.ui.SparkUI class HiveThriftServer2HistoryServerPlugin extends AppHistoryServerPlugin { override def createListeners(conf: SparkConf, store: ElementTrackingStore): Seq[SparkListener] = { - Seq(new HiveThriftServer2Listener(store, None, None, Some(conf), false)) + Seq(new HiveThriftServer2Listener(store, None, None, None, Some(conf), false)) } override def setupUI(ui: SparkUI): Unit = { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala index 44018d158dfe2..e9a1227985757 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala @@ -38,6 +38,7 @@ private[thriftserver] class HiveThriftServer2Listener( kvstore: ElementTrackingStore, server: Option[HiveServer2], sqlContext: Option[SQLContext], + sc: Option[SparkContext], sparkConf: Option[SparkConf] = None, live: Boolean = true) extends SparkListener { @@ -56,12 +57,6 @@ private[thriftserver] class HiveThriftServer2Listener( } } - private val sc: Option[SparkContext] = if (live) { - Some(sqlContext.get.sparkContext) - } else { - None - } - kvstore.addTrigger(classOf[SessionInfo], retainedSessions) { count => cleanupSession(count) } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala index b684178058857..03c09107df1b5 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala @@ -56,7 +56,7 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite val sqlConf = new SQLConf when(sqlContext.conf).thenReturn(sqlConf) when(sqlContext.sparkContext).thenReturn(sc) - val listener = new HiveThriftServer2Listener(kvstore, Some(server), Some(sqlContext)) + val listener = new HiveThriftServer2Listener(kvstore, Some(server), Some(sqlContext), Some(sc)) new HiveThriftServer2AppStatusStore(kvstore, Some(listener)) } From 207de3aedae6edc1df5bcc143a3cec148aeaf4d3 Mon Sep 17 00:00:00 2001 From: shahid Date: Wed, 6 Nov 2019 19:18:38 +0530 Subject: [PATCH 03/24] address comment --- .../hive/thriftserver/HiveThriftServer2.scala | 2 +- .../HiveThriftServer2Listener.scala | 88 ++++++++++++++----- .../SparkExecuteStatementOperation.scala | 37 +++----- .../SparkGetCatalogsOperation.scala | 21 ++--- .../SparkGetColumnsOperation.scala | 21 ++--- .../SparkGetFunctionsOperation.scala | 21 ++--- .../SparkGetSchemasOperation.scala | 21 ++--- .../SparkGetTableTypesOperation.scala | 21 ++--- .../SparkGetTablesOperation.scala | 21 ++--- .../SparkGetTypeInfoOperation.scala | 21 ++--- .../thriftserver/SparkSQLSessionManager.scala | 11 +-- .../HiveThriftServer2ListenerSuite.scala | 15 ++-- 12 files changed, 146 insertions(+), 154 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index 35afe98cb3953..031c19cbf804b 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -107,8 +107,8 @@ object HiveThriftServer2 extends Logging { server.init(executionHive.conf) server.start() logInfo("HiveThriftServer2 started") - val kvStore = SparkSQLEnv.sparkContext.statusStore.store.asInstanceOf[ElementTrackingStore] val sc = SparkSQLEnv.sparkContext + val kvStore = sc.statusStore.store.asInstanceOf[ElementTrackingStore] listener = new HiveThriftServer2Listener(kvStore, Some(server), Some(SparkSQLEnv.sqlContext), Some(sc)) sc.addSparkListener(listener) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala index e9a1227985757..ea3fc45723bb0 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala @@ -107,25 +107,25 @@ private[thriftserver] class HiveThriftServer2Listener( override def onOtherEvent(event: SparkListenerEvent): Unit = { event match { - case e: SparkListenerSessionCreated => onSessionCreated(e) - case e: SparkListenerSessionClosed => onSessionClosed(e) - case e: SparkListenerStatementStart => onStatementStart(e) - case e: SparkListenerStatementParsed => onStatementParsed(e) - case e: SparkListenerStatementCanceled => onStatementCanceled(e) - case e: SparkListenerStatementError => onStatementError(e) - case e: SparkListenerStatementFinish => onStatementFinish(e) - case e: SparkListenerOperationClosed => onOperationClosed(e) + case e: SparkListenerSessionCreated => processEventSessionCreated(e) + case e: SparkListenerSessionClosed => processEventSessionClosed(e) + case e: SparkListenerStatementStart => processEventStatementStart(e) + case e: SparkListenerStatementParsed => processEventStatementParsed(e) + case e: SparkListenerStatementCanceled => processEventStatementCanceled(e) + case e: SparkListenerStatementError => processEventStatementError(e) + case e: SparkListenerStatementFinish => processEventStatementFinish(e) + case e: SparkListenerOperationClosed => processEventOperationClosed(e) case _ => // Ignore } } - def onSessionCreated(e: SparkListenerSessionCreated): Unit = { + def processEventSessionCreated(e: SparkListenerSessionCreated): Unit = { val session = getOrCreateSession(e.sessionId, e.startTime, e.ip, e.userName) sessionList.put(e.sessionId, session) updateLiveStore(session) } - def onSessionClosed(e: SparkListenerSessionClosed): Unit = { + def processEventSessionClosed(e: SparkListenerSessionClosed): Unit = { val session = sessionList.get(e.sessionId) session.finishTimestamp = e.finishTime updateLiveStore(session) @@ -134,14 +134,13 @@ private[thriftserver] class HiveThriftServer2Listener( } } - def onStatementStart( e: SparkListenerStatementStart): Unit = { - val info = - getOrCreateExecution( - e.id, - e.statement, - e.sessionId, - e.startTime, - e.userName) + def processEventStatementStart(e: SparkListenerStatementStart): Unit = { + val info = getOrCreateExecution( + e.id, + e.statement, + e.sessionId, + e.startTime, + e.userName) info.state = ExecutionState.STARTED executionList.put(e.id, info) @@ -151,32 +150,32 @@ private[thriftserver] class HiveThriftServer2Listener( updateLiveStore(sessionList.get(e.sessionId)) } - def onStatementParsed(e: SparkListenerStatementParsed): Unit = { + def processEventStatementParsed(e: SparkListenerStatementParsed): Unit = { executionList.get(e.id).executePlan = e.executionPlan executionList.get(e.id).state = ExecutionState.COMPILED updateLiveStore(executionList.get(e.id)) } - def onStatementCanceled(e: SparkListenerStatementCanceled): Unit = { + def processEventStatementCanceled(e: SparkListenerStatementCanceled): Unit = { executionList.get(e.id).finishTimestamp = e.finishTime executionList.get(e.id).state = ExecutionState.CANCELED updateLiveStore(executionList.get(e.id)) } - def onStatementError(e: SparkListenerStatementError): Unit = { + def processEventStatementError(e: SparkListenerStatementError): Unit = { executionList.get(e.id).finishTimestamp = e.finishTime executionList.get(e.id).detail = e.errorMsg executionList.get(e.id).state = ExecutionState.FAILED updateLiveStore(executionList.get(e.id)) } - def onStatementFinish(e: SparkListenerStatementFinish): Unit = { + def processEventStatementFinish(e: SparkListenerStatementFinish): Unit = { executionList.get(e.id).finishTimestamp = e.finishTime executionList.get(e.id).state = ExecutionState.FINISHED updateLiveStore(executionList.get(e.id)) } - def onOperationClosed(e: SparkListenerOperationClosed): Unit = { + def processEventOperationClosed(e: SparkListenerOperationClosed): Unit = { executionList.get(e.id).closeTimestamp = e.closeTime executionList.get(e.id).state = ExecutionState.CLOSED updateLiveStore(executionList.get(e.id)) @@ -185,6 +184,49 @@ private[thriftserver] class HiveThriftServer2Listener( } } + + def onSessionCreated(ip: String, sessionId: String, userName: String = "UNKNOWN"): Unit = { + postLiveListenerBus(SparkListenerSessionCreated(ip, sessionId, + userName, System.currentTimeMillis())) + } + + def onSessionClosed(sessionId: String): Unit = { + postLiveListenerBus(SparkListenerSessionClosed(sessionId, System.currentTimeMillis())) + } + + def onStatementStart( + id: String, + sessionId: String, + statement: String, + groupId: String, + userName: String = "UNKNOWN"): Unit = { + postLiveListenerBus(SparkListenerStatementStart(id, sessionId, statement, groupId, + System.currentTimeMillis(), userName)) + } + + def onStatementParsed(id: String, executionPlan: String): Unit = { + postLiveListenerBus(SparkListenerStatementParsed(id, executionPlan)) + } + + def onStatementCanceled(id: String): Unit = { + postLiveListenerBus(SparkListenerStatementCanceled(id, System.currentTimeMillis())) + } + + def onStatementError(id: String, errorMsg: String, errorTrace: String): Unit = { + postLiveListenerBus(SparkListenerStatementError(id, errorMsg, errorTrace, + System.currentTimeMillis())) + } + + def onStatementFinish(id: String): Unit = { + postLiveListenerBus(SparkListenerStatementFinish(id, System.currentTimeMillis())) + + } + + def onOperationClosed(id: String): Unit = { + postLiveListenerBus(SparkListenerOperationClosed(id, System.currentTimeMillis())) + } + + /** Go through all `LiveEntity`s and use `entityFlushFunc(entity)` to flush them. */ private def flush(entityFlushFunc: LiveEntity => Unit): Unit = { sessionList.values.asScala.foreach(entityFlushFunc) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index 8b9a347711feb..68197a9de8566 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -77,8 +77,7 @@ private[hive] class SparkExecuteStatementOperation( // RDDs will be cleaned automatically upon garbage collection. logInfo(s"Close statement with $statementId") cleanup(OperationState.CLOSED) - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerOperationClosed(statementId, - System.currentTimeMillis())) + HiveThriftServer2.listener.onOperationClosed(statementId) } def addNonNullColumnValue(from: SparkRow, to: ArrayBuffer[Any], ordinal: Int): Unit = { @@ -196,13 +195,12 @@ private[hive] class SparkExecuteStatementOperation( setState(OperationState.PENDING) statementId = UUID.randomUUID().toString logInfo(s"Submitting query '$statement' with $statementId") - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementStart( + HiveThriftServer2.listener.onStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, statement, statementId, - System.currentTimeMillis(), - parentSession.getUsername)) + parentSession.getUsername) setHasResultSet(true) // avoid no resultset for async run if (!runInBackground) { @@ -247,16 +245,15 @@ private[hive] class SparkExecuteStatementOperation( case rejected: RejectedExecutionException => logError("Error submitting query in background, query rejected", rejected) setState(OperationState.ERROR) - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( - statementId, rejected.getMessage, SparkUtils.exceptionString(rejected), - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementError( + statementId, rejected.getMessage, SparkUtils.exceptionString(rejected)) throw new HiveSQLException("The background threadpool cannot accept" + " new task for execution, please retry the operation", rejected) case NonFatal(e) => logError(s"Error executing query in background", e) setState(OperationState.ERROR) - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( - statementId, e.getMessage, SparkUtils.exceptionString(e), System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementError( + statementId, e.getMessage, SparkUtils.exceptionString(e)) throw new HiveSQLException(e) } } @@ -287,8 +284,7 @@ private[hive] class SparkExecuteStatementOperation( "in this session.") case _ => } - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementParsed(statementId, - result.queryExecution.toString())) + HiveThriftServer2.listener.onStatementParsed(statementId, result.queryExecution.toString()) iter = { if (sqlContext.getConf(SQLConf.THRIFTSERVER_INCREMENTAL_COLLECT.key).toBoolean) { resultList = None @@ -319,15 +315,13 @@ private[hive] class SparkExecuteStatementOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException), - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root), - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root)) throw new HiveSQLException("Error running query: " + root.toString, root) } } @@ -335,8 +329,7 @@ private[hive] class SparkExecuteStatementOperation( synchronized { if (!getStatus.getState.isTerminal) { setState(OperationState.FINISHED) - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementFinish(statementId, - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementFinish(statementId) } } sqlContext.sparkContext.clearJobGroup() @@ -348,9 +341,7 @@ private[hive] class SparkExecuteStatementOperation( if (!getStatus.getState.isTerminal) { logInfo(s"Cancel query with $statementId") cleanup(OperationState.CANCELED) - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementCanceled( - statementId, - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementCanceled(statementId) } } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala index 6d3e6a22d07ba..6c8a5b00992da 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala @@ -44,8 +44,7 @@ private[hive] class SparkGetCatalogsOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerOperationClosed(statementId, - System.currentTimeMillis())) + HiveThriftServer2.listener.onOperationClosed(statementId) } override def runInternal(): Unit = { @@ -57,13 +56,12 @@ private[hive] class SparkGetCatalogsOperation( val executionHiveClassLoader = sqlContext.sharedState.jarClassLoader Thread.currentThread().setContextClassLoader(executionHiveClassLoader) - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementStart( + HiveThriftServer2.listener.onStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, statementId, - System.currentTimeMillis(), - parentSession.getUsername)) + parentSession.getUsername) try { if (isAuthV2Enabled) { @@ -76,19 +74,16 @@ private[hive] class SparkGetCatalogsOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException), - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root), - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root)) throw new HiveSQLException("Error getting catalogs: " + root.toString, root) } } - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementFinish(statementId, - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementFinish(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala index 64d99896b1598..f845a2285b9a3 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala @@ -63,8 +63,7 @@ private[hive] class SparkGetColumnsOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerOperationClosed(statementId, - System.currentTimeMillis())) + HiveThriftServer2.listener.onOperationClosed(statementId) } override def runInternal(): Unit = { @@ -79,13 +78,12 @@ private[hive] class SparkGetColumnsOperation( val executionHiveClassLoader = sqlContext.sharedState.jarClassLoader Thread.currentThread().setContextClassLoader(executionHiveClassLoader) - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementStart( + HiveThriftServer2.listener.onStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, statementId, - System.currentTimeMillis(), - parentSession.getUsername)) + parentSession.getUsername) val schemaPattern = convertSchemaPattern(schemaName) val tablePattern = convertIdentifierPattern(tableName, true) @@ -137,20 +135,17 @@ private[hive] class SparkGetColumnsOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException), - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root), - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root)) throw new HiveSQLException("Error getting columns: " + root.toString, root) } } - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementFinish(statementId, - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementFinish(statementId) } private def addToRowSet( diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala index 46c13b1ad88da..1cdd8918421bb 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala @@ -54,8 +54,7 @@ private[hive] class SparkGetFunctionsOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerOperationClosed(statementId, - System.currentTimeMillis())) + HiveThriftServer2.listener.onOperationClosed(statementId) } override def runInternal(): Unit = { @@ -82,13 +81,12 @@ private[hive] class SparkGetFunctionsOperation( authorizeMetaGets(HiveOperationType.GET_FUNCTIONS, privObjs, cmdStr) } - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementStart( + HiveThriftServer2.listener.onStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, statementId, - System.currentTimeMillis(), - parentSession.getUsername)) + parentSession.getUsername) try { matchingDbs.foreach { db => @@ -112,19 +110,16 @@ private[hive] class SparkGetFunctionsOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException), - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root), - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root)) throw new HiveSQLException("Error getting functions: " + root.toString, root) } } - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementFinish(statementId, - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementFinish(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala index 21def2dd5303b..928610a6bcff9 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala @@ -50,8 +50,7 @@ private[hive] class SparkGetSchemasOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerOperationClosed(statementId, - System.currentTimeMillis())) + HiveThriftServer2.listener.onOperationClosed(statementId) } override def runInternal(): Unit = { @@ -69,13 +68,12 @@ private[hive] class SparkGetSchemasOperation( authorizeMetaGets(HiveOperationType.GET_TABLES, null, cmdStr) } - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementStart( + HiveThriftServer2.listener.onStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, statementId, - System.currentTimeMillis(), - parentSession.getUsername)) + parentSession.getUsername) try { val schemaPattern = convertSchemaPattern(schemaName) @@ -95,19 +93,16 @@ private[hive] class SparkGetSchemasOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException), - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root), - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root)) throw new HiveSQLException("Error getting schemas: " + root.toString, root) } } - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementFinish(statementId, - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementFinish(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala index 845026424a51b..ec03f1e148e69 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala @@ -45,8 +45,7 @@ private[hive] class SparkGetTableTypesOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerOperationClosed(statementId, - System.currentTimeMillis())) + HiveThriftServer2.listener.onOperationClosed(statementId) } override def runInternal(): Unit = { @@ -62,13 +61,12 @@ private[hive] class SparkGetTableTypesOperation( authorizeMetaGets(HiveOperationType.GET_TABLETYPES, null) } - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementStart( + HiveThriftServer2.listener.onStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, statementId, - System.currentTimeMillis(), - parentSession.getUsername)) + parentSession.getUsername) try { val tableTypes = CatalogTableType.tableTypes.map(tableTypeString).toSet @@ -82,19 +80,16 @@ private[hive] class SparkGetTableTypesOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException), - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root), - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root)) throw new HiveSQLException("Error getting table types: " + root.toString, root) } } - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementFinish(statementId, - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementFinish(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala index 96724abdf63bd..bf9cf7ad46d95 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala @@ -59,8 +59,7 @@ private[hive] class SparkGetTablesOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerOperationClosed(statementId, - System.currentTimeMillis())) + HiveThriftServer2.listener.onOperationClosed(statementId) } override def runInternal(): Unit = { @@ -86,13 +85,12 @@ private[hive] class SparkGetTablesOperation( authorizeMetaGets(HiveOperationType.GET_TABLES, privObjs, cmdStr) } - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementStart( + HiveThriftServer2.listener.onStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, statementId, - System.currentTimeMillis(), - parentSession.getUsername)) + parentSession.getUsername) try { // Tables and views @@ -126,20 +124,17 @@ private[hive] class SparkGetTablesOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException), - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root), - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root)) throw new HiveSQLException("Error getting tables: " + root.toString, root) } } - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementFinish(statementId, - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementFinish(statementId) } private def addToRowSet( diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala index 971832b2870c8..0d263b09d57d3 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala @@ -44,8 +44,7 @@ private[hive] class SparkGetTypeInfoOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerOperationClosed(statementId, - System.currentTimeMillis())) + HiveThriftServer2.listener.onOperationClosed(statementId) } override def runInternal(): Unit = { @@ -61,13 +60,12 @@ private[hive] class SparkGetTypeInfoOperation( authorizeMetaGets(HiveOperationType.GET_TYPEINFO, null) } - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementStart( + HiveThriftServer2.listener.onStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, statementId, - System.currentTimeMillis(), - parentSession.getUsername)) + parentSession.getUsername) try { ThriftserverShimUtils.supportedType().foreach(typeInfo => { @@ -100,19 +98,16 @@ private[hive] class SparkGetTypeInfoOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException), - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root), - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root)) throw new HiveSQLException("Error getting type info: " + root.toString, root) } } - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerStatementFinish(statementId, - System.currentTimeMillis())) + HiveThriftServer2.listener.onStatementFinish(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala index e9b1fbd924ec6..c4248bfde38cc 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala @@ -55,12 +55,8 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, sqlContext: super.openSession(protocol, username, passwd, ipAddress, sessionConf, withImpersonation, delegationToken) val session = super.getSession(sessionHandle) - - HiveThriftServer2.listener.postLiveListenerBus(SparkListenerSessionCreated( - session.getIpAddress, - sessionHandle.getSessionId.toString, - session.getUsername, - System.currentTimeMillis())) + HiveThriftServer2.listener.onSessionCreated( + session.getIpAddress, sessionHandle.getSessionId.toString, session.getUsername) val ctx = if (sqlContext.conf.hiveThriftServerSingleSession) { sqlContext } else { @@ -78,8 +74,7 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, sqlContext: } override def closeSession(sessionHandle: SessionHandle): Unit = { - HiveThriftServer2.listener.postLiveListenerBus( - SparkListenerSessionClosed(sessionHandle.getSessionId.toString, System.currentTimeMillis())) + HiveThriftServer2.listener.onSessionClosed(sessionHandle.getSessionId.toString) super.closeSession(sessionHandle) sparkSqlOperationManager.sessionToActivePool.remove(sessionHandle) sparkSqlOperationManager.sessionToContexts.remove(sessionHandle) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala index 03c09107df1b5..107b12a9d2d05 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala @@ -64,22 +64,21 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite val statusStore = createStatusStore() val listener = statusStore.listener.get - listener.onSessionCreated(SparkListenerSessionCreated("localhost", "sessionId", "user", - System.currentTimeMillis())) - listener.onStatementStart(SparkListenerStatementStart("id", "sessionId", "dummy query", - "groupId", System.currentTimeMillis(), "user")) - listener.onStatementParsed(SparkListenerStatementParsed("id", "dummy plan")) + listener.onSessionCreated("localhost", "sessionId", "user") + listener.onStatementStart("id", "sessionId", "dummy query", + "groupId", "user") + listener.onStatementParsed("id", "dummy plan") listener.onJobStart(SparkListenerJobStart( 0, System.currentTimeMillis(), Nil, createProperties)) - listener.onStatementFinish(SparkListenerStatementFinish("id", System.currentTimeMillis())) - listener.onOperationClosed(SparkListenerOperationClosed("id", System.currentTimeMillis())) + listener.onStatementFinish("id") + listener.onOperationClosed("id") assert(statusStore.getOnlineSessionNum == 1) - listener.onSessionClosed(SparkListenerSessionClosed("sessionId", System.currentTimeMillis())) + listener.onSessionClosed("sessionId") assert(statusStore.getOnlineSessionNum == 0) assert(statusStore.getExecutionList.size == 1) From 7046ea2bfae8e8395486303a29c208eccae63ad9 Mon Sep 17 00:00:00 2001 From: shahid Date: Wed, 6 Nov 2019 19:45:41 +0530 Subject: [PATCH 04/24] remove duplicate pom --- sql/hive-thriftserver/pom.xml | 5 ----- 1 file changed, 5 deletions(-) diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 3d933cec48ec0..75c7f77942396 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -129,11 +129,6 @@ net.sf.jpam jpam - - org.mockito - mockito-core - test - target/scala-${scala.binary.version}/classes From c18ba076eef1af0697c44a4d2ab04607ee80f8fe Mon Sep 17 00:00:00 2001 From: shahid Date: Wed, 6 Nov 2019 20:51:26 +0530 Subject: [PATCH 05/24] update --- .../HiveThriftServer2Listener.scala | 6 +- .../HiveThriftServer2ListenerSuite.scala | 18 +++-- .../ui/ThriftServerPageSuite.scala | 70 ++++++++++++++----- 3 files changed, 66 insertions(+), 28 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala index ea3fc45723bb0..6f63fe2435c71 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala @@ -80,13 +80,13 @@ private[thriftserver] class HiveThriftServer2Listener( def postLiveListenerBus(event: SparkListenerEvent): Unit = { if (live) { - sc.get.listenerBus.post(event) + sc.foreach(_.listenerBus.post(event)) } } override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = { - if (live && server.isDefined) { - server.get.stop() + if (live) { + server.foreach(_.stop()) } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala index 107b12a9d2d05..de8690b0292cc 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala @@ -64,21 +64,25 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite val statusStore = createStatusStore() val listener = statusStore.listener.get - listener.onSessionCreated("localhost", "sessionId", "user") - listener.onStatementStart("id", "sessionId", "dummy query", - "groupId", "user") - listener.onStatementParsed("id", "dummy plan") + listener.processEventSessionCreated(SparkListenerSessionCreated("localhost", "sessionId", + "user", System.currentTimeMillis())) + listener.processEventStatementStart(SparkListenerStatementStart("id", "sessionId", + "dummy query", "groupId", System.currentTimeMillis(), "user")) + listener.processEventStatementParsed(SparkListenerStatementParsed("id", "dummy plan")) listener.onJobStart(SparkListenerJobStart( 0, System.currentTimeMillis(), Nil, createProperties)) - listener.onStatementFinish("id") - listener.onOperationClosed("id") + listener.processEventStatementFinish(SparkListenerStatementFinish("id", + System.currentTimeMillis())) + listener.processEventOperationClosed(SparkListenerOperationClosed("id", + System.currentTimeMillis())) assert(statusStore.getOnlineSessionNum == 1) - listener.onSessionClosed("sessionId") + listener.processEventSessionClosed(SparkListenerSessionClosed("sessionId", + System.currentTimeMillis())) assert(statusStore.getOnlineSessionNum == 0) assert(statusStore.getExecutionList.size == 1) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala index 6f040d3f1095b..327b55b2d85b3 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala @@ -17,39 +17,70 @@ package org.apache.spark.sql.hive.thriftserver.ui -import java.util.Locale +import java.util.{Calendar, Locale} import javax.servlet.http.HttpServletRequest import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} +import org.scalatest.BeforeAndAfter -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.scheduler.SparkListenerJobStart -import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2 -import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.HiveThriftServer2Listener +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.hive.thriftserver._ import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.status.ElementTrackingStore +import org.apache.spark.util.kvstore.InMemoryStore -class ThriftServerPageSuite extends SparkFunSuite { + +class ThriftServerPageSuite extends SparkFunSuite with BeforeAndAfter { + + private var kvstore: ElementTrackingStore = _ + + after { + if (kvstore != null) { + kvstore.close() + kvstore = null + } + } /** - * Run a dummy session and return the listener + * Run a dummy session and return the store */ - private def getListener: HiveThriftServer2Listener = { - val listener = new HiveThriftServer2Listener(mock(classOf[HiveThriftServer2]), new SQLConf) - - listener.onSessionCreated("localhost", "sessionid", "user") - listener.onStatementStart("id", "sessionid", "dummy query", "groupid", "user") - listener.onStatementParsed("id", "dummy plan") + private def getStatusStore: HiveThriftServer2AppStatusStore = { + kvstore = new ElementTrackingStore(new InMemoryStore, new SparkConf()) + val server = mock(classOf[HiveThriftServer2], RETURNS_SMART_NULLS) + val sqlContext = mock(classOf[SQLContext]) + val sc = mock(classOf[SparkContext]) + val sqlConf = new SQLConf + when(sqlContext.conf).thenReturn(sqlConf) + when(sqlContext.sparkContext).thenReturn(sc) + + val listener = new HiveThriftServer2Listener(kvstore, Some(server), Some(sqlContext), Some(sc)) + val statusStore = new HiveThriftServer2AppStatusStore(kvstore, Some(listener)) + + listener.processEventSessionCreated(SparkListenerSessionCreated("localhost", "sessionid", + "user", System.currentTimeMillis())) + listener.processEventStatementStart(SparkListenerStatementStart("id", "sessionid", + "dummy query", "groupid", System.currentTimeMillis(),"user")) + listener.processEventStatementParsed(SparkListenerStatementParsed("id", "dummy plan")) listener.onJobStart(SparkListenerJobStart(0, System.currentTimeMillis(), Seq())) - listener.onStatementFinish("id") - listener.onOperationClosed("id") - listener.onSessionClosed("sessionid") - listener + listener.processEventStatementFinish(SparkListenerStatementFinish("id", + System.currentTimeMillis())) + listener.processEventOperationClosed(SparkListenerOperationClosed("id", + System.currentTimeMillis())) + listener.processEventSessionClosed(SparkListenerSessionClosed("sessionid", + System.currentTimeMillis())) + + statusStore } test("thriftserver page should load successfully") { + val store = getStatusStore + val request = mock(classOf[HttpServletRequest]) val tab = mock(classOf[ThriftServerTab], RETURNS_SMART_NULLS) - when(tab.listener).thenReturn(getListener) + when(tab.startTime).thenReturn(Calendar.getInstance().getTime) + when(tab.store).thenReturn(store) when(tab.appName).thenReturn("testing") when(tab.headerTabs).thenReturn(Seq.empty) val page = new ThriftServerPage(tab) @@ -70,10 +101,13 @@ class ThriftServerPageSuite extends SparkFunSuite { } test("thriftserver session page should load successfully") { + val store = getStatusStore + val request = mock(classOf[HttpServletRequest]) when(request.getParameter("id")).thenReturn("sessionid") val tab = mock(classOf[ThriftServerTab], RETURNS_SMART_NULLS) - when(tab.listener).thenReturn(getListener) + when(tab.startTime).thenReturn(Calendar.getInstance().getTime) + when(tab.store).thenReturn(store) when(tab.appName).thenReturn("testing") when(tab.headerTabs).thenReturn(Seq.empty) val page = new ThriftServerSessionPage(tab) From 0a261d1a57cd4d0e9370279b2366e219ac4c58b6 Mon Sep 17 00:00:00 2001 From: shahid Date: Wed, 6 Nov 2019 20:59:27 +0530 Subject: [PATCH 06/24] scalastyle --- .../spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala index 327b55b2d85b3..3af0b75574a37 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala @@ -61,7 +61,7 @@ class ThriftServerPageSuite extends SparkFunSuite with BeforeAndAfter { listener.processEventSessionCreated(SparkListenerSessionCreated("localhost", "sessionid", "user", System.currentTimeMillis())) listener.processEventStatementStart(SparkListenerStatementStart("id", "sessionid", - "dummy query", "groupid", System.currentTimeMillis(),"user")) + "dummy query", "groupid", System.currentTimeMillis(), "user")) listener.processEventStatementParsed(SparkListenerStatementParsed("id", "dummy plan")) listener.onJobStart(SparkListenerJobStart(0, System.currentTimeMillis(), Seq())) listener.processEventStatementFinish(SparkListenerStatementFinish("id", From 38e7424d0add921450f6d53445855a4b73d004e3 Mon Sep 17 00:00:00 2001 From: shahid Date: Wed, 6 Nov 2019 21:32:56 +0530 Subject: [PATCH 07/24] update --- .../HiveThriftServer2Listener.scala | 16 ++++++++-------- .../HiveThriftServer2ListenerSuite.scala | 19 ++++++++----------- .../ui/ThriftServerPageSuite.scala | 19 ++++++++----------- 3 files changed, 24 insertions(+), 30 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala index 6f63fe2435c71..4f36e5441d7c3 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala @@ -119,13 +119,13 @@ private[thriftserver] class HiveThriftServer2Listener( } } - def processEventSessionCreated(e: SparkListenerSessionCreated): Unit = { + private def processEventSessionCreated(e: SparkListenerSessionCreated): Unit = { val session = getOrCreateSession(e.sessionId, e.startTime, e.ip, e.userName) sessionList.put(e.sessionId, session) updateLiveStore(session) } - def processEventSessionClosed(e: SparkListenerSessionClosed): Unit = { + private def processEventSessionClosed(e: SparkListenerSessionClosed): Unit = { val session = sessionList.get(e.sessionId) session.finishTimestamp = e.finishTime updateLiveStore(session) @@ -134,7 +134,7 @@ private[thriftserver] class HiveThriftServer2Listener( } } - def processEventStatementStart(e: SparkListenerStatementStart): Unit = { + private def processEventStatementStart(e: SparkListenerStatementStart): Unit = { val info = getOrCreateExecution( e.id, e.statement, @@ -150,32 +150,32 @@ private[thriftserver] class HiveThriftServer2Listener( updateLiveStore(sessionList.get(e.sessionId)) } - def processEventStatementParsed(e: SparkListenerStatementParsed): Unit = { + private def processEventStatementParsed(e: SparkListenerStatementParsed): Unit = { executionList.get(e.id).executePlan = e.executionPlan executionList.get(e.id).state = ExecutionState.COMPILED updateLiveStore(executionList.get(e.id)) } - def processEventStatementCanceled(e: SparkListenerStatementCanceled): Unit = { + private def processEventStatementCanceled(e: SparkListenerStatementCanceled): Unit = { executionList.get(e.id).finishTimestamp = e.finishTime executionList.get(e.id).state = ExecutionState.CANCELED updateLiveStore(executionList.get(e.id)) } - def processEventStatementError(e: SparkListenerStatementError): Unit = { + private def processEventStatementError(e: SparkListenerStatementError): Unit = { executionList.get(e.id).finishTimestamp = e.finishTime executionList.get(e.id).detail = e.errorMsg executionList.get(e.id).state = ExecutionState.FAILED updateLiveStore(executionList.get(e.id)) } - def processEventStatementFinish(e: SparkListenerStatementFinish): Unit = { + private def processEventStatementFinish(e: SparkListenerStatementFinish): Unit = { executionList.get(e.id).finishTimestamp = e.finishTime executionList.get(e.id).state = ExecutionState.FINISHED updateLiveStore(executionList.get(e.id)) } - def processEventOperationClosed(e: SparkListenerOperationClosed): Unit = { + private def processEventOperationClosed(e: SparkListenerOperationClosed): Unit = { executionList.get(e.id).closeTimestamp = e.closeTime executionList.get(e.id).state = ExecutionState.CLOSED updateLiveStore(executionList.get(e.id)) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala index de8690b0292cc..73884cee32ade 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala @@ -64,25 +64,22 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite val statusStore = createStatusStore() val listener = statusStore.listener.get - listener.processEventSessionCreated(SparkListenerSessionCreated("localhost", "sessionId", - "user", System.currentTimeMillis())) - listener.processEventStatementStart(SparkListenerStatementStart("id", "sessionId", - "dummy query", "groupId", System.currentTimeMillis(), "user")) - listener.processEventStatementParsed(SparkListenerStatementParsed("id", "dummy plan")) + listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionId", "user", + System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerStatementStart("id", "sessionId", "dummy query", + "groupId", System.currentTimeMillis(), "user")) + listener.onOtherEvent(SparkListenerStatementParsed("id", "dummy plan")) listener.onJobStart(SparkListenerJobStart( 0, System.currentTimeMillis(), Nil, createProperties)) - listener.processEventStatementFinish(SparkListenerStatementFinish("id", - System.currentTimeMillis())) - listener.processEventOperationClosed(SparkListenerOperationClosed("id", - System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerStatementFinish("id", System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerOperationClosed("id", System.currentTimeMillis())) assert(statusStore.getOnlineSessionNum == 1) - listener.processEventSessionClosed(SparkListenerSessionClosed("sessionId", - System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerSessionClosed("sessionId", System.currentTimeMillis())) assert(statusStore.getOnlineSessionNum == 0) assert(statusStore.getExecutionList.size == 1) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala index 3af0b75574a37..9c70b3b548d3a 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala @@ -58,18 +58,15 @@ class ThriftServerPageSuite extends SparkFunSuite with BeforeAndAfter { val listener = new HiveThriftServer2Listener(kvstore, Some(server), Some(sqlContext), Some(sc)) val statusStore = new HiveThriftServer2AppStatusStore(kvstore, Some(listener)) - listener.processEventSessionCreated(SparkListenerSessionCreated("localhost", "sessionid", - "user", System.currentTimeMillis())) - listener.processEventStatementStart(SparkListenerStatementStart("id", "sessionid", - "dummy query", "groupid", System.currentTimeMillis(), "user")) - listener.processEventStatementParsed(SparkListenerStatementParsed("id", "dummy plan")) - listener.onJobStart(SparkListenerJobStart(0, System.currentTimeMillis(), Seq())) - listener.processEventStatementFinish(SparkListenerStatementFinish("id", - System.currentTimeMillis())) - listener.processEventOperationClosed(SparkListenerOperationClosed("id", - System.currentTimeMillis())) - listener.processEventSessionClosed(SparkListenerSessionClosed("sessionid", + listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionid", "user", System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerStatementStart("id", "sessionid", + "dummy query", "groupid", System.currentTimeMillis(), "user")) + listener.onOtherEvent(SparkListenerStatementParsed("id", "dummy plan")) + listener.onOtherEvent(SparkListenerJobStart(0, System.currentTimeMillis(), Seq())) + listener.onOtherEvent(SparkListenerStatementFinish("id", System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerOperationClosed("id", System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerSessionClosed("sessionid", System.currentTimeMillis())) statusStore } From c608ad01b8937119b601751324ab5ddde3d767e7 Mon Sep 17 00:00:00 2001 From: shahid Date: Wed, 6 Nov 2019 21:45:08 +0530 Subject: [PATCH 08/24] address comment --- .../spark/sql/hive/thriftserver/HiveThriftServer2.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index 031c19cbf804b..3e774f4f7354f 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -62,7 +62,8 @@ object HiveThriftServer2 extends Logging { val sc = sqlContext.sparkContext val kvStore = sc.statusStore.store.asInstanceOf[ElementTrackingStore] listener = new HiveThriftServer2Listener(kvStore, Some(server), Some(sqlContext), Some(sc)) - sc.addSparkListener(listener) + val liveListenerBus = sc.listenerBus + liveListenerBus.addToStatusQueue(listener) uiTab = if (sc.getConf.get(UI_ENABLED)) { Some(new ThriftServerTab(new HiveThriftServer2AppStatusStore(kvStore, Some(listener)), getSparkUI(sqlContext.sparkContext))) @@ -111,7 +112,8 @@ object HiveThriftServer2 extends Logging { val kvStore = sc.statusStore.store.asInstanceOf[ElementTrackingStore] listener = new HiveThriftServer2Listener(kvStore, Some(server), Some(SparkSQLEnv.sqlContext), Some(sc)) - sc.addSparkListener(listener) + val liveListenerBus = sc.listenerBus + liveListenerBus.addToStatusQueue(listener) uiTab = if (sc.getConf.get(UI_ENABLED)) { Some(new ThriftServerTab(new HiveThriftServer2AppStatusStore(kvStore, Some(listener)), getSparkUI(SparkSQLEnv.sparkContext))) From 2a03b1b60c269053f91f41449d2869bc383fd167 Mon Sep 17 00:00:00 2001 From: shahid Date: Mon, 11 Nov 2019 01:27:25 +0530 Subject: [PATCH 09/24] address comment --- .../hive/thriftserver/HiveThriftServer2.scala | 9 ++- .../HiveThriftServer2Listener.scala | 63 +++++++++++----- .../ui/HiveThriftServer2AppStatusStore.scala | 9 +++ .../HiveThriftServer2ListenerSuite.scala | 74 ++++++++++++++++--- .../ui/ThriftServerPageSuite.scala | 5 +- 5 files changed, 123 insertions(+), 37 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index 3e774f4f7354f..e190a95d288de 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -61,7 +61,8 @@ object HiveThriftServer2 extends Logging { server.start() val sc = sqlContext.sparkContext val kvStore = sc.statusStore.store.asInstanceOf[ElementTrackingStore] - listener = new HiveThriftServer2Listener(kvStore, Some(server), Some(sqlContext), Some(sc)) + val sqlConf = sqlContext.conf + listener = new HiveThriftServer2Listener(kvStore, Some(server), Some(sqlConf), Some(sc)) val liveListenerBus = sc.listenerBus liveListenerBus.addToStatusQueue(listener) uiTab = if (sc.getConf.get(UI_ENABLED)) { @@ -110,13 +111,13 @@ object HiveThriftServer2 extends Logging { logInfo("HiveThriftServer2 started") val sc = SparkSQLEnv.sparkContext val kvStore = sc.statusStore.store.asInstanceOf[ElementTrackingStore] - listener = new HiveThriftServer2Listener(kvStore, Some(server), - Some(SparkSQLEnv.sqlContext), Some(sc)) + val sqlConf = SparkSQLEnv.sqlContext.conf + listener = new HiveThriftServer2Listener(kvStore, Some(server), Some(sqlConf), Some(sc)) val liveListenerBus = sc.listenerBus liveListenerBus.addToStatusQueue(listener) uiTab = if (sc.getConf.get(UI_ENABLED)) { Some(new ThriftServerTab(new HiveThriftServer2AppStatusStore(kvStore, Some(listener)), - getSparkUI(SparkSQLEnv.sparkContext))) + getSparkUI(sc))) } else { None } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala index 4f36e5441d7c3..6a1bc3fc26489 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala @@ -24,8 +24,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd, SparkListenerEvent, SparkListenerJobStart} -import org.apache.spark.sql.SQLContext +import org.apache.spark.scheduler._ import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.ExecutionState import org.apache.spark.sql.hive.thriftserver.ui.{ExecutionInfo, SessionInfo} import org.apache.spark.sql.internal.SQLConf @@ -37,7 +36,7 @@ import org.apache.spark.status.{ElementTrackingStore, KVUtils, LiveEntity} private[thriftserver] class HiveThriftServer2Listener( kvstore: ElementTrackingStore, server: Option[HiveServer2], - sqlContext: Option[SQLContext], + sqlConf: Option[SQLConf], sc: Option[SparkContext], sparkConf: Option[SparkConf] = None, live: Boolean = true) extends SparkListener { @@ -47,7 +46,7 @@ private[thriftserver] class HiveThriftServer2Listener( private val (retainedStatements: Int, retainedSessions: Int) = { if (live) { - val conf = sqlContext.get.conf + val conf = sqlConf.get (conf.getConf(SQLConf.THRIFTSERVER_UI_STATEMENT_LIMIT), conf.getConf(SQLConf.THRIFTSERVER_UI_SESSION_LIMIT)) } else { @@ -57,6 +56,11 @@ private[thriftserver] class HiveThriftServer2Listener( } } + // Returns true if this listener has no live data. Exposed for tests only. + private[thriftserver] def noLiveData(): Boolean = { + sessionList.isEmpty && executionList.isEmpty + } + kvstore.addTrigger(classOf[SessionInfo], retainedSessions) { count => cleanupSession(count) } @@ -91,16 +95,40 @@ private[thriftserver] class HiveThriftServer2Listener( } override def onJobStart(jobStart: SparkListenerJobStart): Unit = { - if (jobStart.properties != null) { - val groupId = jobStart.properties.getProperty(SparkContext.SPARK_JOB_GROUP_ID) + val properties = jobStart.properties + if (properties != null) { + val groupId = properties.getProperty(SparkContext.SPARK_JOB_GROUP_ID) if (groupId != null) { - executionList.values().asScala.filter(_.groupId == groupId).foreach( - exec => { - exec.jobId += jobStart.jobId.toString - exec.groupId = groupId - updateLiveStore(exec) - } - ) + updateJobDetails(jobStart.jobId.toString, groupId) + } + } + } + + /** + * This method is to handle out of order events. ie. Job event come after execution end event. + * @param jobId + * @param groupId + */ + private def updateJobDetails(jobId: String, groupId: String): Unit = { + val execList = executionList.values().asScala.filter(_.groupId == groupId).toSeq + if (execList.nonEmpty) { + execList.foreach { exec => + exec.jobId += jobId.toString + exec.groupId = groupId + updateLiveStore(exec) + } + } else { + // This will happen only if JobStart event comes after Execution End event. This scenario may + // happen only for live listeners, as history listners removes the executionList at the end of + // replay. + val storeExecInfo = kvstore.view(classOf[ExecutionInfo]).asScala.filter(_.groupId == groupId) + storeExecInfo.foreach { exec => + val liveExec = getOrCreateExecution(exec.execId, exec.statement, exec.sessionId, + exec.startTimestamp, exec.userName) + liveExec.jobId += jobId.toString + liveExec.groupId = groupId + updateLiveStore(liveExec) + executionList.remove(liveExec.execId) } } } @@ -265,11 +293,11 @@ private[thriftserver] class HiveThriftServer2Listener( if (countToDelete <= 0L) { return } - val view = kvstore.view(classOf[ExecutionInfo]).index("execId").first(0L) + val view = kvstore.view(classOf[ExecutionInfo]) val toDelete = KVUtils.viewToSeq(view, countToDelete.toInt) { j => j.finishTimestamp != 0 } - toDelete.foreach { j => kvstore.delete(j.getClass(), j.execId) } + toDelete.foreach { j => kvstore.delete(j.getClass, j.execId) } } private def cleanupSession(count: Long): Unit = { @@ -277,11 +305,12 @@ private[thriftserver] class HiveThriftServer2Listener( if (countToDelete <= 0L) { return } - val view = kvstore.view(classOf[SessionInfo]).index("sessionId").first(0L) + val view = kvstore.view(classOf[SessionInfo]) val toDelete = KVUtils.viewToSeq(view, countToDelete.toInt) { j => j.finishTimestamp != 0L } - toDelete.foreach { j => kvstore.delete(j.getClass(), j.sessionId) } + + toDelete.foreach { j => kvstore.delete(j.getClass, j.sessionId) } } /** diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala index b1a8080cadb82..d75f1eef38c11 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala @@ -52,6 +52,15 @@ class HiveThriftServer2AppStatusStore( case _: NoSuchElementException => None } } + + def getExecution(executionId: String): Option[ExecutionInfo] = { + try { + Some(store.read(classOf[ExecutionInfo], executionId)) + } catch { + case _: NoSuchElementException => None + } + } + /** * When an error or a cancellation occurs, we set the finishTimestamp of the statement. * Therefore, when we count the number of running statements, we need to exclude errors and diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala index 73884cee32ade..f2af90f39445d 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala @@ -19,19 +19,18 @@ package org.apache.spark.sql.hive.thriftserver import java.util.Properties -import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} +import org.mockito.Mockito.{mock, RETURNS_SMART_NULLS} import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.internal.config.Status.ASYNC_TRACKING_ENABLED import org.apache.spark.scheduler.SparkListenerJobStart -import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2AppStatusStore +import org.apache.spark.sql.hive.thriftserver.ui.{HiveThriftServer2AppStatusStore, SessionInfo} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.status.ElementTrackingStore import org.apache.spark.util.kvstore.InMemoryStore -class HiveThriftServer2ListenerSuite extends SparkFunSuite - with BeforeAndAfter { +class HiveThriftServer2ListenerSuite extends SparkFunSuite with BeforeAndAfter { private var kvstore: ElementTrackingStore = _ @@ -48,20 +47,21 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite properties } - private def createStatusStore(): HiveThriftServer2AppStatusStore = { - kvstore = new ElementTrackingStore(new InMemoryStore, new SparkConf()) + private def createStatusStore: HiveThriftServer2AppStatusStore = { + val sparkConf = new SparkConf() + sparkConf.set(ASYNC_TRACKING_ENABLED, false) + kvstore = new ElementTrackingStore(new InMemoryStore, sparkConf) val server = mock(classOf[HiveThriftServer2], RETURNS_SMART_NULLS) - val sqlContext = mock(classOf[SQLContext]) val sc = mock(classOf[SparkContext]) val sqlConf = new SQLConf - when(sqlContext.conf).thenReturn(sqlConf) - when(sqlContext.sparkContext).thenReturn(sc) - val listener = new HiveThriftServer2Listener(kvstore, Some(server), Some(sqlContext), Some(sc)) + sqlConf.setConfString("spark.sql.thriftserver.ui.retainedSessions", "1") + val listener = new HiveThriftServer2Listener(kvstore, Some(server), Some(sqlConf), Some(sc)) + new HiveThriftServer2AppStatusStore(kvstore, Some(listener)) } test("listener events should store successfully") { - val statusStore = createStatusStore() + val statusStore = createStatusStore val listener = statusStore.listener.get listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionId", "user", @@ -81,6 +81,7 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite listener.onOtherEvent(SparkListenerSessionClosed("sessionId", System.currentTimeMillis())) + assert(statusStore.getOnlineSessionNum == 0) assert(statusStore.getExecutionList.size == 1) @@ -90,5 +91,54 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite assert(storeExecData.sessionId == "sessionId") assert(storeExecData.executePlan == "dummy plan") assert(storeExecData.jobId == Seq("0")) + assert(listener.noLiveData()) + } + + test("cleanup session if exceeds the threshold") { + val statusStore = createStatusStore + val listener = statusStore.listener.get + var time = 0 + listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionId1", "user", time)) + time += 1 + listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionId2", "user", time)) + + assert(statusStore.getOnlineSessionNum === 2) + assert(statusStore.getSessionCount() === 2) + + time += 1 + listener.onOtherEvent(SparkListenerSessionClosed("sessionId1", time)) + + time += 1 + listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionId3", "user", time)) + + listener.onOtherEvent(SparkListenerSessionClosed("sessionId2", time)) + + assert(statusStore.getOnlineSessionNum === 1) + assert(statusStore.getSessionCount() === 1) + assert(statusStore.getSession("sessionId1") === None) + listener.onOtherEvent(SparkListenerSessionClosed("sessionId3", 4)) + assert(listener.noLiveData()) + } + + test("update execution info when jobstart event come after execution end event") { + val statusStore = createStatusStore + val listener = statusStore.listener.get + listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionId", "user", + System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerStatementStart("id", "sessionId", "dummy query", + "groupId", System.currentTimeMillis(), "user")) + listener.onOtherEvent(SparkListenerStatementParsed("id", "dummy plan")) + listener.onOtherEvent(SparkListenerStatementFinish("id", System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerOperationClosed("id", System.currentTimeMillis())) + listener.onJobStart(SparkListenerJobStart( + 0, + System.currentTimeMillis(), + Nil, + createProperties)) + listener.onOtherEvent(SparkListenerSessionClosed("sessionId", System.currentTimeMillis())) + val exec = statusStore.getExecution("id") + assert(exec.isDefined) + assert(exec.get.jobId.toSeq === Seq("0")) + assert(listener.noLiveData()) } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala index 9c70b3b548d3a..5edf8f125b01a 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala @@ -49,13 +49,10 @@ class ThriftServerPageSuite extends SparkFunSuite with BeforeAndAfter { private def getStatusStore: HiveThriftServer2AppStatusStore = { kvstore = new ElementTrackingStore(new InMemoryStore, new SparkConf()) val server = mock(classOf[HiveThriftServer2], RETURNS_SMART_NULLS) - val sqlContext = mock(classOf[SQLContext]) val sc = mock(classOf[SparkContext]) val sqlConf = new SQLConf - when(sqlContext.conf).thenReturn(sqlConf) - when(sqlContext.sparkContext).thenReturn(sc) - val listener = new HiveThriftServer2Listener(kvstore, Some(server), Some(sqlContext), Some(sc)) + val listener = new HiveThriftServer2Listener(kvstore, Some(server), Some(sqlConf), Some(sc)) val statusStore = new HiveThriftServer2AppStatusStore(kvstore, Some(listener)) listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionid", "user", From a87c851517c4e8ccaa2aeb969d1cb7f79518a583 Mon Sep 17 00:00:00 2001 From: shahid Date: Mon, 11 Nov 2019 15:58:16 +0530 Subject: [PATCH 10/24] update --- .../hive/thriftserver/HiveThriftServer2.scala | 12 +- .../HiveThriftServer2Listener.scala | 33 ++-- .../ui/HiveThriftServer2AppStatusStore.scala | 13 +- .../HiveThriftServer2ListenerSuite.scala | 176 ++++++++++-------- .../ui/ThriftServerPageSuite.scala | 1 - 5 files changed, 123 insertions(+), 112 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index e190a95d288de..d7a017616ee69 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -25,7 +25,6 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hive.service.cli.thrift.{ThriftBinaryCLIService, ThriftHttpCLIService} import org.apache.hive.service.server.HiveServer2 -import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging import org.apache.spark.internal.config.UI.UI_ENABLED @@ -34,7 +33,6 @@ import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ import org.apache.spark.sql.hive.thriftserver.ui.{HiveThriftServer2AppStatusStore, ThriftServerTab} import org.apache.spark.status.ElementTrackingStore -import org.apache.spark.ui.SparkUI import org.apache.spark.util.{ShutdownHookManager, Utils} /** @@ -67,19 +65,13 @@ object HiveThriftServer2 extends Logging { liveListenerBus.addToStatusQueue(listener) uiTab = if (sc.getConf.get(UI_ENABLED)) { Some(new ThriftServerTab(new HiveThriftServer2AppStatusStore(kvStore, Some(listener)), - getSparkUI(sqlContext.sparkContext))) + ThriftServerTab.getSparkUI(sc))) } else { None } server } - def getSparkUI(sparkContext: SparkContext): SparkUI = { - sparkContext.ui.getOrElse { - throw new SparkException("Parent SparkUI to attach this tab to not found!") - } - } - def main(args: Array[String]): Unit = { // If the arguments contains "-h" or "--help", print out the usage and exit. if (args.contains("-h") || args.contains("--help")) { @@ -117,7 +109,7 @@ object HiveThriftServer2 extends Logging { liveListenerBus.addToStatusQueue(listener) uiTab = if (sc.getConf.get(UI_ENABLED)) { Some(new ThriftServerTab(new HiveThriftServer2AppStatusStore(kvStore, Some(listener)), - getSparkUI(sc))) + ThriftServerTab.getSparkUI(sc))) } else { None } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala index 6a1bc3fc26489..3ea0d6dacd9ea 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala @@ -105,7 +105,7 @@ private[thriftserver] class HiveThriftServer2Listener( } /** - * This method is to handle out of order events. ie. Job event come after execution end event. + * This method is to handle out of order events. ie. if Job event come after execution end event. * @param jobId * @param groupId */ @@ -118,16 +118,14 @@ private[thriftserver] class HiveThriftServer2Listener( updateLiveStore(exec) } } else { - // This will happen only if JobStart event comes after Execution End event. This scenario may - // happen only for live listeners, as history listners removes the executionList at the end of - // replay. + // Here will come only if JobStart event comes after Execution End event. val storeExecInfo = kvstore.view(classOf[ExecutionInfo]).asScala.filter(_.groupId == groupId) storeExecInfo.foreach { exec => val liveExec = getOrCreateExecution(exec.execId, exec.statement, exec.sessionId, exec.startTimestamp, exec.userName) liveExec.jobId += jobId.toString liveExec.groupId = groupId - updateLiveStore(liveExec) + updateLiveStore(liveExec, true) executionList.remove(liveExec.execId) } } @@ -150,16 +148,15 @@ private[thriftserver] class HiveThriftServer2Listener( private def processEventSessionCreated(e: SparkListenerSessionCreated): Unit = { val session = getOrCreateSession(e.sessionId, e.startTime, e.ip, e.userName) sessionList.put(e.sessionId, session) - updateLiveStore(session) + updateLiveStore(session, true) } private def processEventSessionClosed(e: SparkListenerSessionClosed): Unit = { val session = sessionList.get(e.sessionId) session.finishTimestamp = e.finishTime - updateLiveStore(session) - if (live) { - sessionList.remove(e.sessionId) - } + updateLiveStore(session, true) + sessionList.remove(e.sessionId) + } private def processEventStatementStart(e: SparkListenerStatementStart): Unit = { @@ -175,7 +172,7 @@ private[thriftserver] class HiveThriftServer2Listener( sessionList.get(e.sessionId).totalExecution += 1 executionList.get(e.id).groupId = e.groupId updateLiveStore(executionList.get(e.id)) - updateLiveStore(sessionList.get(e.sessionId)) + updateLiveStore(sessionList.get(e.sessionId), true) } private def processEventStatementParsed(e: SparkListenerStatementParsed): Unit = { @@ -206,10 +203,8 @@ private[thriftserver] class HiveThriftServer2Listener( private def processEventOperationClosed(e: SparkListenerOperationClosed): Unit = { executionList.get(e.id).closeTimestamp = e.closeTime executionList.get(e.id).state = ExecutionState.CLOSED - updateLiveStore(executionList.get(e.id)) - if (live) { - executionList.remove(e.id) - } + updateLiveStore(executionList.get(e.id), true) + executionList.remove(e.id) } @@ -265,8 +260,8 @@ private[thriftserver] class HiveThriftServer2Listener( entity.write(kvstore, now) } - def updateLiveStore(session: LiveEntity): Unit = { - if (live) { + def updateLiveStore(session: LiveEntity, force: Boolean = false): Unit = { + if (live || force == true) { session.write(kvstore, System.nanoTime()) } } @@ -293,7 +288,7 @@ private[thriftserver] class HiveThriftServer2Listener( if (countToDelete <= 0L) { return } - val view = kvstore.view(classOf[ExecutionInfo]) + val view = kvstore.view(classOf[ExecutionInfo]).index("finishTime").first(0L) val toDelete = KVUtils.viewToSeq(view, countToDelete.toInt) { j => j.finishTimestamp != 0 } @@ -305,7 +300,7 @@ private[thriftserver] class HiveThriftServer2Listener( if (countToDelete <= 0L) { return } - val view = kvstore.view(classOf[SessionInfo]) + val view = kvstore.view(classOf[SessionInfo]).index("finishTime").first(0L) val toDelete = KVUtils.viewToSeq(view, countToDelete.toInt) { j => j.finishTimestamp != 0L } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala index d75f1eef38c11..1278d502a1274 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala @@ -17,13 +17,14 @@ package org.apache.spark.sql.hive.thriftserver.ui +import com.fasterxml.jackson.annotation.JsonIgnore import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.ExecutionState import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2Listener import org.apache.spark.status.KVUtils.KVIndexParam -import org.apache.spark.util.kvstore.KVStore +import org.apache.spark.util.kvstore.{KVIndex, KVStore} /** * Provides a view of a KVStore with methods that make it easy to query SQL-specific state. There's @@ -76,11 +77,11 @@ class HiveThriftServer2AppStatusStore( execInfo.state == ExecutionState.CLOSED) } - def getSessionCount(): Long = { + def getSessionCount: Long = { store.count(classOf[SessionInfo]) } - def getExecutionCount(): Long = { + def getExecutionCount: Long = { store.count(classOf[ExecutionInfo]) } } @@ -92,7 +93,8 @@ private[thriftserver] class SessionInfo( val userName: String, val finishTimestamp: Long, val totalExecution: Long) { - + @JsonIgnore @KVIndex("finishTime") + private def finishTimeIndex: Long = if (finishTimestamp > 0L ) finishTimestamp else -1L def totalTime: Long = { if (finishTimestamp == 0L) { System.currentTimeMillis - startTimestamp @@ -115,7 +117,8 @@ private[thriftserver] class ExecutionInfo( val state: ExecutionState.Value, val jobId: ArrayBuffer[String], val groupId: String) { - + @JsonIgnore @KVIndex("finishTime") + private def finishTimeIndex: Long = if (finishTimestamp > 0L ) finishTimestamp else -1L def totalTime(endTime: Long): Long = { if (endTime == 0L) { System.currentTimeMillis - startTimestamp diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala index f2af90f39445d..9b61d2926f91a 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala @@ -25,7 +25,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.internal.config.Status.ASYNC_TRACKING_ENABLED import org.apache.spark.scheduler.SparkListenerJobStart -import org.apache.spark.sql.hive.thriftserver.ui.{HiveThriftServer2AppStatusStore, SessionInfo} +import org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2AppStatusStore import org.apache.spark.sql.internal.SQLConf import org.apache.spark.status.ElementTrackingStore import org.apache.spark.util.kvstore.InMemoryStore @@ -41,104 +41,126 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite with BeforeAndAfter { } } - private def createProperties: Properties = { - val properties = new Properties() - properties.setProperty(SparkContext.SPARK_JOB_GROUP_ID, "groupId") - properties + Seq(true, false).foreach { live => + test(s"listener events should store successfully (live = $live") { + val (statusStore: HiveThriftServer2AppStatusStore, + listener: HiveThriftServer2Listener) = if (live) { + createLiveStatusStore + } else { + createHistoryStatusStore + } + + listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionId", "user", + System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerStatementStart("id", "sessionId", "dummy query", + "groupId", System.currentTimeMillis(), "user")) + listener.onOtherEvent(SparkListenerStatementParsed("id", "dummy plan")) + listener.onJobStart(SparkListenerJobStart( + 0, + System.currentTimeMillis(), + Nil, + createProperties)) + listener.onOtherEvent(SparkListenerStatementFinish("id", System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerOperationClosed("id", System.currentTimeMillis())) + + assert(statusStore.getOnlineSessionNum === 1) + + listener.onOtherEvent(SparkListenerSessionClosed("sessionId", System.currentTimeMillis())) + + assert(statusStore.getOnlineSessionNum === 0) + assert(statusStore.getExecutionList.size === 1) + + val storeExecData = statusStore.getExecutionList.head + + assert(storeExecData.execId === "id") + assert(storeExecData.sessionId === "sessionId") + assert(storeExecData.executePlan === "dummy plan") + assert(storeExecData.jobId === Seq("0")) + assert(listener.noLiveData()) + } } - private def createStatusStore: HiveThriftServer2AppStatusStore = { - val sparkConf = new SparkConf() - sparkConf.set(ASYNC_TRACKING_ENABLED, false) - kvstore = new ElementTrackingStore(new InMemoryStore, sparkConf) - val server = mock(classOf[HiveThriftServer2], RETURNS_SMART_NULLS) - val sc = mock(classOf[SparkContext]) - val sqlConf = new SQLConf - sqlConf.setConfString("spark.sql.thriftserver.ui.retainedSessions", "1") - val listener = new HiveThriftServer2Listener(kvstore, Some(server), Some(sqlConf), Some(sc)) - - new HiveThriftServer2AppStatusStore(kvstore, Some(listener)) + Seq(true, false).foreach { live => + test(s"cleanup session if exceeds the threshold (live = $live)") { + val (statusStore: HiveThriftServer2AppStatusStore, + listener: HiveThriftServer2Listener) = if (live) { + createLiveStatusStore + } else { + createHistoryStatusStore + } + var time = 0 + listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionId1", "user", time)) + time += 1 + listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionId2", "user", time)) + + assert(statusStore.getOnlineSessionNum === 2) + assert(statusStore.getSessionCount === 2) + + time += 1 + listener.onOtherEvent(SparkListenerSessionClosed("sessionId1", time)) + + time += 1 + listener.onOtherEvent(SparkListenerSessionClosed("sessionId2", time)) + + listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionId3", "user", time)) + assert(statusStore.getOnlineSessionNum === 1) + assert(statusStore.getSessionCount === 1) + assert(statusStore.getSession("sessionId1") === None) + time += 1 + listener.onOtherEvent(SparkListenerSessionClosed("sessionId3", 4)) + + assert(listener.noLiveData()) + } } - test("listener events should store successfully") { - val statusStore = createStatusStore - val listener = statusStore.listener.get + test("update execution info when jobstart event come after execution end event") { + val (statusStore: HiveThriftServer2AppStatusStore, + listener: HiveThriftServer2Listener) = createLiveStatusStore listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionId", "user", System.currentTimeMillis())) listener.onOtherEvent(SparkListenerStatementStart("id", "sessionId", "dummy query", "groupId", System.currentTimeMillis(), "user")) listener.onOtherEvent(SparkListenerStatementParsed("id", "dummy plan")) + listener.onOtherEvent(SparkListenerStatementFinish("id", System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerOperationClosed("id", System.currentTimeMillis())) listener.onJobStart(SparkListenerJobStart( 0, System.currentTimeMillis(), Nil, createProperties)) - listener.onOtherEvent(SparkListenerStatementFinish("id", System.currentTimeMillis())) - listener.onOtherEvent(SparkListenerOperationClosed("id", System.currentTimeMillis())) - - assert(statusStore.getOnlineSessionNum == 1) - listener.onOtherEvent(SparkListenerSessionClosed("sessionId", System.currentTimeMillis())) - - - assert(statusStore.getOnlineSessionNum == 0) - assert(statusStore.getExecutionList.size == 1) - - val storeExecData = statusStore.getExecutionList.head - - assert(storeExecData.execId == "id") - assert(storeExecData.sessionId == "sessionId") - assert(storeExecData.executePlan == "dummy plan") - assert(storeExecData.jobId == Seq("0")) + val exec = statusStore.getExecution("id") + assert(exec.isDefined) + assert(exec.get.jobId.toSeq === Seq("0")) assert(listener.noLiveData()) } - test("cleanup session if exceeds the threshold") { - val statusStore = createStatusStore - val listener = statusStore.listener.get - var time = 0 - listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionId1", "user", time)) - time += 1 - listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionId2", "user", time)) - - assert(statusStore.getOnlineSessionNum === 2) - assert(statusStore.getSessionCount() === 2) - - time += 1 - listener.onOtherEvent(SparkListenerSessionClosed("sessionId1", time)) - - time += 1 - listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionId3", "user", time)) + private def createProperties: Properties = { + val properties = new Properties() + properties.setProperty(SparkContext.SPARK_JOB_GROUP_ID, "groupId") + properties + } - listener.onOtherEvent(SparkListenerSessionClosed("sessionId2", time)) + private def createLiveStatusStore = { + val sparkConf = new SparkConf() + sparkConf.set(ASYNC_TRACKING_ENABLED, false) + kvstore = new ElementTrackingStore(new InMemoryStore, sparkConf) + val server = mock(classOf[HiveThriftServer2], RETURNS_SMART_NULLS) + val sc = mock(classOf[SparkContext]) + val sqlConf = new SQLConf + sqlConf.setConfString(SQLConf.THRIFTSERVER_UI_SESSION_LIMIT.key, "1") + val listener = new HiveThriftServer2Listener(kvstore, Some(server), Some(sqlConf), Some(sc)) - assert(statusStore.getOnlineSessionNum === 1) - assert(statusStore.getSessionCount() === 1) - assert(statusStore.getSession("sessionId1") === None) - listener.onOtherEvent(SparkListenerSessionClosed("sessionId3", 4)) - assert(listener.noLiveData()) + (new HiveThriftServer2AppStatusStore(kvstore, Some(listener)), listener) } - test("update execution info when jobstart event come after execution end event") { - val statusStore = createStatusStore - val listener = statusStore.listener.get - listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionId", "user", - System.currentTimeMillis())) - listener.onOtherEvent(SparkListenerStatementStart("id", "sessionId", "dummy query", - "groupId", System.currentTimeMillis(), "user")) - listener.onOtherEvent(SparkListenerStatementParsed("id", "dummy plan")) - listener.onOtherEvent(SparkListenerStatementFinish("id", System.currentTimeMillis())) - listener.onOtherEvent(SparkListenerOperationClosed("id", System.currentTimeMillis())) - listener.onJobStart(SparkListenerJobStart( - 0, - System.currentTimeMillis(), - Nil, - createProperties)) - listener.onOtherEvent(SparkListenerSessionClosed("sessionId", System.currentTimeMillis())) - val exec = statusStore.getExecution("id") - assert(exec.isDefined) - assert(exec.get.jobId.toSeq === Seq("0")) - assert(listener.noLiveData()) + private def createHistoryStatusStore = { + val sparkConf = new SparkConf() + sparkConf.set(ASYNC_TRACKING_ENABLED, false) + kvstore = new ElementTrackingStore(new InMemoryStore, sparkConf) + sparkConf.set(SQLConf.THRIFTSERVER_UI_SESSION_LIMIT.key, "1") + (new HiveThriftServer2AppStatusStore(kvstore), + new HiveThriftServer2Listener(kvstore, None, None, None, Some(sparkConf), false)) } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala index 5edf8f125b01a..b0bca64268380 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala @@ -25,7 +25,6 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.scheduler.SparkListenerJobStart -import org.apache.spark.sql.SQLContext import org.apache.spark.sql.hive.thriftserver._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.status.ElementTrackingStore From da2ae4c941096acd544ef89d21db3006dfd56115 Mon Sep 17 00:00:00 2001 From: shahid Date: Mon, 11 Nov 2019 16:41:44 +0530 Subject: [PATCH 11/24] compile fix --- .../thriftserver/HiveThriftServer2HistoryServerPlugin.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2HistoryServerPlugin.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2HistoryServerPlugin.scala index 1dcb3ee4f2f8a..26bc52965b142 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2HistoryServerPlugin.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2HistoryServerPlugin.scala @@ -31,7 +31,7 @@ class HiveThriftServer2HistoryServerPlugin extends AppHistoryServerPlugin { override def setupUI(ui: SparkUI): Unit = { val store = new HiveThriftServer2AppStatusStore(ui.store.store) - if (store.getSessionCount() > 0) { + if (store.getSessionCount > 0) { new ThriftServerTab(store, ui) } } From 0b48cc7da12d0e3c5caca4b319a2220e3eed68d2 Mon Sep 17 00:00:00 2001 From: shahid Date: Sun, 24 Nov 2019 02:09:54 +0530 Subject: [PATCH 12/24] This commit has following changes, 1. Seperated posting events from listener bus to a seperate class eventmanager 2. Moved all the listener/store class to ui directory 3. Address comments --- .../deploy/history/FsHistoryProvider.scala | 26 ++- ...apache.spark.status.AppHistoryServerPlugin | 2 +- .../hive/thriftserver/HiveThriftServer2.scala | 31 ++-- .../SparkExecuteStatementOperation.scala | 18 +-- .../SparkGetCatalogsOperation.scala | 10 +- .../SparkGetColumnsOperation.scala | 10 +- .../SparkGetFunctionsOperation.scala | 10 +- .../SparkGetSchemasOperation.scala | 10 +- .../SparkGetTableTypesOperation.scala | 10 +- .../SparkGetTablesOperation.scala | 10 +- .../SparkGetTypeInfoOperation.scala | 10 +- .../thriftserver/SparkSQLSessionManager.scala | 4 +- .../ui/HiveThriftServer2AppStatusStore.scala | 1 - .../ui/HiveThriftServer2EventManager.scala | 110 +++++++++++++ ...HiveThriftServer2HistoryServerPlugin.scala | 5 +- .../{ => ui}/HiveThriftServer2Listener.scala | 150 +++--------------- .../ui/ThriftServerSessionPage.scala | 1 - .../thriftserver/ui/ThriftServerTab.scala | 1 - .../HiveThriftServer2ListenerSuite.scala | 57 +++---- .../ui/ThriftServerPageSuite.scala | 14 +- 20 files changed, 242 insertions(+), 248 deletions(-) create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2EventManager.scala rename sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/{ => ui}/HiveThriftServer2HistoryServerPlugin.scala (85%) rename sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/{ => ui}/HiveThriftServer2Listener.scala (65%) rename sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/{ => ui}/HiveThriftServer2ListenerSuite.scala (75%) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index ba9ec21f87241..bcd182461a075 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -350,11 +350,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) val ui = SparkUI.create(None, new HistoryAppStatusStore(conf, kvstore), conf, secManager, app.info.name, HistoryServer.getAttemptURI(appId, attempt.info.attemptId), attempt.info.startTime.getTime(), attempt.info.appSparkVersion) - // To make SQL tab first, then JDBC/ODBC tab. - loadPlugins().toSeq.reverse.foreach(_.setupUI(ui)) - + setupPluginUI(ui) val loadedUI = LoadedAppUI(ui) - synchronized { activeUIs((appId, attemptId)) = loadedUI } @@ -362,6 +359,27 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) Some(loadedUI) } + private def setupPluginUI(ui: SparkUI): Unit = { + val plugins = loadPlugins().toSeq + + var sqlTab: Option[AppHistoryServerPlugin] = None + var jdbcTab: Option[AppHistoryServerPlugin] = None + val otherTabs: Seq[AppHistoryServerPlugin] = Seq() + + plugins.foreach{ plugin => + if (plugin.toString.contains("SQL")) { + sqlTab = Some(plugin) + } else if (plugin.toString.contains("ThriftServer")) { + jdbcTab = Some(plugin) + } else { + otherTabs :+ plugin + } + } + sqlTab.foreach(_.setupUI(ui)) + jdbcTab.foreach(_.setupUI(ui)) + otherTabs.foreach(_.setupUI(ui)) + } + override def getEmptyListingHtml(): Seq[Node] = {

Did you specify the correct logging directory? Please verify your setting of diff --git a/sql/hive-thriftserver/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin b/sql/hive-thriftserver/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin index c301debfb5206..96d990372ee4c 100644 --- a/sql/hive-thriftserver/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin +++ b/sql/hive-thriftserver/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin @@ -1 +1 @@ -org.apache.spark.sql.hive.thriftserver.HiveThriftServer2HistoryServerPlugin +org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2HistoryServerPlugin diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index d7a017616ee69..f15193b0dc3cc 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -25,13 +25,14 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hive.service.cli.thrift.{ThriftBinaryCLIService, ThriftHttpCLIService} import org.apache.hive.service.server.HiveServer2 +import org.apache.spark.SparkContext import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql.SQLContext import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ -import org.apache.spark.sql.hive.thriftserver.ui.{HiveThriftServer2AppStatusStore, ThriftServerTab} +import org.apache.spark.sql.hive.thriftserver.ui._ import org.apache.spark.status.ElementTrackingStore import org.apache.spark.util.{ShutdownHookManager, Utils} @@ -42,6 +43,7 @@ import org.apache.spark.util.{ShutdownHookManager, Utils} object HiveThriftServer2 extends Logging { var uiTab: Option[ThriftServerTab] = None var listener: HiveThriftServer2Listener = _ + var eventManager: HiveThriftServer2EventManager = _ /** * :: DeveloperApi :: @@ -57,19 +59,21 @@ object HiveThriftServer2 extends Logging { server.init(executionHive.conf) server.start() - val sc = sqlContext.sparkContext + createListenerAndUI(server, sqlContext.sparkContext) + server + } + + private def createListenerAndUI(server: HiveThriftServer2, sc: SparkContext): Unit = { val kvStore = sc.statusStore.store.asInstanceOf[ElementTrackingStore] - val sqlConf = sqlContext.conf - listener = new HiveThriftServer2Listener(kvStore, Some(server), Some(sqlConf), Some(sc)) - val liveListenerBus = sc.listenerBus - liveListenerBus.addToStatusQueue(listener) + eventManager = new HiveThriftServer2EventManager(sc) + listener = new HiveThriftServer2Listener(kvStore, sc.conf, Some(server)) + sc.listenerBus.addToStatusQueue(listener) uiTab = if (sc.getConf.get(UI_ENABLED)) { Some(new ThriftServerTab(new HiveThriftServer2AppStatusStore(kvStore, Some(listener)), ThriftServerTab.getSparkUI(sc))) } else { None } - server } def main(args: Array[String]): Unit = { @@ -101,18 +105,7 @@ object HiveThriftServer2 extends Logging { server.init(executionHive.conf) server.start() logInfo("HiveThriftServer2 started") - val sc = SparkSQLEnv.sparkContext - val kvStore = sc.statusStore.store.asInstanceOf[ElementTrackingStore] - val sqlConf = SparkSQLEnv.sqlContext.conf - listener = new HiveThriftServer2Listener(kvStore, Some(server), Some(sqlConf), Some(sc)) - val liveListenerBus = sc.listenerBus - liveListenerBus.addToStatusQueue(listener) - uiTab = if (sc.getConf.get(UI_ENABLED)) { - Some(new ThriftServerTab(new HiveThriftServer2AppStatusStore(kvStore, Some(listener)), - ThriftServerTab.getSparkUI(sc))) - } else { - None - } + createListenerAndUI(server, SparkSQLEnv.sparkContext) // If application was killed before HiveThriftServer2 start successfully then SparkSubmit // process can not exit, so check whether if SparkContext was stopped. if (SparkSQLEnv.sparkContext.stopped.get()) { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index 68197a9de8566..875c4d2ad8b5c 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -77,7 +77,7 @@ private[hive] class SparkExecuteStatementOperation( // RDDs will be cleaned automatically upon garbage collection. logInfo(s"Close statement with $statementId") cleanup(OperationState.CLOSED) - HiveThriftServer2.listener.onOperationClosed(statementId) + HiveThriftServer2.eventManager.onOperationClosed(statementId) } def addNonNullColumnValue(from: SparkRow, to: ArrayBuffer[Any], ordinal: Int): Unit = { @@ -195,7 +195,7 @@ private[hive] class SparkExecuteStatementOperation( setState(OperationState.PENDING) statementId = UUID.randomUUID().toString logInfo(s"Submitting query '$statement' with $statementId") - HiveThriftServer2.listener.onStatementStart( + HiveThriftServer2.eventManager.onStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, statement, @@ -245,14 +245,14 @@ private[hive] class SparkExecuteStatementOperation( case rejected: RejectedExecutionException => logError("Error submitting query in background, query rejected", rejected) setState(OperationState.ERROR) - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, rejected.getMessage, SparkUtils.exceptionString(rejected)) throw new HiveSQLException("The background threadpool cannot accept" + " new task for execution, please retry the operation", rejected) case NonFatal(e) => logError(s"Error executing query in background", e) setState(OperationState.ERROR) - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, e.getMessage, SparkUtils.exceptionString(e)) throw new HiveSQLException(e) } @@ -284,7 +284,7 @@ private[hive] class SparkExecuteStatementOperation( "in this session.") case _ => } - HiveThriftServer2.listener.onStatementParsed(statementId, result.queryExecution.toString()) + HiveThriftServer2.eventManager.onStatementParsed(statementId, result.queryExecution.toString()) iter = { if (sqlContext.getConf(SQLConf.THRIFTSERVER_INCREMENTAL_COLLECT.key).toBoolean) { resultList = None @@ -315,12 +315,12 @@ private[hive] class SparkExecuteStatementOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, root.getMessage, SparkUtils.exceptionString(root)) throw new HiveSQLException("Error running query: " + root.toString, root) } @@ -329,7 +329,7 @@ private[hive] class SparkExecuteStatementOperation( synchronized { if (!getStatus.getState.isTerminal) { setState(OperationState.FINISHED) - HiveThriftServer2.listener.onStatementFinish(statementId) + HiveThriftServer2.eventManager.onStatementFinish(statementId) } } sqlContext.sparkContext.clearJobGroup() @@ -341,7 +341,7 @@ private[hive] class SparkExecuteStatementOperation( if (!getStatus.getState.isTerminal) { logInfo(s"Cancel query with $statementId") cleanup(OperationState.CANCELED) - HiveThriftServer2.listener.onStatementCanceled(statementId) + HiveThriftServer2.eventManager.onStatementCanceled(statementId) } } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala index 6c8a5b00992da..2945cfd200e46 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala @@ -44,7 +44,7 @@ private[hive] class SparkGetCatalogsOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.onOperationClosed(statementId) + HiveThriftServer2.eventManager.onOperationClosed(statementId) } override def runInternal(): Unit = { @@ -56,7 +56,7 @@ private[hive] class SparkGetCatalogsOperation( val executionHiveClassLoader = sqlContext.sharedState.jarClassLoader Thread.currentThread().setContextClassLoader(executionHiveClassLoader) - HiveThriftServer2.listener.onStatementStart( + HiveThriftServer2.eventManager.onStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, @@ -74,16 +74,16 @@ private[hive] class SparkGetCatalogsOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, root.getMessage, SparkUtils.exceptionString(root)) throw new HiveSQLException("Error getting catalogs: " + root.toString, root) } } - HiveThriftServer2.listener.onStatementFinish(statementId) + HiveThriftServer2.eventManager.onStatementFinish(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala index f845a2285b9a3..ff7cbfeae13be 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala @@ -63,7 +63,7 @@ private[hive] class SparkGetColumnsOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.onOperationClosed(statementId) + HiveThriftServer2.eventManager.onOperationClosed(statementId) } override def runInternal(): Unit = { @@ -78,7 +78,7 @@ private[hive] class SparkGetColumnsOperation( val executionHiveClassLoader = sqlContext.sharedState.jarClassLoader Thread.currentThread().setContextClassLoader(executionHiveClassLoader) - HiveThriftServer2.listener.onStatementStart( + HiveThriftServer2.eventManager.onStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, @@ -135,17 +135,17 @@ private[hive] class SparkGetColumnsOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, root.getMessage, SparkUtils.exceptionString(root)) throw new HiveSQLException("Error getting columns: " + root.toString, root) } } - HiveThriftServer2.listener.onStatementFinish(statementId) + HiveThriftServer2.eventManager.onStatementFinish(statementId) } private def addToRowSet( diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala index 1cdd8918421bb..d9c12b6ca9e64 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala @@ -54,7 +54,7 @@ private[hive] class SparkGetFunctionsOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.onOperationClosed(statementId) + HiveThriftServer2.eventManager.onOperationClosed(statementId) } override def runInternal(): Unit = { @@ -81,7 +81,7 @@ private[hive] class SparkGetFunctionsOperation( authorizeMetaGets(HiveOperationType.GET_FUNCTIONS, privObjs, cmdStr) } - HiveThriftServer2.listener.onStatementStart( + HiveThriftServer2.eventManager.onStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, @@ -110,16 +110,16 @@ private[hive] class SparkGetFunctionsOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, root.getMessage, SparkUtils.exceptionString(root)) throw new HiveSQLException("Error getting functions: " + root.toString, root) } } - HiveThriftServer2.listener.onStatementFinish(statementId) + HiveThriftServer2.eventManager.onStatementFinish(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala index 928610a6bcff9..db19880d1b99f 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala @@ -50,7 +50,7 @@ private[hive] class SparkGetSchemasOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.onOperationClosed(statementId) + HiveThriftServer2.eventManager.onOperationClosed(statementId) } override def runInternal(): Unit = { @@ -68,7 +68,7 @@ private[hive] class SparkGetSchemasOperation( authorizeMetaGets(HiveOperationType.GET_TABLES, null, cmdStr) } - HiveThriftServer2.listener.onStatementStart( + HiveThriftServer2.eventManager.onStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, @@ -93,16 +93,16 @@ private[hive] class SparkGetSchemasOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, root.getMessage, SparkUtils.exceptionString(root)) throw new HiveSQLException("Error getting schemas: " + root.toString, root) } } - HiveThriftServer2.listener.onStatementFinish(statementId) + HiveThriftServer2.eventManager.onStatementFinish(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala index ec03f1e148e69..b4093e58d3c07 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala @@ -45,7 +45,7 @@ private[hive] class SparkGetTableTypesOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.onOperationClosed(statementId) + HiveThriftServer2.eventManager.onOperationClosed(statementId) } override def runInternal(): Unit = { @@ -61,7 +61,7 @@ private[hive] class SparkGetTableTypesOperation( authorizeMetaGets(HiveOperationType.GET_TABLETYPES, null) } - HiveThriftServer2.listener.onStatementStart( + HiveThriftServer2.eventManager.onStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, @@ -80,16 +80,16 @@ private[hive] class SparkGetTableTypesOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, root.getMessage, SparkUtils.exceptionString(root)) throw new HiveSQLException("Error getting table types: " + root.toString, root) } } - HiveThriftServer2.listener.onStatementFinish(statementId) + HiveThriftServer2.eventManager.onStatementFinish(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala index bf9cf7ad46d95..45c6d980aac47 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala @@ -59,7 +59,7 @@ private[hive] class SparkGetTablesOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.onOperationClosed(statementId) + HiveThriftServer2.eventManager.onOperationClosed(statementId) } override def runInternal(): Unit = { @@ -85,7 +85,7 @@ private[hive] class SparkGetTablesOperation( authorizeMetaGets(HiveOperationType.GET_TABLES, privObjs, cmdStr) } - HiveThriftServer2.listener.onStatementStart( + HiveThriftServer2.eventManager.onStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, @@ -124,17 +124,17 @@ private[hive] class SparkGetTablesOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, root.getMessage, SparkUtils.exceptionString(root)) throw new HiveSQLException("Error getting tables: " + root.toString, root) } } - HiveThriftServer2.listener.onStatementFinish(statementId) + HiveThriftServer2.eventManager.onStatementFinish(statementId) } private def addToRowSet( diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala index 0d263b09d57d3..dd5668a93f82d 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala @@ -44,7 +44,7 @@ private[hive] class SparkGetTypeInfoOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.onOperationClosed(statementId) + HiveThriftServer2.eventManager.onOperationClosed(statementId) } override def runInternal(): Unit = { @@ -60,7 +60,7 @@ private[hive] class SparkGetTypeInfoOperation( authorizeMetaGets(HiveOperationType.GET_TYPEINFO, null) } - HiveThriftServer2.listener.onStatementStart( + HiveThriftServer2.eventManager.onStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, @@ -98,16 +98,16 @@ private[hive] class SparkGetTypeInfoOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, root.getMessage, SparkUtils.exceptionString(root)) throw new HiveSQLException("Error getting type info: " + root.toString, root) } } - HiveThriftServer2.listener.onStatementFinish(statementId) + HiveThriftServer2.eventManager.onStatementFinish(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala index c4248bfde38cc..3a91f8214aa4d 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala @@ -55,7 +55,7 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, sqlContext: super.openSession(protocol, username, passwd, ipAddress, sessionConf, withImpersonation, delegationToken) val session = super.getSession(sessionHandle) - HiveThriftServer2.listener.onSessionCreated( + HiveThriftServer2.eventManager.onSessionCreated( session.getIpAddress, sessionHandle.getSessionId.toString, session.getUsername) val ctx = if (sqlContext.conf.hiveThriftServerSingleSession) { sqlContext @@ -74,7 +74,7 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, sqlContext: } override def closeSession(sessionHandle: SessionHandle): Unit = { - HiveThriftServer2.listener.onSessionClosed(sessionHandle.getSessionId.toString) + HiveThriftServer2.eventManager.onSessionClosed(sessionHandle.getSessionId.toString) super.closeSession(sessionHandle) sparkSqlOperationManager.sessionToActivePool.remove(sessionHandle) sparkSqlOperationManager.sessionToContexts.remove(sessionHandle) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala index 1278d502a1274..9ae8dc9af6ccf 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala @@ -22,7 +22,6 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.ExecutionState -import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2Listener import org.apache.spark.status.KVUtils.KVIndexParam import org.apache.spark.util.kvstore.{KVIndex, KVStore} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2EventManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2EventManager.scala new file mode 100644 index 0000000000000..c6024188e79e7 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2EventManager.scala @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.hive.thriftserver.ui + +import org.apache.spark.SparkContext +import org.apache.spark.scheduler.SparkListenerEvent + +/** + * This class manages events generated by the thriftserver application. It converts the + * operation and session events to listener events and post it into the live listener bus. + */ +private[thriftserver] class HiveThriftServer2EventManager(sc: SparkContext) { + + def postLiveListenerBus(event: SparkListenerEvent): Unit = { + sc.listenerBus.post(event) + } + + def onSessionCreated(ip: String, sessionId: String, userName: String = "UNKNOWN"): Unit = { + postLiveListenerBus(SparkListenerSessionCreated(ip, sessionId, + userName, System.currentTimeMillis())) + } + + def onSessionClosed(sessionId: String): Unit = { + postLiveListenerBus(SparkListenerSessionClosed(sessionId, System.currentTimeMillis())) + } + + def onStatementStart( + id: String, + sessionId: String, + statement: String, + groupId: String, + userName: String = "UNKNOWN"): Unit = { + postLiveListenerBus(SparkListenerOperationStart(id, sessionId, statement, groupId, + System.currentTimeMillis(), userName)) + } + + def onStatementParsed(id: String, executionPlan: String): Unit = { + postLiveListenerBus(SparkListenerOperationParsed(id, executionPlan)) + } + + def onStatementCanceled(id: String): Unit = { + postLiveListenerBus(SparkListenerOperationCanceled(id, System.currentTimeMillis())) + } + + def onStatementError(id: String, errorMsg: String, errorTrace: String): Unit = { + postLiveListenerBus(SparkListenerOperationError(id, errorMsg, errorTrace, + System.currentTimeMillis())) + } + + def onStatementFinish(id: String): Unit = { + postLiveListenerBus(SparkListenerOperationFinish(id, System.currentTimeMillis())) + + } + + def onOperationClosed(id: String): Unit = { + postLiveListenerBus(SparkListenerOperationClosed(id, System.currentTimeMillis())) + } +} + +private[thriftserver] case class SparkListenerSessionCreated( + ip: String, + sessionId: String, + userName: String, + startTime: Long) extends SparkListenerEvent + +private[thriftserver] case class SparkListenerSessionClosed( + sessionId: String, finishTime: Long) extends SparkListenerEvent + +private[thriftserver] case class SparkListenerOperationStart( + id: String, + sessionId: String, + statement: String, + groupId: String, + startTime: Long, + userName: String = "UNKNOWN") extends SparkListenerEvent + +private[thriftserver] case class SparkListenerOperationParsed( + id: String, + executionPlan: String) extends SparkListenerEvent + +private[thriftserver] case class SparkListenerOperationCanceled( + id: String, finishTime: Long) extends SparkListenerEvent + +private[thriftserver] case class SparkListenerOperationError( + id: String, + errorMsg: String, + errorTrace: String, + finishTime: Long) extends SparkListenerEvent + +private[thriftserver] case class SparkListenerOperationFinish(id: String, finishTime: Long) + extends SparkListenerEvent + +private[thriftserver] case class SparkListenerOperationClosed(id: String, closeTime: Long) + extends SparkListenerEvent + + diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2HistoryServerPlugin.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2HistoryServerPlugin.scala similarity index 85% rename from sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2HistoryServerPlugin.scala rename to sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2HistoryServerPlugin.scala index 26bc52965b142..75b223ddc5e86 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2HistoryServerPlugin.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2HistoryServerPlugin.scala @@ -15,18 +15,17 @@ * limitations under the License. */ -package org.apache.spark.sql.hive.thriftserver +package org.apache.spark.sql.hive.thriftserver.ui import org.apache.spark.SparkConf import org.apache.spark.scheduler.SparkListener -import org.apache.spark.sql.hive.thriftserver.ui.{HiveThriftServer2AppStatusStore, ThriftServerTab} import org.apache.spark.status.{AppHistoryServerPlugin, ElementTrackingStore} import org.apache.spark.ui.SparkUI class HiveThriftServer2HistoryServerPlugin extends AppHistoryServerPlugin { override def createListeners(conf: SparkConf, store: ElementTrackingStore): Seq[SparkListener] = { - Seq(new HiveThriftServer2Listener(store, None, None, None, Some(conf), false)) + Seq(new HiveThriftServer2Listener(store, conf, None, false)) } override def setupUI(ui: SparkUI): Unit = { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala similarity index 65% rename from sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala rename to sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala index 3ea0d6dacd9ea..010c5ac6e281d 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Listener.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala @@ -15,18 +15,18 @@ * limitations under the License. */ -package org.apache.spark.sql.hive.thriftserver +package org.apache.spark.sql.hive.thriftserver.ui import java.util.concurrent.ConcurrentHashMap -import org.apache.hive.service.server.HiveServer2 import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer +import org.apache.hive.service.server.HiveServer2 + import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.scheduler._ import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.ExecutionState -import org.apache.spark.sql.hive.thriftserver.ui.{ExecutionInfo, SessionInfo} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.status.{ElementTrackingStore, KVUtils, LiveEntity} @@ -35,25 +35,16 @@ import org.apache.spark.status.{ElementTrackingStore, KVUtils, LiveEntity} */ private[thriftserver] class HiveThriftServer2Listener( kvstore: ElementTrackingStore, + sparkConf: SparkConf, server: Option[HiveServer2], - sqlConf: Option[SQLConf], - sc: Option[SparkContext], - sparkConf: Option[SparkConf] = None, live: Boolean = true) extends SparkListener { private val sessionList = new ConcurrentHashMap[String, LiveSessionData]() private val executionList = new ConcurrentHashMap[String, LiveExecutionData]() private val (retainedStatements: Int, retainedSessions: Int) = { - if (live) { - val conf = sqlConf.get - (conf.getConf(SQLConf.THRIFTSERVER_UI_STATEMENT_LIMIT), - conf.getConf(SQLConf.THRIFTSERVER_UI_SESSION_LIMIT)) - } else { - val conf = sparkConf.get - (conf.get(SQLConf.THRIFTSERVER_UI_STATEMENT_LIMIT), - conf.get(SQLConf.THRIFTSERVER_UI_SESSION_LIMIT)) - } + (sparkConf.get(SQLConf.THRIFTSERVER_UI_STATEMENT_LIMIT), + sparkConf.get(SQLConf.THRIFTSERVER_UI_SESSION_LIMIT)) } // Returns true if this listener has no live data. Exposed for tests only. @@ -73,18 +64,6 @@ private[thriftserver] class HiveThriftServer2Listener( if (!live) { val now = System.nanoTime() flush(update(_, now)) - executionList.keys().asScala.foreach( - key => executionList.remove(key) - ) - sessionList.keys().asScala.foreach( - key => sessionList.remove(key) - ) - } - } - - def postLiveListenerBus(event: SparkListenerEvent): Unit = { - if (live) { - sc.foreach(_.listenerBus.post(event)) } } @@ -114,7 +93,6 @@ private[thriftserver] class HiveThriftServer2Listener( if (execList.nonEmpty) { execList.foreach { exec => exec.jobId += jobId.toString - exec.groupId = groupId updateLiveStore(exec) } } else { @@ -124,7 +102,6 @@ private[thriftserver] class HiveThriftServer2Listener( val liveExec = getOrCreateExecution(exec.execId, exec.statement, exec.sessionId, exec.startTimestamp, exec.userName) liveExec.jobId += jobId.toString - liveExec.groupId = groupId updateLiveStore(liveExec, true) executionList.remove(liveExec.execId) } @@ -133,33 +110,32 @@ private[thriftserver] class HiveThriftServer2Listener( override def onOtherEvent(event: SparkListenerEvent): Unit = { event match { - case e: SparkListenerSessionCreated => processEventSessionCreated(e) - case e: SparkListenerSessionClosed => processEventSessionClosed(e) - case e: SparkListenerStatementStart => processEventStatementStart(e) - case e: SparkListenerStatementParsed => processEventStatementParsed(e) - case e: SparkListenerStatementCanceled => processEventStatementCanceled(e) - case e: SparkListenerStatementError => processEventStatementError(e) - case e: SparkListenerStatementFinish => processEventStatementFinish(e) - case e: SparkListenerOperationClosed => processEventOperationClosed(e) + case e: SparkListenerSessionCreated => onSessionCreated(e) + case e: SparkListenerSessionClosed => onSessionClosed(e) + case e: SparkListenerOperationStart => onOperationStart(e) + case e: SparkListenerOperationParsed => onOperationParsed(e) + case e: SparkListenerOperationCanceled => onOperationCanceled(e) + case e: SparkListenerOperationError => onOperationError(e) + case e: SparkListenerOperationFinish => onOperationFinished(e) + case e: SparkListenerOperationClosed => onOperationClosed(e) case _ => // Ignore } } - private def processEventSessionCreated(e: SparkListenerSessionCreated): Unit = { + private def onSessionCreated(e: SparkListenerSessionCreated): Unit = { val session = getOrCreateSession(e.sessionId, e.startTime, e.ip, e.userName) sessionList.put(e.sessionId, session) updateLiveStore(session, true) } - private def processEventSessionClosed(e: SparkListenerSessionClosed): Unit = { + private def onSessionClosed(e: SparkListenerSessionClosed): Unit = { val session = sessionList.get(e.sessionId) session.finishTimestamp = e.finishTime updateLiveStore(session, true) sessionList.remove(e.sessionId) - } - private def processEventStatementStart(e: SparkListenerStatementStart): Unit = { + private def onOperationStart(e: SparkListenerOperationStart): Unit = { val info = getOrCreateExecution( e.id, e.statement, @@ -175,81 +151,38 @@ private[thriftserver] class HiveThriftServer2Listener( updateLiveStore(sessionList.get(e.sessionId), true) } - private def processEventStatementParsed(e: SparkListenerStatementParsed): Unit = { + private def onOperationParsed(e: SparkListenerOperationParsed): Unit = { executionList.get(e.id).executePlan = e.executionPlan executionList.get(e.id).state = ExecutionState.COMPILED updateLiveStore(executionList.get(e.id)) } - private def processEventStatementCanceled(e: SparkListenerStatementCanceled): Unit = { + private def onOperationCanceled(e: SparkListenerOperationCanceled): Unit = { executionList.get(e.id).finishTimestamp = e.finishTime executionList.get(e.id).state = ExecutionState.CANCELED updateLiveStore(executionList.get(e.id)) } - private def processEventStatementError(e: SparkListenerStatementError): Unit = { + private def onOperationError(e: SparkListenerOperationError): Unit = { executionList.get(e.id).finishTimestamp = e.finishTime executionList.get(e.id).detail = e.errorMsg executionList.get(e.id).state = ExecutionState.FAILED updateLiveStore(executionList.get(e.id)) } - private def processEventStatementFinish(e: SparkListenerStatementFinish): Unit = { + private def onOperationFinished(e: SparkListenerOperationFinish): Unit = { executionList.get(e.id).finishTimestamp = e.finishTime executionList.get(e.id).state = ExecutionState.FINISHED updateLiveStore(executionList.get(e.id)) } - private def processEventOperationClosed(e: SparkListenerOperationClosed): Unit = { + private def onOperationClosed(e: SparkListenerOperationClosed): Unit = { executionList.get(e.id).closeTimestamp = e.closeTime executionList.get(e.id).state = ExecutionState.CLOSED updateLiveStore(executionList.get(e.id), true) executionList.remove(e.id) } - - def onSessionCreated(ip: String, sessionId: String, userName: String = "UNKNOWN"): Unit = { - postLiveListenerBus(SparkListenerSessionCreated(ip, sessionId, - userName, System.currentTimeMillis())) - } - - def onSessionClosed(sessionId: String): Unit = { - postLiveListenerBus(SparkListenerSessionClosed(sessionId, System.currentTimeMillis())) - } - - def onStatementStart( - id: String, - sessionId: String, - statement: String, - groupId: String, - userName: String = "UNKNOWN"): Unit = { - postLiveListenerBus(SparkListenerStatementStart(id, sessionId, statement, groupId, - System.currentTimeMillis(), userName)) - } - - def onStatementParsed(id: String, executionPlan: String): Unit = { - postLiveListenerBus(SparkListenerStatementParsed(id, executionPlan)) - } - - def onStatementCanceled(id: String): Unit = { - postLiveListenerBus(SparkListenerStatementCanceled(id, System.currentTimeMillis())) - } - - def onStatementError(id: String, errorMsg: String, errorTrace: String): Unit = { - postLiveListenerBus(SparkListenerStatementError(id, errorMsg, errorTrace, - System.currentTimeMillis())) - } - - def onStatementFinish(id: String): Unit = { - postLiveListenerBus(SparkListenerStatementFinish(id, System.currentTimeMillis())) - - } - - def onOperationClosed(id: String): Unit = { - postLiveListenerBus(SparkListenerOperationClosed(id, System.currentTimeMillis())) - } - - /** Go through all `LiveEntity`s and use `entityFlushFunc(entity)` to flush them. */ private def flush(entityFlushFunc: LiveEntity => Unit): Unit = { sessionList.values.asScala.foreach(entityFlushFunc) @@ -372,43 +305,4 @@ private[thriftserver] class LiveSessionData( finishTimestamp, totalExecution) } - } - -private[thriftserver] case class SparkListenerSessionCreated( - ip: String, - sessionId: String, - userName: String, - startTime: Long) extends SparkListenerEvent - -private[thriftserver] case class SparkListenerSessionClosed( - sessionId: String, finishTime: Long) extends SparkListenerEvent - -private[thriftserver] case class SparkListenerStatementStart( - id: String, - sessionId: String, - statement: String, - groupId: String, - startTime: Long, - userName: String = "UNKNOWN") extends SparkListenerEvent - -private[thriftserver] case class SparkListenerStatementParsed( - id: String, - executionPlan: String) extends SparkListenerEvent - -private[thriftserver] case class SparkListenerStatementCanceled( - id: String, finishTime: Long) extends SparkListenerEvent - -private[thriftserver] case class SparkListenerStatementError( - id: String, - errorMsg: String, - errorTrace: String, - finishTime: Long) extends SparkListenerEvent - -private[thriftserver] case class SparkListenerStatementFinish(id: String, finishTime: Long) - extends SparkListenerEvent - -private[thriftserver] case class SparkListenerOperationClosed(id: String, closeTime: Long) - extends SparkListenerEvent - - diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala index 0bd0239c6a8d8..a6abb5f47a2b7 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.hive.thriftserver.ui -import java.util.Calendar import javax.servlet.http.HttpServletRequest import scala.collection.JavaConverters._ diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala index a3fb7959cd1a7..6d783b1c555a7 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.hive.thriftserver.ui import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.internal.Logging -import org.apache.spark.sql.hive.thriftserver.{HiveThriftServer2, HiveThriftServer2Listener} import org.apache.spark.ui.{SparkUI, SparkUITab} /** diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala similarity index 75% rename from sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala rename to sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala index 9b61d2926f91a..3aa14381f14cb 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2ListenerSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.hive.thriftserver +package org.apache.spark.sql.hive.thriftserver.ui import java.util.Properties @@ -25,7 +25,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.internal.config.Status.ASYNC_TRACKING_ENABLED import org.apache.spark.scheduler.SparkListenerJobStart -import org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2AppStatusStore +import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2 import org.apache.spark.sql.internal.SQLConf import org.apache.spark.status.ElementTrackingStore import org.apache.spark.util.kvstore.InMemoryStore @@ -44,23 +44,19 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite with BeforeAndAfter { Seq(true, false).foreach { live => test(s"listener events should store successfully (live = $live") { val (statusStore: HiveThriftServer2AppStatusStore, - listener: HiveThriftServer2Listener) = if (live) { - createLiveStatusStore - } else { - createHistoryStatusStore - } + listener: HiveThriftServer2Listener) = createAppStatusStore(live) listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionId", "user", System.currentTimeMillis())) - listener.onOtherEvent(SparkListenerStatementStart("id", "sessionId", "dummy query", + listener.onOtherEvent(SparkListenerOperationStart("id", "sessionId", "dummy query", "groupId", System.currentTimeMillis(), "user")) - listener.onOtherEvent(SparkListenerStatementParsed("id", "dummy plan")) + listener.onOtherEvent(SparkListenerOperationParsed("id", "dummy plan")) listener.onJobStart(SparkListenerJobStart( 0, System.currentTimeMillis(), Nil, createProperties)) - listener.onOtherEvent(SparkListenerStatementFinish("id", System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerOperationFinish("id", System.currentTimeMillis())) listener.onOtherEvent(SparkListenerOperationClosed("id", System.currentTimeMillis())) assert(statusStore.getOnlineSessionNum === 1) @@ -83,11 +79,7 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite with BeforeAndAfter { Seq(true, false).foreach { live => test(s"cleanup session if exceeds the threshold (live = $live)") { val (statusStore: HiveThriftServer2AppStatusStore, - listener: HiveThriftServer2Listener) = if (live) { - createLiveStatusStore - } else { - createHistoryStatusStore - } + listener: HiveThriftServer2Listener) = createAppStatusStore(live) var time = 0 listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionId1", "user", time)) time += 1 @@ -115,14 +107,14 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite with BeforeAndAfter { test("update execution info when jobstart event come after execution end event") { val (statusStore: HiveThriftServer2AppStatusStore, - listener: HiveThriftServer2Listener) = createLiveStatusStore + listener: HiveThriftServer2Listener) = createAppStatusStore(true) listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionId", "user", System.currentTimeMillis())) - listener.onOtherEvent(SparkListenerStatementStart("id", "sessionId", "dummy query", + listener.onOtherEvent(SparkListenerOperationStart("id", "sessionId", "dummy query", "groupId", System.currentTimeMillis(), "user")) - listener.onOtherEvent(SparkListenerStatementParsed("id", "dummy plan")) - listener.onOtherEvent(SparkListenerStatementFinish("id", System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerOperationParsed("id", "dummy plan")) + listener.onOtherEvent(SparkListenerOperationFinish("id", System.currentTimeMillis())) listener.onOtherEvent(SparkListenerOperationClosed("id", System.currentTimeMillis())) listener.onJobStart(SparkListenerJobStart( 0, @@ -142,25 +134,18 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite with BeforeAndAfter { properties } - private def createLiveStatusStore = { + private def createAppStatusStore(live: Boolean) = { val sparkConf = new SparkConf() sparkConf.set(ASYNC_TRACKING_ENABLED, false) - kvstore = new ElementTrackingStore(new InMemoryStore, sparkConf) - val server = mock(classOf[HiveThriftServer2], RETURNS_SMART_NULLS) - val sc = mock(classOf[SparkContext]) - val sqlConf = new SQLConf - sqlConf.setConfString(SQLConf.THRIFTSERVER_UI_SESSION_LIMIT.key, "1") - val listener = new HiveThriftServer2Listener(kvstore, Some(server), Some(sqlConf), Some(sc)) - - (new HiveThriftServer2AppStatusStore(kvstore, Some(listener)), listener) - } - - private def createHistoryStatusStore = { - val sparkConf = new SparkConf() - sparkConf.set(ASYNC_TRACKING_ENABLED, false) - kvstore = new ElementTrackingStore(new InMemoryStore, sparkConf) sparkConf.set(SQLConf.THRIFTSERVER_UI_SESSION_LIMIT.key, "1") - (new HiveThriftServer2AppStatusStore(kvstore), - new HiveThriftServer2Listener(kvstore, None, None, None, Some(sparkConf), false)) + kvstore = new ElementTrackingStore(new InMemoryStore, sparkConf) + if (live) { + val server = mock(classOf[HiveThriftServer2], RETURNS_SMART_NULLS) + val listener = new HiveThriftServer2Listener(kvstore, sparkConf, Some(server)) + (new HiveThriftServer2AppStatusStore(kvstore, Some(listener)), listener) + } else { + (new HiveThriftServer2AppStatusStore(kvstore), + new HiveThriftServer2Listener(kvstore, sparkConf, None, false)) + } } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala index b0bca64268380..f828df126f0db 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala @@ -23,10 +23,9 @@ import javax.servlet.http.HttpServletRequest import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} import org.scalatest.BeforeAndAfter -import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.scheduler.SparkListenerJobStart import org.apache.spark.sql.hive.thriftserver._ -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.status.ElementTrackingStore import org.apache.spark.util.kvstore.InMemoryStore @@ -48,19 +47,18 @@ class ThriftServerPageSuite extends SparkFunSuite with BeforeAndAfter { private def getStatusStore: HiveThriftServer2AppStatusStore = { kvstore = new ElementTrackingStore(new InMemoryStore, new SparkConf()) val server = mock(classOf[HiveThriftServer2], RETURNS_SMART_NULLS) - val sc = mock(classOf[SparkContext]) - val sqlConf = new SQLConf + val sparkConf = new SparkConf - val listener = new HiveThriftServer2Listener(kvstore, Some(server), Some(sqlConf), Some(sc)) + val listener = new HiveThriftServer2Listener(kvstore, sparkConf, Some(server)) val statusStore = new HiveThriftServer2AppStatusStore(kvstore, Some(listener)) listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionid", "user", System.currentTimeMillis())) - listener.onOtherEvent(SparkListenerStatementStart("id", "sessionid", + listener.onOtherEvent(SparkListenerOperationStart("id", "sessionid", "dummy query", "groupid", System.currentTimeMillis(), "user")) - listener.onOtherEvent(SparkListenerStatementParsed("id", "dummy plan")) + listener.onOtherEvent(SparkListenerOperationParsed("id", "dummy plan")) listener.onOtherEvent(SparkListenerJobStart(0, System.currentTimeMillis(), Seq())) - listener.onOtherEvent(SparkListenerStatementFinish("id", System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerOperationFinish("id", System.currentTimeMillis())) listener.onOtherEvent(SparkListenerOperationClosed("id", System.currentTimeMillis())) listener.onOtherEvent(SparkListenerSessionClosed("sessionid", System.currentTimeMillis())) From c27718356a2463df8f20c88fdc97282f09b391ab Mon Sep 17 00:00:00 2001 From: shahid Date: Sun, 24 Nov 2019 02:24:26 +0530 Subject: [PATCH 13/24] scalastyle --- .../sql/hive/thriftserver/SparkExecuteStatementOperation.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index 875c4d2ad8b5c..76d07848f79a9 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -284,7 +284,8 @@ private[hive] class SparkExecuteStatementOperation( "in this session.") case _ => } - HiveThriftServer2.eventManager.onStatementParsed(statementId, result.queryExecution.toString()) + HiveThriftServer2.eventManager.onStatementParsed(statementId, + result.queryExecution.toString()) iter = { if (sqlContext.getConf(SQLConf.THRIFTSERVER_INCREMENTAL_COLLECT.key).toBoolean) { resultList = None From 928711814a0d9f01aa4d1f8ac442bca118bf9381 Mon Sep 17 00:00:00 2001 From: shahid Date: Sun, 24 Nov 2019 22:52:13 +0530 Subject: [PATCH 14/24] address comment --- .../hive/thriftserver/ui/ThriftServerPage.scala | 14 +++++++++----- .../thriftserver/ui/ThriftServerSessionPage.scala | 2 +- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala index 0041b0c69af26..adfda0c56585f 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala @@ -40,15 +40,19 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" /** Render the page */ def render(request: HttpServletRequest): Seq[Node] = { - val content = // make sure all parts in this page are consistent - generateBasicStats() ++ -
++ + val content = store.synchronized { // make sure all parts in this page are consistent + generateBasicStats() ++ +
++

- {store.getOnlineSessionNum} session(s) are online, - running {store.getTotalRunning} SQL statement(s) + {store.getOnlineSessionNum} + session(s) are online, + running + {store.getTotalRunning} + SQL statement(s)

++ generateSessionStatsTable(request) ++ generateSQLStatsTable(request) + } UIUtils.headerSparkPage(request, "JDBC/ODBC Server", content, parent) } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala index a6abb5f47a2b7..c46c3d6b68a43 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala @@ -38,7 +38,7 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) val parameterId = request.getParameter("id") require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") - val content = { // make sure all parts in this page are consistent + val content = store.synchronized { // make sure all parts in this page are consistent val sessionStat = store.getSession(parameterId).getOrElse(null) require(sessionStat != null, "Invalid sessionID[" + parameterId + "]") From 4ab4e92e7dbe05b7a6f49c6eee1a6744c795e7a5 Mon Sep 17 00:00:00 2001 From: shahid Date: Tue, 26 Nov 2019 00:30:34 +0530 Subject: [PATCH 15/24] address comment --- .../deploy/history/FsHistoryProvider.scala | 4 +- .../ui/HiveThriftServer2AppStatusStore.scala | 19 ++++++---- .../ui/HiveThriftServer2EventManager.scala | 36 +++++++++--------- .../ui/HiveThriftServer2Listener.scala | 36 +++++++++--------- .../ui/HiveThriftServer2ListenerSuite.scala | 37 +++++++++++-------- .../ui/ThriftServerPageSuite.scala | 10 ++--- 6 files changed, 77 insertions(+), 65 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 32c21cee85a9c..8227237676cc5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -369,9 +369,9 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) val otherTabs: Seq[AppHistoryServerPlugin] = Seq() plugins.foreach{ plugin => - if (plugin.toString.contains("SQL")) { + if (plugin.toString.contains("SQLHistoryServerPlugin")) { sqlTab = Some(plugin) - } else if (plugin.toString.contains("ThriftServer")) { + } else if (plugin.toString.contains("HiveThriftServer2HistoryServerPlugin")) { jdbcTab = Some(plugin) } else { otherTabs :+ plugin diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala index 9ae8dc9af6ccf..db26370a6c1f8 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala @@ -67,13 +67,7 @@ class HiveThriftServer2AppStatusStore( * cancellations and count all statements that have not been closed so far. */ def getTotalRunning: Int = { - store.view(classOf[ExecutionInfo]).asScala.count(isExecutionActive) - } - - def isExecutionActive(execInfo: ExecutionInfo): Boolean = { - !(execInfo.state == ExecutionState.FAILED || - execInfo.state == ExecutionState.CANCELED || - execInfo.state == ExecutionState.CLOSED) + store.view(classOf[ExecutionInfo]).asScala.count(_.isExecutionActive) } def getSessionCount: Long = { @@ -117,7 +111,16 @@ private[thriftserver] class ExecutionInfo( val jobId: ArrayBuffer[String], val groupId: String) { @JsonIgnore @KVIndex("finishTime") - private def finishTimeIndex: Long = if (finishTimestamp > 0L ) finishTimestamp else -1L + private def finishTimeIndex: Long = if (finishTimestamp > 0L && !isExecutionActive) { + finishTimestamp + } else -1L + + def isExecutionActive: Boolean = { + !(state == ExecutionState.FAILED || + state == ExecutionState.CANCELED || + state == ExecutionState.CLOSED) + } + def totalTime(endTime: Long): Long = { if (endTime == 0L) { System.currentTimeMillis - startTimestamp diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2EventManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2EventManager.scala index c6024188e79e7..949c9166b3231 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2EventManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2EventManager.scala @@ -26,11 +26,11 @@ import org.apache.spark.scheduler.SparkListenerEvent private[thriftserver] class HiveThriftServer2EventManager(sc: SparkContext) { def postLiveListenerBus(event: SparkListenerEvent): Unit = { - sc.listenerBus.post(event) + sc.listenerBus.post(event) } def onSessionCreated(ip: String, sessionId: String, userName: String = "UNKNOWN"): Unit = { - postLiveListenerBus(SparkListenerSessionCreated(ip, sessionId, + postLiveListenerBus(SparkListenerThriftServerSessionCreated(ip, sessionId, userName, System.currentTimeMillis())) } @@ -44,34 +44,34 @@ private[thriftserver] class HiveThriftServer2EventManager(sc: SparkContext) { statement: String, groupId: String, userName: String = "UNKNOWN"): Unit = { - postLiveListenerBus(SparkListenerOperationStart(id, sessionId, statement, groupId, + postLiveListenerBus(SparkListenerThriftServerOperationStart(id, sessionId, statement, groupId, System.currentTimeMillis(), userName)) } def onStatementParsed(id: String, executionPlan: String): Unit = { - postLiveListenerBus(SparkListenerOperationParsed(id, executionPlan)) + postLiveListenerBus(SparkListenerThriftServerOperationParsed(id, executionPlan)) } def onStatementCanceled(id: String): Unit = { - postLiveListenerBus(SparkListenerOperationCanceled(id, System.currentTimeMillis())) + postLiveListenerBus(SparkListenerThriftServerOperationCanceled(id, System.currentTimeMillis())) } def onStatementError(id: String, errorMsg: String, errorTrace: String): Unit = { - postLiveListenerBus(SparkListenerOperationError(id, errorMsg, errorTrace, + postLiveListenerBus(SparkListenerThriftServerOperationError(id, errorMsg, errorTrace, System.currentTimeMillis())) } def onStatementFinish(id: String): Unit = { - postLiveListenerBus(SparkListenerOperationFinish(id, System.currentTimeMillis())) + postLiveListenerBus(SparkListenerThriftServerOperationFinish(id, System.currentTimeMillis())) } def onOperationClosed(id: String): Unit = { - postLiveListenerBus(SparkListenerOperationClosed(id, System.currentTimeMillis())) + postLiveListenerBus(SparkListenerThriftServerOperationClosed(id, System.currentTimeMillis())) } } -private[thriftserver] case class SparkListenerSessionCreated( +private[thriftserver] case class SparkListenerThriftServerSessionCreated( ip: String, sessionId: String, userName: String, @@ -80,7 +80,7 @@ private[thriftserver] case class SparkListenerSessionCreated( private[thriftserver] case class SparkListenerSessionClosed( sessionId: String, finishTime: Long) extends SparkListenerEvent -private[thriftserver] case class SparkListenerOperationStart( +private[thriftserver] case class SparkListenerThriftServerOperationStart( id: String, sessionId: String, statement: String, @@ -88,23 +88,25 @@ private[thriftserver] case class SparkListenerOperationStart( startTime: Long, userName: String = "UNKNOWN") extends SparkListenerEvent -private[thriftserver] case class SparkListenerOperationParsed( +private[thriftserver] case class SparkListenerThriftServerOperationParsed( id: String, executionPlan: String) extends SparkListenerEvent -private[thriftserver] case class SparkListenerOperationCanceled( +private[thriftserver] case class SparkListenerThriftServerOperationCanceled( id: String, finishTime: Long) extends SparkListenerEvent -private[thriftserver] case class SparkListenerOperationError( +private[thriftserver] case class SparkListenerThriftServerOperationError( id: String, errorMsg: String, errorTrace: String, finishTime: Long) extends SparkListenerEvent -private[thriftserver] case class SparkListenerOperationFinish(id: String, finishTime: Long) - extends SparkListenerEvent +private[thriftserver] case class SparkListenerThriftServerOperationFinish( + id: String, + finishTime: Long) extends SparkListenerEvent -private[thriftserver] case class SparkListenerOperationClosed(id: String, closeTime: Long) - extends SparkListenerEvent +private[thriftserver] case class SparkListenerThriftServerOperationClosed( + id: String, + closeTime: Long) extends SparkListenerEvent diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala index 010c5ac6e281d..10d43aab7d68e 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala @@ -34,10 +34,10 @@ import org.apache.spark.status.{ElementTrackingStore, KVUtils, LiveEntity} * An inner sparkListener called in sc.stop to clean up the HiveThriftServer2 */ private[thriftserver] class HiveThriftServer2Listener( - kvstore: ElementTrackingStore, - sparkConf: SparkConf, - server: Option[HiveServer2], - live: Boolean = true) extends SparkListener { + kvstore: ElementTrackingStore, + sparkConf: SparkConf, + server: Option[HiveServer2], + live: Boolean = true) extends SparkListener { private val sessionList = new ConcurrentHashMap[String, LiveSessionData]() private val executionList = new ConcurrentHashMap[String, LiveExecutionData]() @@ -110,19 +110,19 @@ private[thriftserver] class HiveThriftServer2Listener( override def onOtherEvent(event: SparkListenerEvent): Unit = { event match { - case e: SparkListenerSessionCreated => onSessionCreated(e) + case e: SparkListenerThriftServerSessionCreated => onSessionCreated(e) case e: SparkListenerSessionClosed => onSessionClosed(e) - case e: SparkListenerOperationStart => onOperationStart(e) - case e: SparkListenerOperationParsed => onOperationParsed(e) - case e: SparkListenerOperationCanceled => onOperationCanceled(e) - case e: SparkListenerOperationError => onOperationError(e) - case e: SparkListenerOperationFinish => onOperationFinished(e) - case e: SparkListenerOperationClosed => onOperationClosed(e) + case e: SparkListenerThriftServerOperationStart => onOperationStart(e) + case e: SparkListenerThriftServerOperationParsed => onOperationParsed(e) + case e: SparkListenerThriftServerOperationCanceled => onOperationCanceled(e) + case e: SparkListenerThriftServerOperationError => onOperationError(e) + case e: SparkListenerThriftServerOperationFinish => onOperationFinished(e) + case e: SparkListenerThriftServerOperationClosed => onOperationClosed(e) case _ => // Ignore } } - private def onSessionCreated(e: SparkListenerSessionCreated): Unit = { + private def onSessionCreated(e: SparkListenerThriftServerSessionCreated): Unit = { val session = getOrCreateSession(e.sessionId, e.startTime, e.ip, e.userName) sessionList.put(e.sessionId, session) updateLiveStore(session, true) @@ -135,7 +135,7 @@ private[thriftserver] class HiveThriftServer2Listener( sessionList.remove(e.sessionId) } - private def onOperationStart(e: SparkListenerOperationStart): Unit = { + private def onOperationStart(e: SparkListenerThriftServerOperationStart): Unit = { val info = getOrCreateExecution( e.id, e.statement, @@ -151,32 +151,32 @@ private[thriftserver] class HiveThriftServer2Listener( updateLiveStore(sessionList.get(e.sessionId), true) } - private def onOperationParsed(e: SparkListenerOperationParsed): Unit = { + private def onOperationParsed(e: SparkListenerThriftServerOperationParsed): Unit = { executionList.get(e.id).executePlan = e.executionPlan executionList.get(e.id).state = ExecutionState.COMPILED updateLiveStore(executionList.get(e.id)) } - private def onOperationCanceled(e: SparkListenerOperationCanceled): Unit = { + private def onOperationCanceled(e: SparkListenerThriftServerOperationCanceled): Unit = { executionList.get(e.id).finishTimestamp = e.finishTime executionList.get(e.id).state = ExecutionState.CANCELED updateLiveStore(executionList.get(e.id)) } - private def onOperationError(e: SparkListenerOperationError): Unit = { + private def onOperationError(e: SparkListenerThriftServerOperationError): Unit = { executionList.get(e.id).finishTimestamp = e.finishTime executionList.get(e.id).detail = e.errorMsg executionList.get(e.id).state = ExecutionState.FAILED updateLiveStore(executionList.get(e.id)) } - private def onOperationFinished(e: SparkListenerOperationFinish): Unit = { + private def onOperationFinished(e: SparkListenerThriftServerOperationFinish): Unit = { executionList.get(e.id).finishTimestamp = e.finishTime executionList.get(e.id).state = ExecutionState.FINISHED updateLiveStore(executionList.get(e.id)) } - private def onOperationClosed(e: SparkListenerOperationClosed): Unit = { + private def onOperationClosed(e: SparkListenerThriftServerOperationClosed): Unit = { executionList.get(e.id).closeTimestamp = e.closeTime executionList.get(e.id).state = ExecutionState.CLOSED updateLiveStore(executionList.get(e.id), true) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala index 3aa14381f14cb..2edf600b01564 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala @@ -46,18 +46,20 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite with BeforeAndAfter { val (statusStore: HiveThriftServer2AppStatusStore, listener: HiveThriftServer2Listener) = createAppStatusStore(live) - listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionId", "user", - System.currentTimeMillis())) - listener.onOtherEvent(SparkListenerOperationStart("id", "sessionId", "dummy query", - "groupId", System.currentTimeMillis(), "user")) - listener.onOtherEvent(SparkListenerOperationParsed("id", "dummy plan")) + listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId", + "user", System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerOperationStart("id", "sessionId", + "dummy query", "groupId", System.currentTimeMillis(), "user")) + listener.onOtherEvent(SparkListenerThriftServerOperationParsed("id", "dummy plan")) listener.onJobStart(SparkListenerJobStart( 0, System.currentTimeMillis(), Nil, createProperties)) - listener.onOtherEvent(SparkListenerOperationFinish("id", System.currentTimeMillis())) - listener.onOtherEvent(SparkListenerOperationClosed("id", System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerOperationFinish("id", + System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerOperationClosed("id", + System.currentTimeMillis())) assert(statusStore.getOnlineSessionNum === 1) @@ -81,9 +83,11 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite with BeforeAndAfter { val (statusStore: HiveThriftServer2AppStatusStore, listener: HiveThriftServer2Listener) = createAppStatusStore(live) var time = 0 - listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionId1", "user", time)) + listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId1", + "user", time)) time += 1 - listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionId2", "user", time)) + listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId2", + "user", time)) assert(statusStore.getOnlineSessionNum === 2) assert(statusStore.getSessionCount === 2) @@ -94,7 +98,8 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite with BeforeAndAfter { time += 1 listener.onOtherEvent(SparkListenerSessionClosed("sessionId2", time)) - listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionId3", "user", time)) + listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId3", + "user", time)) assert(statusStore.getOnlineSessionNum === 1) assert(statusStore.getSessionCount === 1) assert(statusStore.getSession("sessionId1") === None) @@ -109,13 +114,15 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite with BeforeAndAfter { val (statusStore: HiveThriftServer2AppStatusStore, listener: HiveThriftServer2Listener) = createAppStatusStore(true) - listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionId", "user", + listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId", "user", System.currentTimeMillis())) - listener.onOtherEvent(SparkListenerOperationStart("id", "sessionId", "dummy query", + listener.onOtherEvent(SparkListenerThriftServerOperationStart("id", "sessionId", "dummy query", "groupId", System.currentTimeMillis(), "user")) - listener.onOtherEvent(SparkListenerOperationParsed("id", "dummy plan")) - listener.onOtherEvent(SparkListenerOperationFinish("id", System.currentTimeMillis())) - listener.onOtherEvent(SparkListenerOperationClosed("id", System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerOperationParsed("id", "dummy plan")) + listener.onOtherEvent(SparkListenerThriftServerOperationFinish("id", + System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerOperationClosed("id", + System.currentTimeMillis())) listener.onJobStart(SparkListenerJobStart( 0, System.currentTimeMillis(), diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala index f828df126f0db..aac2bf58542cc 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala @@ -52,14 +52,14 @@ class ThriftServerPageSuite extends SparkFunSuite with BeforeAndAfter { val listener = new HiveThriftServer2Listener(kvstore, sparkConf, Some(server)) val statusStore = new HiveThriftServer2AppStatusStore(kvstore, Some(listener)) - listener.onOtherEvent(SparkListenerSessionCreated("localhost", "sessionid", "user", + listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionid", "user", System.currentTimeMillis())) - listener.onOtherEvent(SparkListenerOperationStart("id", "sessionid", + listener.onOtherEvent(SparkListenerThriftServerOperationStart("id", "sessionid", "dummy query", "groupid", System.currentTimeMillis(), "user")) - listener.onOtherEvent(SparkListenerOperationParsed("id", "dummy plan")) + listener.onOtherEvent(SparkListenerThriftServerOperationParsed("id", "dummy plan")) listener.onOtherEvent(SparkListenerJobStart(0, System.currentTimeMillis(), Seq())) - listener.onOtherEvent(SparkListenerOperationFinish("id", System.currentTimeMillis())) - listener.onOtherEvent(SparkListenerOperationClosed("id", System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerOperationFinish("id", System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerOperationClosed("id", System.currentTimeMillis())) listener.onOtherEvent(SparkListenerSessionClosed("sessionid", System.currentTimeMillis())) statusStore From 607ddedf9599addc598426694e9eafab296514e3 Mon Sep 17 00:00:00 2001 From: shahid Date: Tue, 26 Nov 2019 00:39:07 +0530 Subject: [PATCH 16/24] scalastyle --- .../sql/hive/thriftserver/ui/ThriftServerPageSuite.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala index aac2bf58542cc..3505e8c86ab5c 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala @@ -58,8 +58,10 @@ class ThriftServerPageSuite extends SparkFunSuite with BeforeAndAfter { "dummy query", "groupid", System.currentTimeMillis(), "user")) listener.onOtherEvent(SparkListenerThriftServerOperationParsed("id", "dummy plan")) listener.onOtherEvent(SparkListenerJobStart(0, System.currentTimeMillis(), Seq())) - listener.onOtherEvent(SparkListenerThriftServerOperationFinish("id", System.currentTimeMillis())) - listener.onOtherEvent(SparkListenerThriftServerOperationClosed("id", System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerOperationFinish("id", + System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerOperationClosed("id", + System.currentTimeMillis())) listener.onOtherEvent(SparkListenerSessionClosed("sessionid", System.currentTimeMillis())) statusStore From ff7267cfd7d4d300d003a8316f1769991bbbf3d8 Mon Sep 17 00:00:00 2001 From: shahid Date: Wed, 27 Nov 2019 04:08:17 +0530 Subject: [PATCH 17/24] Address comments including refactoring of update --- .../deploy/history/FsHistoryProvider.scala | 1 + .../ui/HiveThriftServer2AppStatusStore.scala | 1 + .../ui/HiveThriftServer2Listener.scala | 44 ++-- .../ui/HiveThriftServer2ListenerSuite.scala | 195 ++++++++++++------ 4 files changed, 156 insertions(+), 85 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 8227237676cc5..1b2f993cf3f54 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -377,6 +377,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) otherTabs :+ plugin } } + // SQL tab should come first, then JDBC/ODBC server tab sqlTab.foreach(_.setupUI(ui)) jdbcTab.foreach(_.setupUI(ui)) otherTabs.foreach(_.setupUI(ui)) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala index db26370a6c1f8..5cb78f6e64650 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala @@ -115,6 +115,7 @@ private[thriftserver] class ExecutionInfo( finishTimestamp } else -1L + @JsonIgnore @KVIndex("isExecutionActive") def isExecutionActive: Boolean = { !(state == ExecutionState.FAILED || state == ExecutionState.CANCELED || diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala index 10d43aab7d68e..e7a63305d5898 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala @@ -25,6 +25,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.hive.service.server.HiveServer2 import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.internal.config.Status.LIVE_ENTITY_UPDATE_PERIOD import org.apache.spark.scheduler._ import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.ExecutionState import org.apache.spark.sql.internal.SQLConf @@ -47,6 +48,11 @@ private[thriftserver] class HiveThriftServer2Listener( sparkConf.get(SQLConf.THRIFTSERVER_UI_SESSION_LIMIT)) } + // How often to update live entities. -1 means "never update" when replaying applications, + // meaning only the last write will happen. For live applications, this avoids a few + // operations that we can live without when rapidly processing incoming task events. + private val liveUpdatePeriodNs = if (live) sparkConf.get(LIVE_ENTITY_UPDATE_PERIOD) else -1L + // Returns true if this listener has no live data. Exposed for tests only. private[thriftserver] def noLiveData(): Boolean = { sessionList.isEmpty && executionList.isEmpty @@ -62,8 +68,7 @@ private[thriftserver] class HiveThriftServer2Listener( kvstore.onFlush { if (!live) { - val now = System.nanoTime() - flush(update(_, now)) + flush(updateStore(_, trigger = true)) } } @@ -102,7 +107,7 @@ private[thriftserver] class HiveThriftServer2Listener( val liveExec = getOrCreateExecution(exec.execId, exec.statement, exec.sessionId, exec.startTimestamp, exec.userName) liveExec.jobId += jobId.toString - updateLiveStore(liveExec, true) + updateStore(liveExec, trigger = true) executionList.remove(liveExec.execId) } } @@ -125,13 +130,13 @@ private[thriftserver] class HiveThriftServer2Listener( private def onSessionCreated(e: SparkListenerThriftServerSessionCreated): Unit = { val session = getOrCreateSession(e.sessionId, e.startTime, e.ip, e.userName) sessionList.put(e.sessionId, session) - updateLiveStore(session, true) + updateLiveStore(session) } private def onSessionClosed(e: SparkListenerSessionClosed): Unit = { val session = sessionList.get(e.sessionId) session.finishTimestamp = e.finishTime - updateLiveStore(session, true) + updateStore(session, trigger = true) sessionList.remove(e.sessionId) } @@ -148,7 +153,7 @@ private[thriftserver] class HiveThriftServer2Listener( sessionList.get(e.sessionId).totalExecution += 1 executionList.get(e.id).groupId = e.groupId updateLiveStore(executionList.get(e.id)) - updateLiveStore(sessionList.get(e.sessionId), true) + updateLiveStore(sessionList.get(e.sessionId)) } private def onOperationParsed(e: SparkListenerThriftServerOperationParsed): Unit = { @@ -179,24 +184,29 @@ private[thriftserver] class HiveThriftServer2Listener( private def onOperationClosed(e: SparkListenerThriftServerOperationClosed): Unit = { executionList.get(e.id).closeTimestamp = e.closeTime executionList.get(e.id).state = ExecutionState.CLOSED - updateLiveStore(executionList.get(e.id), true) + updateStore(executionList.get(e.id), trigger = true) executionList.remove(e.id) } - /** Go through all `LiveEntity`s and use `entityFlushFunc(entity)` to flush them. */ - private def flush(entityFlushFunc: LiveEntity => Unit): Unit = { - sessionList.values.asScala.foreach(entityFlushFunc) - executionList.values.asScala.foreach(entityFlushFunc) + // Update both live and history stores. If trigger is enabled, it will cleanup + // entity which exceeds the threshold. + def updateStore(entity: LiveEntity, trigger: Boolean = false): Unit = { + entity.write(kvstore, System.nanoTime(), checkTriggers = trigger) } - private def update(entity: LiveEntity, now: Long): Unit = { - entity.write(kvstore, now) + // Update only live stores. If trigger is enabled, it will cleanup entity + // which exceeds the threshold. + def updateLiveStore(entity: LiveEntity, trigger: Boolean = false): Unit = { + val now = System.nanoTime() + if (live && liveUpdatePeriodNs >= 0 && now - entity.lastWriteTime > liveUpdatePeriodNs) { + entity.write(kvstore, System.nanoTime(), checkTriggers = trigger) + } } - def updateLiveStore(session: LiveEntity, force: Boolean = false): Unit = { - if (live || force == true) { - session.write(kvstore, System.nanoTime()) - } + /** Go through all `LiveEntity`s and use `entityFlushFunc(entity)` to flush them. */ + private def flush(entityFlushFunc: LiveEntity => Unit): Unit = { + sessionList.values.asScala.foreach(entityFlushFunc) + executionList.values.asScala.foreach(entityFlushFunc) } private def getOrCreateSession( diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala index 2edf600b01564..4ab10382cb004 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala @@ -23,7 +23,7 @@ import org.mockito.Mockito.{mock, RETURNS_SMART_NULLS} import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} -import org.apache.spark.internal.config.Status.ASYNC_TRACKING_ENABLED +import org.apache.spark.internal.config.Status.{ASYNC_TRACKING_ENABLED, LIVE_ENTITY_UPDATE_PERIOD} import org.apache.spark.scheduler.SparkListenerJobStart import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2 import org.apache.spark.sql.internal.SQLConf @@ -41,73 +41,131 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite with BeforeAndAfter { } } - Seq(true, false).foreach { live => - test(s"listener events should store successfully (live = $live") { - val (statusStore: HiveThriftServer2AppStatusStore, - listener: HiveThriftServer2Listener) = createAppStatusStore(live) - - listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId", - "user", System.currentTimeMillis())) - listener.onOtherEvent(SparkListenerThriftServerOperationStart("id", "sessionId", - "dummy query", "groupId", System.currentTimeMillis(), "user")) - listener.onOtherEvent(SparkListenerThriftServerOperationParsed("id", "dummy plan")) - listener.onJobStart(SparkListenerJobStart( - 0, - System.currentTimeMillis(), - Nil, - createProperties)) - listener.onOtherEvent(SparkListenerThriftServerOperationFinish("id", - System.currentTimeMillis())) - listener.onOtherEvent(SparkListenerThriftServerOperationClosed("id", - System.currentTimeMillis())) - - assert(statusStore.getOnlineSessionNum === 1) - - listener.onOtherEvent(SparkListenerSessionClosed("sessionId", System.currentTimeMillis())) - - assert(statusStore.getOnlineSessionNum === 0) - assert(statusStore.getExecutionList.size === 1) - - val storeExecData = statusStore.getExecutionList.head - - assert(storeExecData.execId === "id") - assert(storeExecData.sessionId === "sessionId") - assert(storeExecData.executePlan === "dummy plan") - assert(storeExecData.jobId === Seq("0")) - assert(listener.noLiveData()) - } + test(s"listener events should store successfully (live = true)") { + val (statusStore: HiveThriftServer2AppStatusStore, + listener: HiveThriftServer2Listener) = createAppStatusStore(true) + + listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId", + "user", System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerOperationStart("id", "sessionId", + "dummy query", "groupId", System.currentTimeMillis(), "user")) + listener.onOtherEvent(SparkListenerThriftServerOperationParsed("id", "dummy plan")) + listener.onJobStart(SparkListenerJobStart( + 0, + System.currentTimeMillis(), + Nil, + createProperties)) + listener.onOtherEvent(SparkListenerThriftServerOperationFinish("id", + System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerOperationClosed("id", + System.currentTimeMillis())) + + + assert(statusStore.getOnlineSessionNum === 1) + + listener.onOtherEvent(SparkListenerSessionClosed("sessionId", System.currentTimeMillis())) + + assert(statusStore.getOnlineSessionNum === 0) + assert(statusStore.getExecutionList.size === 1) + + val storeExecData = statusStore.getExecutionList.head + + assert(storeExecData.execId === "id") + assert(storeExecData.sessionId === "sessionId") + assert(storeExecData.executePlan === "dummy plan") + assert(storeExecData.jobId === Seq("0")) + assert(listener.noLiveData()) } - Seq(true, false).foreach { live => - test(s"cleanup session if exceeds the threshold (live = $live)") { - val (statusStore: HiveThriftServer2AppStatusStore, - listener: HiveThriftServer2Listener) = createAppStatusStore(live) - var time = 0 - listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId1", - "user", time)) - time += 1 - listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId2", - "user", time)) - - assert(statusStore.getOnlineSessionNum === 2) - assert(statusStore.getSessionCount === 2) - - time += 1 - listener.onOtherEvent(SparkListenerSessionClosed("sessionId1", time)) - - time += 1 - listener.onOtherEvent(SparkListenerSessionClosed("sessionId2", time)) - - listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId3", - "user", time)) - assert(statusStore.getOnlineSessionNum === 1) - assert(statusStore.getSessionCount === 1) - assert(statusStore.getSession("sessionId1") === None) - time += 1 - listener.onOtherEvent(SparkListenerSessionClosed("sessionId3", 4)) - - assert(listener.noLiveData()) - } + test(s"listener events should store successfully (live = false)") { + val (statusStore: HiveThriftServer2AppStatusStore, + listener: HiveThriftServer2Listener) = createAppStatusStore(false) + + listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId", + "user", System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerOperationStart("id", "sessionId", + "dummy query", "groupId", System.currentTimeMillis(), "user")) + listener.onOtherEvent(SparkListenerThriftServerOperationParsed("id", "dummy plan")) + listener.onJobStart(SparkListenerJobStart( + 0, + System.currentTimeMillis(), + Nil, + createProperties)) + listener.onOtherEvent(SparkListenerThriftServerOperationFinish("id", + System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerOperationClosed("id", + System.currentTimeMillis())) + + listener.onOtherEvent(SparkListenerSessionClosed("sessionId", System.currentTimeMillis())) + + kvstore.close(false) + assert(statusStore.getOnlineSessionNum === 0) + assert(statusStore.getExecutionList.size === 1) + + val storeExecData = statusStore.getExecutionList.head + + assert(storeExecData.execId === "id") + assert(storeExecData.sessionId === "sessionId") + assert(storeExecData.executePlan === "dummy plan") + assert(storeExecData.jobId === Seq("0")) + assert(listener.noLiveData()) + } + + test(s"cleanup session if exceeds the threshold (live = true)") { + val (statusStore: HiveThriftServer2AppStatusStore, + listener: HiveThriftServer2Listener) = createAppStatusStore(true) + var time = 0 + listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId1", + "user", time)) + time += 1 + listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId2", + "user", time)) + + assert(statusStore.getOnlineSessionNum === 2) + + assert(statusStore.getSessionCount === 2) + + time += 1 + listener.onOtherEvent(SparkListenerSessionClosed("sessionId1", time)) + + time += 1 + listener.onOtherEvent(SparkListenerSessionClosed("sessionId2", time)) + + listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId3", + "user", time)) + assert(statusStore.getOnlineSessionNum === 1) + assert(statusStore.getSessionCount === 1) + assert(statusStore.getSession("sessionId1") === None) + time += 1 + listener.onOtherEvent(SparkListenerSessionClosed("sessionId3", 4)) + assert(listener.noLiveData()) + } + + test(s"cleanup session if exceeds the threshold (live = false)") { + val (statusStore: HiveThriftServer2AppStatusStore, + listener: HiveThriftServer2Listener) = createAppStatusStore(true) + var time = 0 + listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId1", + "user", time)) + time += 1 + listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId2", + "user", time)) + + time += 1 + listener.onOtherEvent(SparkListenerSessionClosed("sessionId1", time)) + + time += 1 + listener.onOtherEvent(SparkListenerSessionClosed("sessionId2", time)) + + listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId3", + "user", time)) + time += 1 + listener.onOtherEvent(SparkListenerSessionClosed("sessionId3", 4)) + kvstore.close(false) + assert(statusStore.getOnlineSessionNum === 0) + assert(statusStore.getSessionCount === 1) + assert(statusStore.getSession("sessionId1") === None) + assert(listener.noLiveData()) } test("update execution info when jobstart event come after execution end event") { @@ -131,7 +189,7 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite with BeforeAndAfter { listener.onOtherEvent(SparkListenerSessionClosed("sessionId", System.currentTimeMillis())) val exec = statusStore.getExecution("id") assert(exec.isDefined) - assert(exec.get.jobId.toSeq === Seq("0")) + assert(exec.get.jobId === Seq("0")) assert(listener.noLiveData()) } @@ -144,7 +202,8 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite with BeforeAndAfter { private def createAppStatusStore(live: Boolean) = { val sparkConf = new SparkConf() sparkConf.set(ASYNC_TRACKING_ENABLED, false) - sparkConf.set(SQLConf.THRIFTSERVER_UI_SESSION_LIMIT.key, "1") + .set(SQLConf.THRIFTSERVER_UI_SESSION_LIMIT.key, "1") + .set(LIVE_ENTITY_UPDATE_PERIOD, 0L) kvstore = new ElementTrackingStore(new InMemoryStore, sparkConf) if (live) { val server = mock(classOf[HiveThriftServer2], RETURNS_SMART_NULLS) From eaea6e35b5d976aea03a29a27249080af4b8deb4 Mon Sep 17 00:00:00 2001 From: shahid Date: Wed, 27 Nov 2019 04:50:10 +0530 Subject: [PATCH 18/24] refactor test --- .../ui/HiveThriftServer2Listener.scala | 2 +- .../ui/HiveThriftServer2ListenerSuite.scala | 194 +++++++----------- 2 files changed, 72 insertions(+), 124 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala index e7a63305d5898..656a48941b98c 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala @@ -199,7 +199,7 @@ private[thriftserver] class HiveThriftServer2Listener( def updateLiveStore(entity: LiveEntity, trigger: Boolean = false): Unit = { val now = System.nanoTime() if (live && liveUpdatePeriodNs >= 0 && now - entity.lastWriteTime > liveUpdatePeriodNs) { - entity.write(kvstore, System.nanoTime(), checkTriggers = trigger) + entity.write(kvstore, now, checkTriggers = trigger) } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala index 4ab10382cb004..54eafb3b9b72f 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala @@ -41,131 +41,79 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite with BeforeAndAfter { } } - test(s"listener events should store successfully (live = true)") { - val (statusStore: HiveThriftServer2AppStatusStore, - listener: HiveThriftServer2Listener) = createAppStatusStore(true) - - listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId", - "user", System.currentTimeMillis())) - listener.onOtherEvent(SparkListenerThriftServerOperationStart("id", "sessionId", - "dummy query", "groupId", System.currentTimeMillis(), "user")) - listener.onOtherEvent(SparkListenerThriftServerOperationParsed("id", "dummy plan")) - listener.onJobStart(SparkListenerJobStart( - 0, - System.currentTimeMillis(), - Nil, - createProperties)) - listener.onOtherEvent(SparkListenerThriftServerOperationFinish("id", - System.currentTimeMillis())) - listener.onOtherEvent(SparkListenerThriftServerOperationClosed("id", - System.currentTimeMillis())) - - - assert(statusStore.getOnlineSessionNum === 1) - - listener.onOtherEvent(SparkListenerSessionClosed("sessionId", System.currentTimeMillis())) - - assert(statusStore.getOnlineSessionNum === 0) - assert(statusStore.getExecutionList.size === 1) - - val storeExecData = statusStore.getExecutionList.head - - assert(storeExecData.execId === "id") - assert(storeExecData.sessionId === "sessionId") - assert(storeExecData.executePlan === "dummy plan") - assert(storeExecData.jobId === Seq("0")) - assert(listener.noLiveData()) - } - - test(s"listener events should store successfully (live = false)") { - val (statusStore: HiveThriftServer2AppStatusStore, - listener: HiveThriftServer2Listener) = createAppStatusStore(false) - - listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId", - "user", System.currentTimeMillis())) - listener.onOtherEvent(SparkListenerThriftServerOperationStart("id", "sessionId", - "dummy query", "groupId", System.currentTimeMillis(), "user")) - listener.onOtherEvent(SparkListenerThriftServerOperationParsed("id", "dummy plan")) - listener.onJobStart(SparkListenerJobStart( - 0, - System.currentTimeMillis(), - Nil, - createProperties)) - listener.onOtherEvent(SparkListenerThriftServerOperationFinish("id", - System.currentTimeMillis())) - listener.onOtherEvent(SparkListenerThriftServerOperationClosed("id", - System.currentTimeMillis())) - - listener.onOtherEvent(SparkListenerSessionClosed("sessionId", System.currentTimeMillis())) - - kvstore.close(false) - assert(statusStore.getOnlineSessionNum === 0) - assert(statusStore.getExecutionList.size === 1) - - val storeExecData = statusStore.getExecutionList.head - - assert(storeExecData.execId === "id") - assert(storeExecData.sessionId === "sessionId") - assert(storeExecData.executePlan === "dummy plan") - assert(storeExecData.jobId === Seq("0")) - assert(listener.noLiveData()) - } - - test(s"cleanup session if exceeds the threshold (live = true)") { - val (statusStore: HiveThriftServer2AppStatusStore, - listener: HiveThriftServer2Listener) = createAppStatusStore(true) - var time = 0 - listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId1", - "user", time)) - time += 1 - listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId2", - "user", time)) - - assert(statusStore.getOnlineSessionNum === 2) - - assert(statusStore.getSessionCount === 2) - - time += 1 - listener.onOtherEvent(SparkListenerSessionClosed("sessionId1", time)) - - time += 1 - listener.onOtherEvent(SparkListenerSessionClosed("sessionId2", time)) - - listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId3", - "user", time)) - assert(statusStore.getOnlineSessionNum === 1) - assert(statusStore.getSessionCount === 1) - assert(statusStore.getSession("sessionId1") === None) - time += 1 - listener.onOtherEvent(SparkListenerSessionClosed("sessionId3", 4)) - assert(listener.noLiveData()) + Seq(true, false).foreach { live => + test(s"listener events should store successfully (live = $live)") { + val (statusStore: HiveThriftServer2AppStatusStore, + listener: HiveThriftServer2Listener) = createAppStatusStore(live) + + listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId", + "user", System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerOperationStart("id", "sessionId", + "dummy query", "groupId", System.currentTimeMillis(), "user")) + listener.onOtherEvent(SparkListenerThriftServerOperationParsed("id", "dummy plan")) + listener.onJobStart(SparkListenerJobStart( + 0, + System.currentTimeMillis(), + Nil, + createProperties)) + listener.onOtherEvent(SparkListenerThriftServerOperationFinish("id", + System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerOperationClosed("id", + System.currentTimeMillis())) + + if (live) { + assert(statusStore.getOnlineSessionNum === 1) + } + + listener.onOtherEvent(SparkListenerSessionClosed("sessionId", System.currentTimeMillis())) + + if (!live) { + // To update history store + kvstore.close(false) + } + assert(statusStore.getOnlineSessionNum === 0) + assert(statusStore.getExecutionList.size === 1) + + val storeExecData = statusStore.getExecutionList.head + + assert(storeExecData.execId === "id") + assert(storeExecData.sessionId === "sessionId") + assert(storeExecData.executePlan === "dummy plan") + assert(storeExecData.jobId === Seq("0")) + assert(listener.noLiveData()) + } } - test(s"cleanup session if exceeds the threshold (live = false)") { - val (statusStore: HiveThriftServer2AppStatusStore, - listener: HiveThriftServer2Listener) = createAppStatusStore(true) - var time = 0 - listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId1", - "user", time)) - time += 1 - listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId2", - "user", time)) - - time += 1 - listener.onOtherEvent(SparkListenerSessionClosed("sessionId1", time)) - - time += 1 - listener.onOtherEvent(SparkListenerSessionClosed("sessionId2", time)) - - listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId3", - "user", time)) - time += 1 - listener.onOtherEvent(SparkListenerSessionClosed("sessionId3", 4)) - kvstore.close(false) - assert(statusStore.getOnlineSessionNum === 0) - assert(statusStore.getSessionCount === 1) - assert(statusStore.getSession("sessionId1") === None) - assert(listener.noLiveData()) + Seq(true, false).foreach { live => + test(s"cleanup session if exceeds the threshold (live = $live)") { + val (statusStore: HiveThriftServer2AppStatusStore, + listener: HiveThriftServer2Listener) = createAppStatusStore(true) + var time = 0 + listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId1", + "user", time)) + time += 1 + listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId2", + "user", time)) + + time += 1 + listener.onOtherEvent(SparkListenerSessionClosed("sessionId1", time)) + + time += 1 + listener.onOtherEvent(SparkListenerSessionClosed("sessionId2", time)) + + listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId3", + "user", time)) + time += 1 + listener.onOtherEvent(SparkListenerSessionClosed("sessionId3", 4)) + + if (!live) { + kvstore.close(false) + } + assert(statusStore.getOnlineSessionNum === 0) + assert(statusStore.getSessionCount === 1) + assert(statusStore.getSession("sessionId1") === None) + assert(listener.noLiveData()) + } } test("update execution info when jobstart event come after execution end event") { From a55927edce083d30f8119ad9a1fdc3b1a23e6518 Mon Sep 17 00:00:00 2001 From: shahid Date: Wed, 27 Nov 2019 06:30:04 +0530 Subject: [PATCH 19/24] address comments --- .../deploy/history/FsHistoryProvider.scala | 4 ++-- .../ui/HiveThriftServer2Listener.scala | 21 +++++++++++-------- 2 files changed, 14 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 1b2f993cf3f54..b1bcde7360a0a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -366,7 +366,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) var sqlTab: Option[AppHistoryServerPlugin] = None var jdbcTab: Option[AppHistoryServerPlugin] = None - val otherTabs: Seq[AppHistoryServerPlugin] = Seq() + var otherTabs: Seq[AppHistoryServerPlugin] = Seq() plugins.foreach{ plugin => if (plugin.toString.contains("SQLHistoryServerPlugin")) { @@ -374,7 +374,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } else if (plugin.toString.contains("HiveThriftServer2HistoryServerPlugin")) { jdbcTab = Some(plugin) } else { - otherTabs :+ plugin + otherTabs = otherTabs :+ plugin } } // SQL tab should come first, then JDBC/ODBC server tab diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala index 656a48941b98c..73ac89ff6e23c 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala @@ -68,7 +68,7 @@ private[thriftserver] class HiveThriftServer2Listener( kvstore.onFlush { if (!live) { - flush(updateStore(_, trigger = true)) + flush((entity: LiveEntity) => updateStoreWithTriggerEnabled(entity)) } } @@ -101,13 +101,16 @@ private[thriftserver] class HiveThriftServer2Listener( updateLiveStore(exec) } } else { - // Here will come only if JobStart event comes after Execution End event. + // It may possible that event reordering happens such a way that JobStart event come after + // Execution end event (Refer SPARK-27019). To handle that situation, if occurs in + // Thriftserver, following code will take care. Here will come only if JobStart event comes + // after Execution End event. val storeExecInfo = kvstore.view(classOf[ExecutionInfo]).asScala.filter(_.groupId == groupId) storeExecInfo.foreach { exec => val liveExec = getOrCreateExecution(exec.execId, exec.statement, exec.sessionId, exec.startTimestamp, exec.userName) liveExec.jobId += jobId.toString - updateStore(liveExec, trigger = true) + updateStoreWithTriggerEnabled(liveExec) executionList.remove(liveExec.execId) } } @@ -136,7 +139,7 @@ private[thriftserver] class HiveThriftServer2Listener( private def onSessionClosed(e: SparkListenerSessionClosed): Unit = { val session = sessionList.get(e.sessionId) session.finishTimestamp = e.finishTime - updateStore(session, trigger = true) + updateStoreWithTriggerEnabled(session) sessionList.remove(e.sessionId) } @@ -184,14 +187,14 @@ private[thriftserver] class HiveThriftServer2Listener( private def onOperationClosed(e: SparkListenerThriftServerOperationClosed): Unit = { executionList.get(e.id).closeTimestamp = e.closeTime executionList.get(e.id).state = ExecutionState.CLOSED - updateStore(executionList.get(e.id), trigger = true) + updateStoreWithTriggerEnabled(executionList.get(e.id)) executionList.remove(e.id) } - // Update both live and history stores. If trigger is enabled, it will cleanup - // entity which exceeds the threshold. - def updateStore(entity: LiveEntity, trigger: Boolean = false): Unit = { - entity.write(kvstore, System.nanoTime(), checkTriggers = trigger) + // Update both live and history stores. Trigger is enabled by default, hence + // it will cleanup the entity which exceeds the threshold. + def updateStoreWithTriggerEnabled(entity: LiveEntity): Unit = { + entity.write(kvstore, System.nanoTime(), checkTriggers = true) } // Update only live stores. If trigger is enabled, it will cleanup entity From af65eed90db00ddac1668c6b3bd819bb93c6f68f Mon Sep 17 00:00:00 2001 From: shahid Date: Wed, 27 Nov 2019 06:39:30 +0530 Subject: [PATCH 20/24] add comment --- .../org/apache/spark/deploy/history/FsHistoryProvider.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index b1bcde7360a0a..885df84d28b79 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -361,6 +361,12 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) Some(loadedUI) } + /** + * This method is mainly to handle ordering of the tab compared to + * the Live UI. For Thriftserver applications, this method ensure + * SQL tab to come first, then JDBC/ODBC tab. + * @param ui + */ private def setupPluginUI(ui: SparkUI): Unit = { val plugins = loadPlugins().toSeq From db4269cb8c9815608e710ea7ca397d46eb0465b1 Mon Sep 17 00:00:00 2001 From: shahid Date: Wed, 27 Nov 2019 07:11:45 +0530 Subject: [PATCH 21/24] address comment --- .../ui/HiveThriftServer2EventManager.scala | 5 +++-- .../ui/HiveThriftServer2Listener.scala | 14 ++++---------- .../ui/HiveThriftServer2ListenerSuite.scala | 12 +++++++----- .../thriftserver/ui/ThriftServerPageSuite.scala | 2 +- 4 files changed, 15 insertions(+), 18 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2EventManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2EventManager.scala index 949c9166b3231..fa04c67896a69 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2EventManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2EventManager.scala @@ -35,7 +35,8 @@ private[thriftserver] class HiveThriftServer2EventManager(sc: SparkContext) { } def onSessionClosed(sessionId: String): Unit = { - postLiveListenerBus(SparkListenerSessionClosed(sessionId, System.currentTimeMillis())) + postLiveListenerBus(SparkListenerThriftServerSessionClosed(sessionId, + System.currentTimeMillis())) } def onStatementStart( @@ -77,7 +78,7 @@ private[thriftserver] case class SparkListenerThriftServerSessionCreated( userName: String, startTime: Long) extends SparkListenerEvent -private[thriftserver] case class SparkListenerSessionClosed( +private[thriftserver] case class SparkListenerThriftServerSessionClosed( sessionId: String, finishTime: Long) extends SparkListenerEvent private[thriftserver] case class SparkListenerThriftServerOperationStart( diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala index 73ac89ff6e23c..6d0a506fa94dc 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala @@ -50,7 +50,7 @@ private[thriftserver] class HiveThriftServer2Listener( // How often to update live entities. -1 means "never update" when replaying applications, // meaning only the last write will happen. For live applications, this avoids a few - // operations that we can live without when rapidly processing incoming task events. + // operations that we can live without when rapidly processing incoming events. private val liveUpdatePeriodNs = if (live) sparkConf.get(LIVE_ENTITY_UPDATE_PERIOD) else -1L // Returns true if this listener has no live data. Exposed for tests only. @@ -88,11 +88,6 @@ private[thriftserver] class HiveThriftServer2Listener( } } - /** - * This method is to handle out of order events. ie. if Job event come after execution end event. - * @param jobId - * @param groupId - */ private def updateJobDetails(jobId: String, groupId: String): Unit = { val execList = executionList.values().asScala.filter(_.groupId == groupId).toSeq if (execList.nonEmpty) { @@ -101,7 +96,7 @@ private[thriftserver] class HiveThriftServer2Listener( updateLiveStore(exec) } } else { - // It may possible that event reordering happens such a way that JobStart event come after + // It may possible that event reordering happens, such a way that JobStart event come after // Execution end event (Refer SPARK-27019). To handle that situation, if occurs in // Thriftserver, following code will take care. Here will come only if JobStart event comes // after Execution End event. @@ -119,7 +114,7 @@ private[thriftserver] class HiveThriftServer2Listener( override def onOtherEvent(event: SparkListenerEvent): Unit = { event match { case e: SparkListenerThriftServerSessionCreated => onSessionCreated(e) - case e: SparkListenerSessionClosed => onSessionClosed(e) + case e: SparkListenerThriftServerSessionClosed => onSessionClosed(e) case e: SparkListenerThriftServerOperationStart => onOperationStart(e) case e: SparkListenerThriftServerOperationParsed => onOperationParsed(e) case e: SparkListenerThriftServerOperationCanceled => onOperationCanceled(e) @@ -136,7 +131,7 @@ private[thriftserver] class HiveThriftServer2Listener( updateLiveStore(session) } - private def onSessionClosed(e: SparkListenerSessionClosed): Unit = { + private def onSessionClosed(e: SparkListenerThriftServerSessionClosed): Unit = { val session = sessionList.get(e.sessionId) session.finishTimestamp = e.finishTime updateStoreWithTriggerEnabled(session) @@ -299,7 +294,6 @@ private[thriftserver] class LiveExecutionData( } } - private[thriftserver] class LiveSessionData( val sessionId: String, val startTimeStamp: Long, diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala index 54eafb3b9b72f..e42cd65dcb878 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala @@ -65,7 +65,8 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(statusStore.getOnlineSessionNum === 1) } - listener.onOtherEvent(SparkListenerSessionClosed("sessionId", System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerSessionClosed("sessionId", + System.currentTimeMillis())) if (!live) { // To update history store @@ -96,15 +97,15 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite with BeforeAndAfter { "user", time)) time += 1 - listener.onOtherEvent(SparkListenerSessionClosed("sessionId1", time)) + listener.onOtherEvent(SparkListenerThriftServerSessionClosed("sessionId1", time)) time += 1 - listener.onOtherEvent(SparkListenerSessionClosed("sessionId2", time)) + listener.onOtherEvent(SparkListenerThriftServerSessionClosed("sessionId2", time)) listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId3", "user", time)) time += 1 - listener.onOtherEvent(SparkListenerSessionClosed("sessionId3", 4)) + listener.onOtherEvent(SparkListenerThriftServerSessionClosed("sessionId3", 4)) if (!live) { kvstore.close(false) @@ -134,7 +135,8 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite with BeforeAndAfter { System.currentTimeMillis(), Nil, createProperties)) - listener.onOtherEvent(SparkListenerSessionClosed("sessionId", System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerSessionClosed("sessionId", + System.currentTimeMillis())) val exec = statusStore.getExecution("id") assert(exec.isDefined) assert(exec.get.jobId === Seq("0")) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala index 3505e8c86ab5c..eacbcd7904ebe 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala @@ -62,7 +62,7 @@ class ThriftServerPageSuite extends SparkFunSuite with BeforeAndAfter { System.currentTimeMillis())) listener.onOtherEvent(SparkListenerThriftServerOperationClosed("id", System.currentTimeMillis())) - listener.onOtherEvent(SparkListenerSessionClosed("sessionid", System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerSessionClosed("sessionid", System.currentTimeMillis())) statusStore } From f7fcc75c8327586250d02a0c593af0e7fbd762cd Mon Sep 17 00:00:00 2001 From: shahid Date: Wed, 27 Nov 2019 20:03:47 +0530 Subject: [PATCH 22/24] scalastyle --- .../spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala index eacbcd7904ebe..9f3c2957a182d 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala @@ -62,7 +62,8 @@ class ThriftServerPageSuite extends SparkFunSuite with BeforeAndAfter { System.currentTimeMillis())) listener.onOtherEvent(SparkListenerThriftServerOperationClosed("id", System.currentTimeMillis())) - listener.onOtherEvent(SparkListenerThriftServerSessionClosed("sessionid", System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerSessionClosed("sessionid", + System.currentTimeMillis())) statusStore } From 5db0667d964bd14304187da7660b666bff336bc0 Mon Sep 17 00:00:00 2001 From: shahid Date: Thu, 28 Nov 2019 08:19:51 +0530 Subject: [PATCH 23/24] add display order. Minor refactor test. --- .../deploy/history/FsHistoryProvider.scala | 33 +++---------------- .../spark/status/AppHistoryServerPlugin.scala | 7 ++++ .../execution/ui/SQLHistoryServerPlugin.scala | 8 +++++ ...HiveThriftServer2HistoryServerPlugin.scala | 8 +++++ .../ui/HiveThriftServer2ListenerSuite.scala | 7 ++-- 5 files changed, 29 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 885df84d28b79..a3776b3ad756d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -352,7 +352,10 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) val ui = SparkUI.create(None, new HistoryAppStatusStore(conf, kvstore), conf, secManager, app.info.name, HistoryServer.getAttemptURI(appId, attempt.info.attemptId), attempt.info.startTime.getTime(), attempt.info.appSparkVersion) - setupPluginUI(ui) + + // place the tab in UI based on the display order + loadPlugins().toSeq.sortBy(_.displayOrder).foreach(_.setupUI(ui)) + val loadedUI = LoadedAppUI(ui) synchronized { activeUIs((appId, attemptId)) = loadedUI @@ -361,34 +364,6 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) Some(loadedUI) } - /** - * This method is mainly to handle ordering of the tab compared to - * the Live UI. For Thriftserver applications, this method ensure - * SQL tab to come first, then JDBC/ODBC tab. - * @param ui - */ - private def setupPluginUI(ui: SparkUI): Unit = { - val plugins = loadPlugins().toSeq - - var sqlTab: Option[AppHistoryServerPlugin] = None - var jdbcTab: Option[AppHistoryServerPlugin] = None - var otherTabs: Seq[AppHistoryServerPlugin] = Seq() - - plugins.foreach{ plugin => - if (plugin.toString.contains("SQLHistoryServerPlugin")) { - sqlTab = Some(plugin) - } else if (plugin.toString.contains("HiveThriftServer2HistoryServerPlugin")) { - jdbcTab = Some(plugin) - } else { - otherTabs = otherTabs :+ plugin - } - } - // SQL tab should come first, then JDBC/ODBC server tab - sqlTab.foreach(_.setupUI(ui)) - jdbcTab.foreach(_.setupUI(ui)) - otherTabs.foreach(_.setupUI(ui)) - } - override def getEmptyListingHtml(): Seq[Node] = {

Did you specify the correct logging directory? Please verify your setting of diff --git a/core/src/main/scala/org/apache/spark/status/AppHistoryServerPlugin.scala b/core/src/main/scala/org/apache/spark/status/AppHistoryServerPlugin.scala index d144a0e998fa1..288406b6eedfe 100644 --- a/core/src/main/scala/org/apache/spark/status/AppHistoryServerPlugin.scala +++ b/core/src/main/scala/org/apache/spark/status/AppHistoryServerPlugin.scala @@ -35,4 +35,11 @@ private[spark] trait AppHistoryServerPlugin { * Sets up UI of this plugin to rebuild the history UI. */ def setupUI(ui: SparkUI): Unit + + /** + * Order of the plugin tab that need to display in the history UI. + */ + def displayOrder: Int = { + Integer.MAX_VALUE + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLHistoryServerPlugin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLHistoryServerPlugin.scala index 522d0cf79bffa..a05104b94eb02 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLHistoryServerPlugin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLHistoryServerPlugin.scala @@ -33,4 +33,12 @@ class SQLHistoryServerPlugin extends AppHistoryServerPlugin { new SQLTab(sqlStatusStore, ui) } } + + override def displayOrder: Int = { + SQLHistoryServerPlugin.DISPLAY_ORDER + } +} + +private object SQLHistoryServerPlugin { + val DISPLAY_ORDER = 0 } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2HistoryServerPlugin.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2HistoryServerPlugin.scala index 75b223ddc5e86..5da6bce863426 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2HistoryServerPlugin.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2HistoryServerPlugin.scala @@ -34,5 +34,13 @@ class HiveThriftServer2HistoryServerPlugin extends AppHistoryServerPlugin { new ThriftServerTab(store, ui) } } + + override def displayOrder: Int = { + HiveThriftServer2HistoryServerPlugin.DISPLAY_ORDER + } +} + +private object HiveThriftServer2HistoryServerPlugin { + val DISPLAY_ORDER = 1 } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala index e42cd65dcb878..075032fa5d099 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala @@ -95,17 +95,14 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite with BeforeAndAfter { time += 1 listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId2", "user", time)) - time += 1 listener.onOtherEvent(SparkListenerThriftServerSessionClosed("sessionId1", time)) - time += 1 listener.onOtherEvent(SparkListenerThriftServerSessionClosed("sessionId2", time)) - listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId3", "user", time)) time += 1 - listener.onOtherEvent(SparkListenerThriftServerSessionClosed("sessionId3", 4)) + listener.onOtherEvent(SparkListenerThriftServerSessionClosed("sessionId3", time)) if (!live) { kvstore.close(false) @@ -152,7 +149,7 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite with BeforeAndAfter { private def createAppStatusStore(live: Boolean) = { val sparkConf = new SparkConf() sparkConf.set(ASYNC_TRACKING_ENABLED, false) - .set(SQLConf.THRIFTSERVER_UI_SESSION_LIMIT.key, "1") + .set(SQLConf.THRIFTSERVER_UI_SESSION_LIMIT, 1) .set(LIVE_ENTITY_UPDATE_PERIOD, 0L) kvstore = new ElementTrackingStore(new InMemoryStore, sparkConf) if (live) { From 39ddc891582db48a7d5e2ece771aac064cd41f1f Mon Sep 17 00:00:00 2001 From: shahid Date: Thu, 28 Nov 2019 15:54:06 +0530 Subject: [PATCH 24/24] address comments --- .../org/apache/spark/status/AppHistoryServerPlugin.scala | 6 ++---- .../spark/sql/execution/ui/SQLHistoryServerPlugin.scala | 7 +------ .../ui/HiveThriftServer2HistoryServerPlugin.scala | 8 +------- 3 files changed, 4 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/AppHistoryServerPlugin.scala b/core/src/main/scala/org/apache/spark/status/AppHistoryServerPlugin.scala index 288406b6eedfe..2e9a31d5ac69c 100644 --- a/core/src/main/scala/org/apache/spark/status/AppHistoryServerPlugin.scala +++ b/core/src/main/scala/org/apache/spark/status/AppHistoryServerPlugin.scala @@ -37,9 +37,7 @@ private[spark] trait AppHistoryServerPlugin { def setupUI(ui: SparkUI): Unit /** - * Order of the plugin tab that need to display in the history UI. + * The position of a plugin tab relative to the other plugin tabs in the history UI. */ - def displayOrder: Int = { - Integer.MAX_VALUE - } + def displayOrder: Int = Integer.MAX_VALUE } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLHistoryServerPlugin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLHistoryServerPlugin.scala index a05104b94eb02..5bf1ce5eb8a90 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLHistoryServerPlugin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLHistoryServerPlugin.scala @@ -34,11 +34,6 @@ class SQLHistoryServerPlugin extends AppHistoryServerPlugin { } } - override def displayOrder: Int = { - SQLHistoryServerPlugin.DISPLAY_ORDER - } + override def displayOrder: Int = 0 } -private object SQLHistoryServerPlugin { - val DISPLAY_ORDER = 0 -} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2HistoryServerPlugin.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2HistoryServerPlugin.scala index 5da6bce863426..aec4125801f68 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2HistoryServerPlugin.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2HistoryServerPlugin.scala @@ -35,12 +35,6 @@ class HiveThriftServer2HistoryServerPlugin extends AppHistoryServerPlugin { } } - override def displayOrder: Int = { - HiveThriftServer2HistoryServerPlugin.DISPLAY_ORDER - } -} - -private object HiveThriftServer2HistoryServerPlugin { - val DISPLAY_ORDER = 1 + override def displayOrder: Int = 1 }