-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-44394][CONNECT][WEBUI] Add a Spark UI page for Spark Connect #41964
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
Changes from all commits
Commits
Show all changes
4 commits
Select commit
Hold shift + click to select a range
cf3a49e
Add a Spark UI page for Spark Connect
jasonli-db 5280131
Address comments and fix tests
jasonli-db 8abdecd
Show RUNNING as the state in the UI if the execution is active
jasonli-db c44bc52
Address issue where handler can be attached more than once
jasonli-db File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
131 changes: 131 additions & 0 deletions
131
...ver/src/main/scala/org/apache/spark/sql/connect/ui/SparkConnectServerAppStatusStore.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,131 @@ | ||
| /* | ||
| * 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.connect.ui | ||
|
|
||
| import scala.collection.mutable | ||
| import scala.collection.mutable.ArrayBuffer | ||
|
|
||
| import com.fasterxml.jackson.annotation.JsonIgnore | ||
|
|
||
| import org.apache.spark.status.KVUtils | ||
| import org.apache.spark.status.KVUtils.KVIndexParam | ||
| import org.apache.spark.util.kvstore.{KVIndex, KVStore} | ||
|
|
||
| class SparkConnectServerAppStatusStore(store: KVStore) { | ||
| def getSessionList: Seq[SessionInfo] = { | ||
| KVUtils.viewToSeq(store.view(classOf[SessionInfo])) | ||
| } | ||
|
|
||
| def getExecutionList: Seq[ExecutionInfo] = { | ||
| KVUtils.viewToSeq(store.view(classOf[ExecutionInfo])) | ||
| } | ||
|
|
||
| def getOnlineSessionNum: Int = { | ||
| KVUtils.count(store.view(classOf[SessionInfo]))(_.finishTimestamp == 0) | ||
| } | ||
|
|
||
| def getSession(sessionId: String): Option[SessionInfo] = { | ||
| try { | ||
| Some(store.read(classOf[SessionInfo], sessionId)) | ||
| } catch { | ||
| 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 | ||
| * cancellations and count all statements that have not been closed so far. | ||
| */ | ||
| def getTotalRunning: Int = { | ||
| KVUtils.count(store.view(classOf[ExecutionInfo]))(_.isExecutionActive) | ||
| } | ||
|
|
||
| def getSessionCount: Long = { | ||
| store.count(classOf[SessionInfo]) | ||
| } | ||
|
|
||
| def getExecutionCount: Long = { | ||
| store.count(classOf[ExecutionInfo]) | ||
| } | ||
| } | ||
|
|
||
| private[connect] class SessionInfo( | ||
| @KVIndexParam val sessionId: String, | ||
| val startTimestamp: Long, | ||
| val userId: 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 | ||
| } else { | ||
| finishTimestamp - startTimestamp | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private[connect] class ExecutionInfo( | ||
| @KVIndexParam val jobTag: String, | ||
| val statement: String, | ||
| val sessionId: String, | ||
| val startTimestamp: Long, | ||
| val userId: String, | ||
| val operationId: String, | ||
| val sparkSessionTags: Set[String], | ||
| val finishTimestamp: Long, | ||
| val closeTimestamp: Long, | ||
| val detail: String, | ||
| val state: ExecutionState.Value, | ||
| val jobId: ArrayBuffer[String], | ||
| val sqlExecId: mutable.Set[String]) { | ||
| @JsonIgnore @KVIndex("finishTime") | ||
| private def finishTimeIndex: Long = if (finishTimestamp > 0L && !isExecutionActive) { | ||
| finishTimestamp | ||
| } else -1L | ||
|
|
||
| @JsonIgnore @KVIndex("isExecutionActive") | ||
| def isExecutionActive: Boolean = { | ||
| state == ExecutionState.STARTED || | ||
| state == ExecutionState.COMPILED || | ||
| state == ExecutionState.READY | ||
| } | ||
|
|
||
| def totalTime(endTime: Long): Long = { | ||
| if (endTime == 0L) { | ||
| System.currentTimeMillis - startTimestamp | ||
| } else { | ||
| endTime - startTimestamp | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private[connect] object ExecutionState extends Enumeration { | ||
| val STARTED, COMPILED, READY, CANCELED, FAILED, FINISHED, CLOSED = Value | ||
| type ExecutionState = Value | ||
| } | ||
jasonli-db marked this conversation as resolved.
Show resolved
Hide resolved
|
||
41 changes: 41 additions & 0 deletions
41
...rc/main/scala/org/apache/spark/sql/connect/ui/SparkConnectServerHistoryServerPlugin.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,41 @@ | ||
| /* | ||
| * 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.connect.ui | ||
|
|
||
| import org.apache.spark.SparkConf | ||
| import org.apache.spark.scheduler.SparkListener | ||
| import org.apache.spark.status.{AppHistoryServerPlugin, ElementTrackingStore} | ||
| import org.apache.spark.ui.SparkUI | ||
|
|
||
| class SparkConnectServerHistoryServerPlugin extends AppHistoryServerPlugin { | ||
|
|
||
| override def createListeners( | ||
| conf: SparkConf, | ||
| store: ElementTrackingStore): Seq[SparkListener] = { | ||
| Seq(new SparkConnectServerListener(store, conf)) | ||
| } | ||
|
|
||
| override def setupUI(ui: SparkUI): Unit = { | ||
| val store = new SparkConnectServerAppStatusStore(ui.store.store) | ||
| if (store.getSessionCount > 0) { | ||
| new SparkConnectServerTab(store, ui) | ||
| } | ||
| } | ||
|
|
||
| override def displayOrder: Int = 3 | ||
| } |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.