You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kyuubi.apache.org by fe...@apache.org on 2023/03/07 07:43:31 UTC

[kyuubi] branch master updated: [KYUUBI #4439][FOLLOWUP] Add dto class for operation data

This is an automated email from the ASF dual-hosted git repository.

feiwang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kyuubi.git


The following commit(s) were added to refs/heads/master by this push:
     new 222a3345f [KYUUBI #4439][FOLLOWUP] Add dto class for operation data
222a3345f is described below

commit 222a3345f2d615df3c32270c5a546db0f5f4a23c
Author: fwang12 <fw...@ebay.com>
AuthorDate: Tue Mar 7 15:43:23 2023 +0800

    [KYUUBI #4439][FOLLOWUP] Add dto class for operation data
    
    ### _Why are the changes needed?_
    
    - add dto for operation data, which is programming friendly
    - show exception for session data
    
    ### _How was this patch tested?_
    - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible
    
    - [ ] Add screenshots for manual tests if appropriate
    
    - [x] [Run test](https://kyuubi.readthedocs.io/en/master/develop_tools/testing.html#running-tests) locally before make a pull request
    
    Closes #4468 from turboFei/session_op_dto.
    
    Closes #4439
    
    fa905e70c [fwang12] fix ut
    5c1c7c845 [fwang12] save
    2d20215a0 [fwang12] comments
    46cd2384e [fwang12] saev
    
    Authored-by: fwang12 <fw...@ebay.com>
    Signed-off-by: fwang12 <fw...@ebay.com>
---
 .../src/main/scala/org/apache/kyuubi/Utils.scala   |  15 +++
 .../dto/{SessionData.java => OperationData.java}   | 122 +++++++++++++--------
 .../kyuubi/client/api/v1/dto/SessionData.java      |  13 ++-
 .../org/apache/kyuubi/server/api/ApiUtils.scala    |  56 ++++++++++
 .../kyuubi/server/api/v1/AdminResource.scala       |  26 ++---
 .../kyuubi/server/api/v1/SessionsResource.scala    |  13 +--
 .../kyuubi/server/api/v1/AdminResourceSuite.scala  |  11 +-
 7 files changed, 175 insertions(+), 81 deletions(-)

diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/Utils.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/Utils.scala
index 7ab312fa1..e4fb23936 100644
--- a/kyuubi-common/src/main/scala/org/apache/kyuubi/Utils.scala
+++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/Utils.scala
@@ -387,4 +387,19 @@ object Utils extends Logging {
     Option(Thread.currentThread().getContextClassLoader).getOrElse(getKyuubiClassLoader)
 
   def isOnK8s: Boolean = Files.exists(Paths.get("/var/run/secrets/kubernetes.io"))
+
+  /**
+   * Return a nice string representation of the exception. It will call "printStackTrace" to
+   * recursively generate the stack trace including the exception and its causes.
+   */
+  def prettyPrint(e: Throwable): String = {
+    if (e == null) {
+      ""
+    } else {
+      // Use e.printStackTrace here because e.getStackTrace doesn't include the cause
+      val stringWriter = new StringWriter()
+      e.printStackTrace(new PrintWriter(stringWriter))
+      stringWriter.toString
+    }
+  }
 }
diff --git a/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/SessionData.java b/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/OperationData.java
similarity index 51%
copy from kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/SessionData.java
copy to kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/OperationData.java
index bae6f39da..bc7af4ac9 100644
--- a/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/SessionData.java
+++ b/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/OperationData.java
@@ -17,38 +17,45 @@
 
 package org.apache.kyuubi.client.api.v1.dto;
 
-import java.util.Collections;
-import java.util.Map;
 import java.util.Objects;
 import org.apache.commons.lang3.builder.ReflectionToStringBuilder;
 import org.apache.commons.lang3.builder.ToStringStyle;
 
-public class SessionData {
+public class OperationData {
   private String identifier;
-  private String user;
-  private String ipAddr;
-  private Map<String, String> conf;
+  private String statement;
+  private String state;
   private Long createTime;
-  private Long duration;
-  private Long idleTime;
+  private Long startTime;
+  private Long completeTime;
+  private String exception;
+  private String sessionId;
+  private String sessionUser;
+  private String sessionType;
 
-  public SessionData() {}
+  public OperationData() {}
 
-  public SessionData(
+  public OperationData(
       String identifier,
-      String user,
-      String ipAddr,
-      Map<String, String> conf,
+      String statement,
+      String state,
       Long createTime,
-      Long duration,
-      Long idleTime) {
+      Long startTime,
+      Long completeTime,
+      String exception,
+      String sessionId,
+      String sessionUser,
+      String sessionType) {
     this.identifier = identifier;
-    this.user = user;
-    this.ipAddr = ipAddr;
-    this.conf = conf;
+    this.statement = statement;
+    this.state = state;
     this.createTime = createTime;
-    this.duration = duration;
-    this.idleTime = idleTime;
+    this.startTime = startTime;
+    this.completeTime = completeTime;
+    this.exception = exception;
+    this.sessionId = sessionId;
+    this.sessionUser = sessionUser;
+    this.sessionType = sessionType;
   }
 
   public String getIdentifier() {
@@ -59,31 +66,20 @@ public class SessionData {
     this.identifier = identifier;
   }
 
-  public String getUser() {
-    return user;
+  public String getStatement() {
+    return statement;
   }
 
-  public void setUser(String user) {
-    this.user = user;
+  public void setStatement(String statement) {
+    this.statement = statement;
   }
 
-  public String getIpAddr() {
-    return ipAddr;
+  public String getState() {
+    return state;
   }
 
-  public void setIpAddr(String ipAddr) {
-    this.ipAddr = ipAddr;
-  }
-
-  public Map<String, String> getConf() {
-    if (null == conf) {
-      return Collections.emptyMap();
-    }
-    return conf;
-  }
-
-  public void setConf(Map<String, String> conf) {
-    this.conf = conf;
+  public void setState(String state) {
+    this.state = state;
   }
 
   public Long getCreateTime() {
@@ -94,20 +90,52 @@ public class SessionData {
     this.createTime = createTime;
   }
 
-  public Long getDuration() {
-    return duration;
+  public Long getStartTime() {
+    return startTime;
+  }
+
+  public void setStartTime(Long startTime) {
+    this.startTime = startTime;
+  }
+
+  public Long getCompleteTime() {
+    return completeTime;
+  }
+
+  public void setCompleteTime(Long completeTime) {
+    this.completeTime = completeTime;
+  }
+
+  public String getException() {
+    return exception;
+  }
+
+  public void setException(String exception) {
+    this.exception = exception;
+  }
+
+  public String getSessionId() {
+    return sessionId;
+  }
+
+  public void setSessionId(String sessionId) {
+    this.sessionId = sessionId;
+  }
+
+  public String getSessionUser() {
+    return sessionUser;
   }
 
-  public void setDuration(Long duration) {
-    this.duration = duration;
+  public void setSessionUser(String sessionUser) {
+    this.sessionUser = sessionUser;
   }
 
-  public Long getIdleTime() {
-    return idleTime;
+  public String getSessionType() {
+    return sessionType;
   }
 
-  public void setIdleTime(Long idleTime) {
-    this.idleTime = idleTime;
+  public void setSessionType(String sessionType) {
+    this.sessionType = sessionType;
   }
 
   @Override
diff --git a/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/SessionData.java b/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/SessionData.java
index bae6f39da..233fee721 100644
--- a/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/SessionData.java
+++ b/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/SessionData.java
@@ -31,6 +31,7 @@ public class SessionData {
   private Long createTime;
   private Long duration;
   private Long idleTime;
+  private String exception;
 
   public SessionData() {}
 
@@ -41,7 +42,8 @@ public class SessionData {
       Map<String, String> conf,
       Long createTime,
       Long duration,
-      Long idleTime) {
+      Long idleTime,
+      String exception) {
     this.identifier = identifier;
     this.user = user;
     this.ipAddr = ipAddr;
@@ -49,6 +51,7 @@ public class SessionData {
     this.createTime = createTime;
     this.duration = duration;
     this.idleTime = idleTime;
+    this.exception = exception;
   }
 
   public String getIdentifier() {
@@ -110,6 +113,14 @@ public class SessionData {
     this.idleTime = idleTime;
   }
 
+  public String getException() {
+    return exception;
+  }
+
+  public void setException(String exception) {
+    this.exception = exception;
+  }
+
   @Override
   public boolean equals(Object o) {
     if (this == o) return true;
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/ApiUtils.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/ApiUtils.scala
new file mode 100644
index 000000000..dbdd34ead
--- /dev/null
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/ApiUtils.scala
@@ -0,0 +1,56 @@
+/*
+ * 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.kyuubi.server.api
+
+import scala.collection.JavaConverters._
+
+import org.apache.kyuubi.Utils
+import org.apache.kyuubi.client.api.v1.dto.{OperationData, SessionData}
+import org.apache.kyuubi.events.KyuubiOperationEvent
+import org.apache.kyuubi.operation.KyuubiOperation
+import org.apache.kyuubi.session.KyuubiSession
+
+object ApiUtils {
+
+  def sessionData(session: KyuubiSession): SessionData = {
+    new SessionData(
+      session.handle.identifier.toString,
+      session.user,
+      session.ipAddress,
+      session.conf.asJava,
+      session.createTime,
+      session.lastAccessTime - session.createTime,
+      session.getNoOperationTime,
+      session.getSessionEvent.flatMap(_.exception).map(Utils.prettyPrint).getOrElse(""))
+  }
+
+  def operationData(operation: KyuubiOperation): OperationData = {
+    val opEvent = KyuubiOperationEvent(operation)
+    new OperationData(
+      opEvent.statementId,
+      opEvent.statement,
+      opEvent.state,
+      opEvent.createTime,
+      opEvent.startTime,
+      opEvent.completeTime,
+      opEvent.exception.map(Utils.prettyPrint).getOrElse(""),
+      opEvent.sessionId,
+      opEvent.sessionUser,
+      opEvent.sessionType)
+  }
+}
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/AdminResource.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/AdminResource.scala
index ceb7179b8..4418bc8d6 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/AdminResource.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/AdminResource.scala
@@ -30,17 +30,16 @@ import io.swagger.v3.oas.annotations.tags.Tag
 import org.apache.zookeeper.KeeperException.NoNodeException
 
 import org.apache.kyuubi.{KYUUBI_VERSION, Logging, Utils}
-import org.apache.kyuubi.client.api.v1.dto.{Engine, SessionData}
+import org.apache.kyuubi.client.api.v1.dto.{Engine, OperationData, SessionData}
 import org.apache.kyuubi.config.KyuubiConf
 import org.apache.kyuubi.config.KyuubiConf._
-import org.apache.kyuubi.events.KyuubiOperationEvent
 import org.apache.kyuubi.ha.HighAvailabilityConf.HA_NAMESPACE
 import org.apache.kyuubi.ha.client.{DiscoveryPaths, ServiceNodeInfo}
 import org.apache.kyuubi.ha.client.DiscoveryClientProvider.withDiscoveryClient
 import org.apache.kyuubi.operation.{KyuubiOperation, OperationHandle}
 import org.apache.kyuubi.server.KyuubiServer
-import org.apache.kyuubi.server.api.ApiRequestContext
-import org.apache.kyuubi.session.SessionHandle
+import org.apache.kyuubi.server.api.{ApiRequestContext, ApiUtils}
+import org.apache.kyuubi.session.{KyuubiSession, SessionHandle}
 
 @Tag(name = "Admin")
 @Produces(Array(MediaType.APPLICATION_JSON))
@@ -122,15 +121,8 @@ private[v1] class AdminResource extends ApiRequestContext with Logging {
       throw new NotAllowedException(
         s"$userName is not allowed to list all live sessions")
     }
-    fe.be.sessionManager.allSessions().map { session =>
-      new SessionData(
-        session.handle.identifier.toString,
-        session.user,
-        session.ipAddress,
-        session.conf.asJava,
-        session.createTime,
-        session.lastAccessTime - session.createTime,
-        session.getNoOperationTime)
+    fe.be.sessionManager.allSessions().map { case session =>
+      ApiUtils.sessionData(session.asInstanceOf[KyuubiSession])
     }.toSeq
   }
 
@@ -157,12 +149,12 @@ private[v1] class AdminResource extends ApiRequestContext with Logging {
     content = Array(new Content(
       mediaType = MediaType.APPLICATION_JSON,
       array = new ArraySchema(schema = new Schema(implementation =
-        classOf[KyuubiOperationEvent])))),
+        classOf[OperationData])))),
     description =
-      "get the list of all active operation events")
+      "get the list of all active operations")
   @GET
   @Path("operations")
-  def listOperations(): Seq[KyuubiOperationEvent] = {
+  def listOperations(): Seq[OperationData] = {
     val userName = fe.getSessionUser(Map.empty[String, String])
     val ipAddress = fe.getIpAddress
     info(s"Received listing all of the active operations request from $userName/$ipAddress")
@@ -171,7 +163,7 @@ private[v1] class AdminResource extends ApiRequestContext with Logging {
         s"$userName is not allowed to list all the operations")
     }
     fe.be.sessionManager.operationManager.allOperations()
-      .map(operation => KyuubiOperationEvent(operation.asInstanceOf[KyuubiOperation])).toSeq
+      .map(operation => ApiUtils.operationData(operation.asInstanceOf[KyuubiOperation])).toSeq
   }
 
   @ApiResponse(
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/SessionsResource.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/SessionsResource.scala
index 84b19eb00..600ac3c87 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/SessionsResource.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/SessionsResource.scala
@@ -35,7 +35,7 @@ import org.apache.kyuubi.client.api.v1.dto._
 import org.apache.kyuubi.config.KyuubiReservedKeys._
 import org.apache.kyuubi.events.KyuubiEvent
 import org.apache.kyuubi.operation.OperationHandle
-import org.apache.kyuubi.server.api.ApiRequestContext
+import org.apache.kyuubi.server.api.{ApiRequestContext, ApiUtils}
 import org.apache.kyuubi.session.KyuubiSession
 import org.apache.kyuubi.session.SessionHandle
 
@@ -54,15 +54,8 @@ private[v1] class SessionsResource extends ApiRequestContext with Logging {
     description = "get the list of all live sessions")
   @GET
   def sessions(): Seq[SessionData] = {
-    sessionManager.allSessions().map { session =>
-      new SessionData(
-        session.handle.identifier.toString,
-        session.user,
-        session.ipAddress,
-        session.conf.asJava,
-        session.createTime,
-        session.lastAccessTime - session.createTime,
-        session.getNoOperationTime)
+    sessionManager.allSessions().map { case session =>
+      ApiUtils.sessionData(session.asInstanceOf[KyuubiSession])
     }.toSeq
   }
 
diff --git a/kyuubi-server/src/test/scala/org/apache/kyuubi/server/api/v1/AdminResourceSuite.scala b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/api/v1/AdminResourceSuite.scala
index 8aaf6c512..0fc912e7a 100644
--- a/kyuubi-server/src/test/scala/org/apache/kyuubi/server/api/v1/AdminResourceSuite.scala
+++ b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/api/v1/AdminResourceSuite.scala
@@ -27,13 +27,12 @@ import org.apache.hive.service.rpc.thrift.TProtocolVersion.HIVE_CLI_SERVICE_PROT
 import org.scalatest.time.SpanSugar.convertIntToGrainOfTime
 
 import org.apache.kyuubi.{KYUUBI_VERSION, KyuubiFunSuite, RestFrontendTestHelper, Utils}
-import org.apache.kyuubi.client.api.v1.dto.{Engine, SessionData, SessionHandle, SessionOpenRequest}
+import org.apache.kyuubi.client.api.v1.dto.{Engine, OperationData, SessionData, SessionHandle, SessionOpenRequest}
 import org.apache.kyuubi.config.KyuubiConf
 import org.apache.kyuubi.config.KyuubiReservedKeys.KYUUBI_SESSION_CONNECTION_URL_KEY
 import org.apache.kyuubi.engine.{ApplicationState, EngineRef, KyuubiApplicationManager}
 import org.apache.kyuubi.engine.EngineType.SPARK_SQL
 import org.apache.kyuubi.engine.ShareLevel.{CONNECTION, USER}
-import org.apache.kyuubi.events.KyuubiOperationEvent
 import org.apache.kyuubi.ha.HighAvailabilityConf
 import org.apache.kyuubi.ha.client.DiscoveryClientProvider.withDiscoveryClient
 import org.apache.kyuubi.ha.client.DiscoveryPaths
@@ -189,9 +188,9 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper {
       .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization")
       .get()
     assert(200 == response.getStatus)
-    var operations = response.readEntity(new GenericType[Seq[KyuubiOperationEvent]]() {})
+    var operations = response.readEntity(new GenericType[Seq[OperationData]]() {})
     assert(operations.nonEmpty)
-    assert(operations.map(op => op.statementId).contains(operation.identifier.toString))
+    assert(operations.map(op => op.getIdentifier).contains(operation.identifier.toString))
 
     // close operation
     response = webTarget.path(s"api/v1/admin/operations/${operation.identifier}").request()
@@ -203,8 +202,8 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper {
     response = webTarget.path("api/v1/admin/operations").request()
       .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization")
       .get()
-    operations = response.readEntity(new GenericType[Seq[KyuubiOperationEvent]]() {})
-    assert(!operations.map(op => op.statementId).contains(operation.identifier.toString))
+    operations = response.readEntity(new GenericType[Seq[OperationData]]() {})
+    assert(!operations.map(op => op.getIdentifier).contains(operation.identifier.toString))
   }
 
   test("delete engine - user share level") {