You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@linkis.apache.org by ca...@apache.org on 2022/10/18 02:36:23 UTC

[incubator-linkis] branch dev-1.3.1-errorcode updated: [linkis-scheduler]module errorcode optimization and documentation (#3546)

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

casion pushed a commit to branch dev-1.3.1-errorcode
in repository https://gitbox.apache.org/repos/asf/incubator-linkis.git


The following commit(s) were added to refs/heads/dev-1.3.1-errorcode by this push:
     new 4455d5806 [linkis-scheduler]module errorcode optimization and documentation (#3546)
4455d5806 is described below

commit 4455d580639d1bff956e2aad7b73529d32a02102
Author: 成彬彬 <10...@users.noreply.github.com>
AuthorDate: Tue Oct 18 10:36:17 2022 +0800

    [linkis-scheduler]module errorcode optimization and documentation (#3546)
---
 docs/errorcode/linkis-scheduler-errorcode.md       | 11 +++
 .../errorcode/LinkisSchedulerErrorCodeSummary.java | 82 ++++++++++++++++++++++
 .../linkis/scheduler/AbstractScheduler.scala       |  9 +--
 .../scheduler/executer/AbstractExecutor.scala      | 21 ++++--
 .../linkis/scheduler/queue/SchedulerEvent.scala    |  3 +-
 .../queue/fifoqueue/FIFOConsumerManager.scala      |  3 +-
 .../queue/fifoqueue/FIFOUserConsumer.scala         |  5 +-
 .../executor/entity/AccessibleExecutor.scala       | 21 ++++--
 .../scheduler/cache/ReadCacheConsumer.scala        |  5 +-
 9 files changed, 142 insertions(+), 18 deletions(-)

diff --git a/docs/errorcode/linkis-scheduler-errorcode.md b/docs/errorcode/linkis-scheduler-errorcode.md
new file mode 100644
index 000000000..916920996
--- /dev/null
+++ b/docs/errorcode/linkis-scheduler-errorcode.md
@@ -0,0 +1,11 @@
+## linkis-scheduler   errorcode
+
+| module name(模块名) | error code(错误码)  | describe(描述) |enumeration name(枚举)| Exception Class(类名)|
+| -------- | -------- | ----- |-----|-----|
+|linkis-scheduler  |11055|The job request engine time out (任务请求引擎超时,可能是EngineConnManager 启动EngineConn失败导致,可以查看EngineConnManager对应的out和log日志).|REQUEST_ENGINE_TIME_OUT|LinkisSchedulerErrorCodeSummary|
+|linkis-scheduler  |12000|Task status flip error! Cause: Failed to flip from {} to {}.(任务状态翻转出错!原因:不允许从{} 翻转为{}.)|TASK_STATUS_FLIP_ERROR|LinkisSchedulerErrorCodeSummary|
+|linkis-scheduler  |12001|The submission job failed and the queue is full!(提交作业失败,队列已满!)|JOB_QUEUE_IS_FULL|LinkisSchedulerErrorCodeSummary|
+|linkis-scheduler  |12011|Unrecognized execId (不能识别的execId):", "Unrecognized execId (不能识别的execId):|UNRECOGNIZED_EXECID|LinkisSchedulerErrorCodeSummary|
+|linkis-scheduler  |13000|FIFOConsumerManager just support FIFO group, {} is not FIFO group.(FIFOConsumerManager只支持FIFO类型的消费组,{} 不是这类消费组.)|NEED_SUPPORTTED_GROUP|LinkisSchedulerErrorCodeSummary|
+|linkis-scheduler  |13000|{} is in state {}. ({} 处于状态 {}.)|NODE_STATE_ERROR|LinkisSchedulerErrorCodeSummary|
+
diff --git a/linkis-commons/linkis-scheduler/src/main/java/org/apache/linkis/scheduler/errorcode/LinkisSchedulerErrorCodeSummary.java b/linkis-commons/linkis-scheduler/src/main/java/org/apache/linkis/scheduler/errorcode/LinkisSchedulerErrorCodeSummary.java
new file mode 100644
index 000000000..fb5228252
--- /dev/null
+++ b/linkis-commons/linkis-scheduler/src/main/java/org/apache/linkis/scheduler/errorcode/LinkisSchedulerErrorCodeSummary.java
@@ -0,0 +1,82 @@
+/*
+ * 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.linkis.scheduler.errorcode;
+
+public enum LinkisSchedulerErrorCodeSummary {
+  REQUEST_ENGINE_TIMES_OUT(
+      11055,
+      "The job request engine time out (任务请求引擎超时,可能是EngineConnManager 启动EngineConn失败导致,可以查看EngineConnManager对应的out和log日志).",
+      "The job request engine time out (任务请求引擎超时,可能是EngineConnManager 启动EngineConn失败导致,可以查看EngineConnManager对应的out和log日志)."),
+  TASK_STATUS_FLIP_ERROR(
+      12000,
+      "Task status flip error! Cause: Failed to flip from {} to {}.(任务状态翻转出错!原因:不允许从{} 翻转为{}.)",
+      "Task status flip error! Cause: Failed to flip from {} to {}.(任务状态翻转出错!原因:不允许从{} 翻转为{}.)"),
+  JOB_QUEUE_IS_FULL(
+      12001,
+      "The submission job failed and the queue is full!(提交作业失败,队列已满!)",
+      "The submission job failed and the queue is full!(提交作业失败,队列已满!)"),
+  UNRECOGNIZED_EXECID(
+      12011, "Unrecognized execId (不能识别的execId):", "Unrecognized execId (不能识别的execId):"),
+  NEED_SUPPORTTED_GROUP(
+      13000,
+      "FIFOConsumerManager just support FIFO group, {} is not FIFO group.(FIFOConsumerManager只支持FIFO类型的消费组,{} 不是这类消费组.)",
+      "FIFOConsumerManager just support FIFO group, {} is not FIFO group.(FIFOConsumerManager只支持FIFO类型的消费组,{} 不是这类消费组.)"),
+  NODE_STATE_ERROR(20001, "{} is in state {}. ({} 处于状态 {}.)", "节点状态不符合预期");
+
+  /** (errorCode)错误码 */
+  private int errorCode;
+  /** (errorDesc)错误描述 */
+  private String errorDesc;
+  /** Possible reasons for the error(错误可能出现的原因) */
+  private String comment;
+
+  LinkisSchedulerErrorCodeSummary(int errorCode, String errorDesc, String comment) {
+    this.errorCode = errorCode;
+    this.errorDesc = errorDesc;
+    this.comment = comment;
+  }
+
+  public int getErrorCode() {
+    return errorCode;
+  }
+
+  public void setErrorCode(int errorCode) {
+    this.errorCode = errorCode;
+  }
+
+  public String getErrorDesc() {
+    return errorDesc;
+  }
+
+  public void setErrorDesc(String errorDesc) {
+    this.errorDesc = errorDesc;
+  }
+
+  public String getComment() {
+    return comment;
+  }
+
+  public void setComment(String comment) {
+    this.comment = comment;
+  }
+
+  @Override
+  public String toString() {
+    return "errorCode: " + this.errorCode + ", errorDesc:" + this.errorDesc;
+  }
+}
diff --git a/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/AbstractScheduler.scala b/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/AbstractScheduler.scala
index c34185322..de2b81bca 100644
--- a/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/AbstractScheduler.scala
+++ b/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/AbstractScheduler.scala
@@ -18,6 +18,7 @@
 package org.apache.linkis.scheduler
 
 import org.apache.linkis.common.utils.Utils
+import org.apache.linkis.scheduler.errorcode.LinkisSchedulerErrorCodeSummary._
 import org.apache.linkis.scheduler.exception.SchedulerErrorException
 import org.apache.linkis.scheduler.queue.SchedulerEvent
 
@@ -39,8 +40,8 @@ abstract class AbstractScheduler extends Scheduler {
           .contains(EVENT_ID_SPLIT) || eventId.startsWith(EVENT_ID_SPLIT)
     ) {
       throw new SchedulerErrorException(
-        12011,
-        s"Unrecognized execId $eventId.(不能识别的execId $eventId.)"
+        UNRECOGNIZED_EXECID.getErrorCode,
+        UNRECOGNIZED_EXECID.getErrorDesc + s"$eventId."
       )
     }
     val index = eventId.lastIndexOf(EVENT_ID_SPLIT)
@@ -55,8 +56,8 @@ abstract class AbstractScheduler extends Scheduler {
     index.map(getEventId(_, group.getGroupName)).foreach(event.setId)
     if (index.isEmpty) {
       throw new SchedulerErrorException(
-        12001,
-        "The submission job failed and the queue is full!(提交作业失败,队列已满!)"
+        JOB_QUEUE_IS_FULL.getErrorCode,
+        JOB_QUEUE_IS_FULL.getErrorDesc
       )
     }
   }
diff --git a/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/executer/AbstractExecutor.scala b/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/executer/AbstractExecutor.scala
index b733c5156..80bec820f 100644
--- a/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/executer/AbstractExecutor.scala
+++ b/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/executer/AbstractExecutor.scala
@@ -18,6 +18,7 @@
 package org.apache.linkis.scheduler.executer
 
 import org.apache.linkis.common.utils.{Logging, Utils}
+import org.apache.linkis.scheduler.errorcode.LinkisSchedulerErrorCodeSummary._
 import org.apache.linkis.scheduler.exception.SchedulerErrorException
 import org.apache.linkis.scheduler.executer.ExecutorState._
 import org.apache.linkis.scheduler.listener.ExecutorListener
@@ -48,7 +49,10 @@ abstract class AbstractExecutor(id: Long) extends Executor with Logging {
     if (_state == Busy) synchronized {
       if (_state == Busy) return f
     }
-    throw new SchedulerErrorException(20001, "%s is in state %s." format (toString, _state))
+    throw new SchedulerErrorException(
+      NODE_STATE_ERROR.getErrorCode,
+      "%s is in state %s." format (toString, _state)
+    )
   }
 
   protected def ensureIdle[A](f: => A): A = ensureIdle(f, true)
@@ -63,19 +67,28 @@ abstract class AbstractExecutor(id: Long) extends Executor with Logging {
         }
       }
     }
-    throw new SchedulerErrorException(20001, "%s is in state %s." format (toString, _state))
+    throw new SchedulerErrorException(
+      NODE_STATE_ERROR.getErrorCode,
+      "%s is in state %s." format (toString, _state)
+    )
   }
 
   protected def ensureAvailable[A](f: => A): A = {
     if (ExecutorState.isAvailable(_state)) synchronized {
       if (ExecutorState.isAvailable(_state)) return Utils.tryFinally(f)(callback())
     }
-    throw new SchedulerErrorException(20001, "%s is in state %s." format (toString, _state))
+    throw new SchedulerErrorException(
+      NODE_STATE_ERROR.getErrorCode,
+      "%s is in state %s." format (toString, _state)
+    )
   }
 
   protected def whenAvailable[A](f: => A): A = {
     if (ExecutorState.isAvailable(_state)) return Utils.tryFinally(f)(callback())
-    throw new SchedulerErrorException(20001, "%s is in state %s." format (toString, _state))
+    throw new SchedulerErrorException(
+      NODE_STATE_ERROR.getErrorCode,
+      "%s is in state %s." format (toString, _state)
+    )
   }
 
   protected def transition(state: ExecutorState) = this synchronized {
diff --git a/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/SchedulerEvent.scala b/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/SchedulerEvent.scala
index 007def269..ee2f4f5f1 100644
--- a/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/SchedulerEvent.scala
+++ b/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/SchedulerEvent.scala
@@ -18,6 +18,7 @@
 package org.apache.linkis.scheduler.queue
 
 import org.apache.linkis.common.utils.Logging
+import org.apache.linkis.scheduler.errorcode.LinkisSchedulerErrorCodeSummary._
 import org.apache.linkis.scheduler.exception.SchedulerErrorException
 import org.apache.linkis.scheduler.queue.SchedulerEventState._
 
@@ -87,7 +88,7 @@ trait SchedulerEvent extends Logging {
   protected def transition(state: SchedulerEventState): Unit = synchronized {
     if (state.id < this.state.id && state != WaitForRetry) {
       throw new SchedulerErrorException(
-        12000,
+        TASK_STATUS_FLIP_ERROR.getErrorCode,
         s"Task status flip error! Cause: Failed to flip from ${this.state} to $state.(任务状态翻转出错!原因:不允许从${this.state} 翻转为$state.)"
       ) // 抛异常
     }
diff --git a/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/fifoqueue/FIFOConsumerManager.scala b/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/fifoqueue/FIFOConsumerManager.scala
index a681bbd0d..17410b229 100644
--- a/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/fifoqueue/FIFOConsumerManager.scala
+++ b/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/fifoqueue/FIFOConsumerManager.scala
@@ -19,6 +19,7 @@ package org.apache.linkis.scheduler.queue.fifoqueue
 
 import org.apache.linkis.common.utils.Utils
 import org.apache.linkis.scheduler.SchedulerContext
+import org.apache.linkis.scheduler.errorcode.LinkisSchedulerErrorCodeSummary._
 import org.apache.linkis.scheduler.exception.SchedulerErrorException
 import org.apache.linkis.scheduler.listener.ConsumerListener
 import org.apache.linkis.scheduler.queue.{Consumer, ConsumerManager, Group, LoopArrayQueue}
@@ -43,7 +44,7 @@ class FIFOConsumerManager(groupName: String) extends ConsumerManager {
         Utils.newCachedThreadPool(g.getMaxRunningJobs + 2, groupName + "-Thread-")
       case _ =>
         throw new SchedulerErrorException(
-          13000,
+          NEED_SUPPORTTED_GROUP.getErrorCode,
           s"FIFOConsumerManager need a FIFOGroup, but ${group.getClass} is supported."
         )
     }
diff --git a/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/fifoqueue/FIFOUserConsumer.scala b/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/fifoqueue/FIFOUserConsumer.scala
index 787c5aa06..926d7ba17 100644
--- a/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/fifoqueue/FIFOUserConsumer.scala
+++ b/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/fifoqueue/FIFOUserConsumer.scala
@@ -21,6 +21,7 @@ import org.apache.linkis.common.exception.{ErrorException, WarnException}
 import org.apache.linkis.common.log.LogUtils
 import org.apache.linkis.common.utils.Utils
 import org.apache.linkis.scheduler.SchedulerContext
+import org.apache.linkis.scheduler.errorcode.LinkisSchedulerErrorCodeSummary._
 import org.apache.linkis.scheduler.exception.SchedulerErrorException
 import org.apache.linkis.scheduler.executer.Executor
 import org.apache.linkis.scheduler.future.{BDPFuture, BDPFutureTask}
@@ -168,8 +169,8 @@ class FIFOUserConsumer(
           job.onFailure(
             "The request engine times out (请求引擎超时,可能是EngineConnManager 启动EngineConn失败导致,可以去查看看EngineConnManager的linkis.out和linkis.log日志).",
             new SchedulerErrorException(
-              11055,
-              "The request engine times out (请求引擎超时,可能是EngineConnManager 启动EngineConn失败导致,可以去观看EngineConnManager的linkis.out和linkis.log日志)."
+              REQUEST_ENGINE_TIMES_OUT.getErrorCode,
+              REQUEST_ENGINE_TIMES_OUT.getErrorDesc
             )
           )
         case error: Throwable =>
diff --git a/linkis-computation-governance/linkis-engineconn/linkis-engineconn-executor/accessible-executor/src/main/scala/org/apache/linkis/engineconn/acessible/executor/entity/AccessibleExecutor.scala b/linkis-computation-governance/linkis-engineconn/linkis-engineconn-executor/accessible-executor/src/main/scala/org/apache/linkis/engineconn/acessible/executor/entity/AccessibleExecutor.scala
index 2bafbce30..c72847a38 100644
--- a/linkis-computation-governance/linkis-engineconn/linkis-engineconn-executor/accessible-executor/src/main/scala/org/apache/linkis/engineconn/acessible/executor/entity/AccessibleExecutor.scala
+++ b/linkis-computation-governance/linkis-engineconn/linkis-engineconn-executor/accessible-executor/src/main/scala/org/apache/linkis/engineconn/acessible/executor/entity/AccessibleExecutor.scala
@@ -25,6 +25,7 @@ import org.apache.linkis.engineconn.acessible.executor.listener.event.{
 import org.apache.linkis.engineconn.executor.entity.SensibleExecutor
 import org.apache.linkis.engineconn.executor.listener.ExecutorListenerBusContext
 import org.apache.linkis.manager.common.entity.enumeration.NodeStatus
+import org.apache.linkis.scheduler.errorcode.LinkisSchedulerErrorCodeSummary._
 import org.apache.linkis.scheduler.exception.SchedulerErrorException
 
 abstract class AccessibleExecutor extends SensibleExecutor {
@@ -46,7 +47,10 @@ abstract class AccessibleExecutor extends SensibleExecutor {
     if (isBusy) synchronized {
       if (isBusy) return f
     }
-    throw new SchedulerErrorException(20001, "%s is in status %s." format (toString, getStatus))
+    throw new SchedulerErrorException(
+      NODE_STATE_ERROR.getErrorCode,
+      "%s is in status %s." format (toString, getStatus)
+    )
   }
 
   def ensureIdle[A](f: => A): A = ensureIdle(f, true)
@@ -61,19 +65,28 @@ abstract class AccessibleExecutor extends SensibleExecutor {
         }
       }
     }
-    throw new SchedulerErrorException(20001, "%s is in status %s." format (toString, getStatus))
+    throw new SchedulerErrorException(
+      NODE_STATE_ERROR.getErrorCode,
+      "%s is in status %s." format (toString, getStatus)
+    )
   }
 
   def ensureAvailable[A](f: => A): A = {
     if (NodeStatus.isAvailable(getStatus)) synchronized {
       if (NodeStatus.isAvailable(getStatus)) return Utils.tryFinally(f)(callback())
     }
-    throw new SchedulerErrorException(20001, "%s is in status %s." format (toString, getStatus))
+    throw new SchedulerErrorException(
+      NODE_STATE_ERROR.getErrorCode,
+      "%s is in status %s." format (toString, getStatus)
+    )
   }
 
   def whenAvailable[A](f: => A): A = {
     if (NodeStatus.isAvailable(getStatus)) return Utils.tryFinally(f)(callback())
-    throw new SchedulerErrorException(20001, "%s is in status %s." format (toString, getStatus))
+    throw new SchedulerErrorException(
+      NODE_STATE_ERROR.getErrorCode,
+      "%s is in status %s." format (toString, getStatus)
+    )
   }
 
   protected def callback(): Unit
diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/cache/ReadCacheConsumer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/cache/ReadCacheConsumer.scala
index 50318e563..8cbfc9a98 100644
--- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/cache/ReadCacheConsumer.scala
+++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/cache/ReadCacheConsumer.scala
@@ -29,6 +29,7 @@ import org.apache.linkis.manager.label.constant.LabelKeyConstant
 import org.apache.linkis.protocol.constants.TaskConstant
 import org.apache.linkis.protocol.utils.TaskUtils
 import org.apache.linkis.scheduler.SchedulerContext
+import org.apache.linkis.scheduler.errorcode.LinkisSchedulerErrorCodeSummary._
 import org.apache.linkis.scheduler.exception.SchedulerErrorException
 import org.apache.linkis.scheduler.executer.SuccessExecuteResponse
 import org.apache.linkis.scheduler.queue.Group
@@ -148,8 +149,8 @@ class ReadCacheConsumer(
     // index.map(getEventId(_, groupName)).foreach(job.setId)
     if (index.isEmpty) {
       throw new SchedulerErrorException(
-        12001,
-        "The submission job failed and the queue is full!(提交作业失败,队列已满!)"
+        JOB_QUEUE_IS_FULL.getErrorCode,
+        JOB_QUEUE_IS_FULL.getErrorDesc
       )
     }
   }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@linkis.apache.org
For additional commands, e-mail: commits-help@linkis.apache.org