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:29:05 UTC

[incubator-linkis] branch dev-1.3.1-errorcode updated: [linkis-manager-common ]module errorcode optimization and documentation (#3591)

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 8f01b37c7 [linkis-manager-common ]module errorcode optimization and documentation (#3591)
8f01b37c7 is described below

commit 8f01b37c7ba4f8ab55b58feb9cc640881733110e
Author: 成彬彬 <10...@users.noreply.github.com>
AuthorDate: Tue Oct 18 10:29:00 2022 +0800

    [linkis-manager-common ]module errorcode optimization and documentation (#3591)
---
 docs/errorcode/linkis-manager-common-errorcode.md  |  23 ++++
 .../service/impl/ExternalResourceServiceImpl.java  |  20 +--
 .../rm/external/yarn/YarnResourceRequester.scala   |  25 ++--
 .../linkis/manager/rm/restful/RMMonitorRest.scala  |   5 +-
 .../rm/service/RequestResourceService.scala        |  19 ++-
 .../rm/service/impl/DefaultResourceManager.scala   |   9 +-
 .../manager/rm/utils/UserConfiguration.scala       |   7 +-
 .../errorcode/ManagerCommonErrorCodeSummary.java   | 137 +++++++++++++++++++++
 .../manager/common/entity/resource/Resource.scala  |  81 +++++++++---
 9 files changed, 280 insertions(+), 46 deletions(-)

diff --git a/docs/errorcode/linkis-manager-common-errorcode.md b/docs/errorcode/linkis-manager-common-errorcode.md
new file mode 100644
index 000000000..2a513e918
--- /dev/null
+++ b/docs/errorcode/linkis-manager-common-errorcode.md
@@ -0,0 +1,23 @@
+## linkis-manager-common  errorcode
+
+| module name(模块名) | error code(错误码)  | describe(描述) |enumeration name(枚举)| Exception Class(类名)|
+| -------- | -------- | ----- |-----|-----|
+|linkis-manager-common |10022|The tag resource was created later than the used resource was created(无需清理该标签的资源,该标签资源的创建时间晚于已用资源的创建时间)|RESOURCE_LATER_CREATED|ManagerCommonErrorCodeSummary|
+|linkis-manager-common |11002|Unsupported operation: multiplied(不支持的操作:multiplied)|OPERATION_MULTIPLIED|ManagerCommonErrorCodeSummary|
+|linkis-manager-common |11003|Not supported resource result policy (不支持的资源结果策略)|NOT_RESOURCE_POLICY|ManagerCommonErrorCodeSummary|
+|linkis-manager-common |11003|Not supported resource result type(不支持的资源结果类型)|NOT_RESOURCE_RESULT_TYPE|ManagerCommonErrorCodeSummary|
+|linkis-manager-common |11003|Not supported resource type(不支持的资源类型):|NOT_RESOURCE_TYPE|ManagerCommonErrorCodeSummary|
+|linkis-manager-common |11003|Not supported resource serializable string(不支持资源可序列化字符串)|NOT_RESOURCE_STRING|ManagerCommonErrorCodeSummary|
+|linkis-manager-common |11006|Failed to request external resource(请求外部资源失败):|FAILED_REQUEST_RESOURCE|ManagerCommonErrorCodeSummary|
+|linkis-manager-common |11006|Get the Yarn queue information exception(获取Yarn队列信息异常)|YARN_QUEUE_EXCEPTION|ManagerCommonErrorCodeSummary|
+|linkis-manager-common |11006|Get the Yarn Application information exception.(获取Yarn Application信息异常)|YARN_APPLICATION_EXCEPTION|ManagerCommonErrorCodeSummary|
+|linkis-manager-common |11006|Queue  is not exists in YARN(YARN 中不存在队列)|YARN_NOT_EXISTS_QUEUE|ManagerCommonErrorCodeSummary|
+|linkis-manager-common |11006|Only support fairScheduler or capacityScheduler, schedulerType(仅支持 fairScheduler 或 capacityScheduler、schedulerType):|ONLY_SUPPORT_FAIRORCAPA|ManagerCommonErrorCodeSummary|
+|linkis-manager-common |11006|Get active Yarn resourcemanager from : ${haAddress} exception.(从 ${haAddress} 获取主Yarn resourcemanager异常)|GET_YARN_EXCEPTION|ManagerCommonErrorCodeSummary|
+|linkis-manager-common |11201| No resource available found for em(没有为 em 找到可用的资源)|NO_RESOURCE_AVAILABLE|ManagerCommonErrorCodeSummary|
+|linkis-manager-common |11201|The resource tag has no resource, please check the resource in the database. Label(资源标签没有资源,请检查数据库中的资源.标签):|NO_RESOURCE|ManagerCommonErrorCodeSummary|
+|linkis-manager-common |110012|No ExternalResourceRequester found for resource type(找不到资源类型的 ExternalResourceRequester):|NO_FOUND_RESOURCE_TYPE|ManagerCommonErrorCodeSummary|
+|linkis-manager-common |110013|No suitable ExternalResourceProvider found for cluster(没有为集群找到合适的 ExternalResourceProvider):|NO_SUITABLE_CLUSTER|ManagerCommonErrorCodeSummary|
+|linkis-manager-common |110022|Resource label:{0} has no usedResource, please check, refuse request usedResource(资源标签:{0}没有usedResource,请检查,拒绝请求usedResource)|REFUSE_REQUEST|ManagerCommonErrorCodeSummary|
+|linkis-manager-common |120010|Only admin can read all user's resource.(只有管理员可以读取所有用户的资源.)|ONLY_ADMIN_READ|ManagerCommonErrorCodeSummary|
+|linkis-manager-common |120011|Only admin can reset user's resource.(只有管理员可以重置用户的资源.)|ONLY_ADMIN_RESET|ManagerCommonErrorCodeSummary|
\ No newline at end of file
diff --git a/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/java/org/apache/linkis/manager/rm/external/service/impl/ExternalResourceServiceImpl.java b/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/java/org/apache/linkis/manager/rm/external/service/impl/ExternalResourceServiceImpl.java
index 2eb51e386..090e27923 100644
--- a/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/java/org/apache/linkis/manager/rm/external/service/impl/ExternalResourceServiceImpl.java
+++ b/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/java/org/apache/linkis/manager/rm/external/service/impl/ExternalResourceServiceImpl.java
@@ -56,6 +56,8 @@ import com.google.common.cache.LoadingCache;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.linkis.manager.common.errorcode.ManagerCommonErrorCodeSummary.*;
+
 @Component
 public class ExternalResourceServiceImpl implements ExternalResourceService, InitializingBean {
 
@@ -163,7 +165,7 @@ public class ExternalResourceServiceImpl implements ExternalResourceService, Ini
         times++;
       }
     }
-    throw new RMErrorException(11006, errorMsg);
+    throw new RMErrorException(FAILED_REQUEST_RESOURCE.getErrorCode(), errorMsg);
   }
 
   @Override
@@ -188,15 +190,13 @@ public class ExternalResourceServiceImpl implements ExternalResourceService, Ini
       }
     } catch (ExecutionException e) {
       throw new RMErrorException(
-          110013,
-          "No suitable ExternalResourceProvider found for cluster: "
-              + realClusterLabel.getClusterName(),
+          NO_SUITABLE_CLUSTER.getErrorCode(),
+          NO_SUITABLE_CLUSTER.getErrorDesc() + realClusterLabel.getClusterName(),
           e);
     }
     throw new RMErrorException(
-        110013,
-        "No suitable ExternalResourceProvider found for cluster: "
-            + realClusterLabel.getClusterName());
+        NO_SUITABLE_CLUSTER.getErrorCode(),
+        NO_SUITABLE_CLUSTER.getErrorDesc() + realClusterLabel.getClusterName());
   }
 
   private ExternalResourceRequester getRequester(ResourceType resourceType)
@@ -207,7 +207,8 @@ public class ExternalResourceServiceImpl implements ExternalResourceService, Ini
       }
     }
     throw new RMErrorException(
-        110012, "No ExternalResourceRequester found for resource type: " + resourceType);
+        NO_FOUND_RESOURCE_TYPE.getErrorCode(),
+        NO_FOUND_RESOURCE_TYPE.getErrorDesc() + resourceType);
   }
 
   private ExternalResourceIdentifierParser getIdentifierParser(ResourceType resourceType)
@@ -218,6 +219,7 @@ public class ExternalResourceServiceImpl implements ExternalResourceService, Ini
       }
     }
     throw new RMErrorException(
-        110012, "No ExternalResourceIdentifierParser found for resource type: " + resourceType);
+        NO_FOUND_RESOURCE_TYPE.getErrorCode(),
+        NO_FOUND_RESOURCE_TYPE.getErrorDesc() + resourceType);
   }
 }
diff --git a/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/rm/external/yarn/YarnResourceRequester.scala b/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/rm/external/yarn/YarnResourceRequester.scala
index 8f72b92e2..9284ddcd4 100644
--- a/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/rm/external/yarn/YarnResourceRequester.scala
+++ b/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/rm/external/yarn/YarnResourceRequester.scala
@@ -25,6 +25,7 @@ import org.apache.linkis.manager.common.entity.resource.{
   ResourceType,
   YarnResource
 }
+import org.apache.linkis.manager.common.errorcode.ManagerCommonErrorCodeSummary._
 import org.apache.linkis.manager.common.exception.{RMErrorException, RMWarnException}
 import org.apache.linkis.manager.rm.external.domain.{
   ExternalAppInfo,
@@ -187,7 +188,10 @@ class YarnResourceRequester extends ExternalResourceRequester with Logging {
         val queue = getQueueOfCapacity(childQueues)
         if (queue.isEmpty) {
           logger.debug(s"cannot find any information about queue $queueName, response: " + resp)
-          throw new RMWarnException(11006, s"queue $queueName is not exists in YARN.")
+          throw new RMWarnException(
+            YARN_NOT_EXISTS_QUEUE.getErrorCode,
+            YARN_NOT_EXISTS_QUEUE.getErrorDesc + s" $queueName"
+          )
         }
         (maxEffectiveHandle(queue).get, getYarnResource(queue.map(_ \ "resourcesUsed")).get)
       } else if ("fairScheduler".equals(schedulerType)) {
@@ -195,7 +199,10 @@ class YarnResourceRequester extends ExternalResourceRequester with Logging {
         val queue = getQueue(childQueues)
         if (queue.isEmpty) {
           logger.debug(s"cannot find any information about queue $queueName, response: " + resp)
-          throw new RMWarnException(11006, s"queue $queueName is not exists in YARN.")
+          throw new RMWarnException(
+            YARN_NOT_EXISTS_QUEUE.getErrorCode,
+            YARN_NOT_EXISTS_QUEUE.getErrorDesc + s" $queueName"
+          )
         }
         (
           getYarnResource(queue.map(_ \ "maxResources")).get,
@@ -206,8 +213,8 @@ class YarnResourceRequester extends ExternalResourceRequester with Logging {
           s"only support fairScheduler or capacityScheduler, schedulerType: $schedulerType , response: " + resp
         )
         throw new RMWarnException(
-          11006,
-          s"only support fairScheduler or capacityScheduler, schedulerType: $schedulerType"
+          ONLY_SUPPORT_FAIRORCAPA.getErrorCode,
+          ONLY_SUPPORT_FAIRORCAPA.getErrorDesc + s" $schedulerType"
         )
       }
     }
@@ -220,8 +227,8 @@ class YarnResourceRequester extends ExternalResourceRequester with Logging {
       nodeResource
     }(t => {
       throw new RMErrorException(
-        11006,
-        "Get the Yarn queue information exception" + ".(获取Yarn队列信息异常)",
+        YARN_QUEUE_EXCEPTION.getErrorCode,
+        YARN_QUEUE_EXCEPTION.getErrorDesc,
         t
       )
     })
@@ -274,8 +281,8 @@ class YarnResourceRequester extends ExternalResourceRequester with Logging {
 
     Utils.tryCatch(getAppInfos().toList.asJava)(t => {
       throw new RMErrorException(
-        11006,
-        "Get the Yarn Application information exception.(获取Yarn Application信息异常)",
+        YARN_APPLICATION_EXCEPTION.getErrorCode,
+        YARN_APPLICATION_EXCEPTION.getErrorDesc,
         t
       )
     })
@@ -354,7 +361,7 @@ class YarnResourceRequester extends ExternalResourceRequester with Logging {
           rmAddressMap.put(haAddress, activeAddress)
         } else {
           throw new RMErrorException(
-            11007,
+            GET_YARN_EXCEPTION.getErrorCode,
             s"Get active Yarn resourcemanager from : ${haAddress} exception.(从 ${haAddress} 获取主Yarn resourcemanager异常)"
           )
         }
diff --git a/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/rm/restful/RMMonitorRest.scala b/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/rm/restful/RMMonitorRest.scala
index 87061caa4..ef7bf3378 100644
--- a/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/rm/restful/RMMonitorRest.scala
+++ b/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/rm/restful/RMMonitorRest.scala
@@ -22,6 +22,7 @@ import org.apache.linkis.manager.common.conf.RMConfiguration
 import org.apache.linkis.manager.common.entity.enumeration.NodeStatus
 import org.apache.linkis.manager.common.entity.node.EngineNode
 import org.apache.linkis.manager.common.entity.resource._
+import org.apache.linkis.manager.common.errorcode.ManagerCommonErrorCodeSummary._
 import org.apache.linkis.manager.common.exception.RMErrorException
 import org.apache.linkis.manager.common.serializer.NodeResourceSerializer
 import org.apache.linkis.manager.common.utils.ResourceUtils
@@ -239,7 +240,7 @@ class RMMonitorRest extends Logging {
     val queryUser = SecurityFilter.getLoginUser(request)
     val admins = RMUtils.GOVERNANCE_STATION_ADMIN.getValue.split(",")
     if (!admins.contains(queryUser.get)) {
-      throw new RMErrorException(120011, "only admin can reset user's resource.")
+      throw new RMErrorException(ONLY_ADMIN_RESET.getErrorCode, ONLY_ADMIN_RESET.getErrorDesc)
     }
     if (resourceId == null || resourceId <= 0) {
       userResourceService.resetAllUserResource(COMBINED_USERCREATOR_ENGINETYPE)
@@ -270,7 +271,7 @@ class RMMonitorRest extends Logging {
     val queryUser = SecurityFilter.getLoginUser(request)
     val admins = RMUtils.GOVERNANCE_STATION_ADMIN.getValue.split(",")
     if (!admins.contains(queryUser.get)) {
-      throw new RMErrorException(120010, "only admin can read all user's resource.")
+      throw new RMErrorException(ONLY_ADMIN_READ.getErrorCode, ONLY_ADMIN_READ.getErrorDesc)
     }
     // 1. Construct a string for SQL LIKE query, query the label_value of the label table
     val searchUsername = if (StringUtils.isEmpty(username)) "" else username
diff --git a/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/rm/service/RequestResourceService.scala b/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/rm/service/RequestResourceService.scala
index 5d43cfc5f..2e2d0b5bc 100644
--- a/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/rm/service/RequestResourceService.scala
+++ b/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/rm/service/RequestResourceService.scala
@@ -20,6 +20,7 @@ package org.apache.linkis.manager.rm.service
 import org.apache.linkis.common.utils.Logging
 import org.apache.linkis.manager.common.constant.RMConstant
 import org.apache.linkis.manager.common.entity.resource._
+import org.apache.linkis.manager.common.errorcode.ManagerCommonErrorCodeSummary._
 import org.apache.linkis.manager.common.exception.RMWarnException
 import org.apache.linkis.manager.label.entity.em.EMInstanceLabel
 import org.apache.linkis.manager.rm.domain.RMLabelContainer
@@ -102,8 +103,8 @@ abstract class RequestResourceService(labelResourceService: LabelResourceService
     } else {
       logger.warn(s"No resource available found for label ${labelContainer.getCurrentLabel}")
       throw new RMWarnException(
-        11201,
-        s"Resource label ${labelContainer.getCurrentLabel} has no resource, please check resource in db."
+        NO_RESOURCE.getErrorCode,
+        NO_RESOURCE.getErrorDesc + s"${labelContainer.getCurrentLabel}"
       )
     }
   }
@@ -136,8 +137,8 @@ abstract class RequestResourceService(labelResourceService: LabelResourceService
     } else {
       logger.warn(s"No resource available found for em ${emInstanceLabel.getInstance()} ")
       throw new RMWarnException(
-        11201,
-        s"No resource available found for em ${emInstanceLabel.getInstance()} "
+        NO_RESOURCE_AVAILABLE.getErrorCode,
+        NO_RESOURCE_AVAILABLE.getErrorDesc + s" ${emInstanceLabel.getInstance()} "
       )
     }
   }
@@ -381,9 +382,15 @@ abstract class RequestResourceService(labelResourceService: LabelResourceService
           (detail._1, { detail._2 })
         }
       case s: SpecialResource =>
-        throw new RMWarnException(11003, " not supported resource type " + s.getClass)
+        throw new RMWarnException(
+          NOT_RESOURCE_TYPE.getErrorCode,
+          NOT_RESOURCE_TYPE.getErrorDesc + s.getClass
+        )
       case r: Resource =>
-        throw new RMWarnException(11003, "not supported resource type " + r.getClass)
+        throw new RMWarnException(
+          NOT_RESOURCE_TYPE.getErrorCode,
+          NOT_RESOURCE_TYPE.getErrorDesc + r.getClass
+        )
     }
   }
 
diff --git a/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/rm/service/impl/DefaultResourceManager.scala b/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/rm/service/impl/DefaultResourceManager.scala
index 419854cda..25cc082cd 100644
--- a/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/rm/service/impl/DefaultResourceManager.scala
+++ b/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/rm/service/impl/DefaultResourceManager.scala
@@ -26,6 +26,7 @@ import org.apache.linkis.manager.common.entity.enumeration.NodeStatus
 import org.apache.linkis.manager.common.entity.node.{AMEMNode, AMEngineNode, InfoRMNode}
 import org.apache.linkis.manager.common.entity.persistence.{PersistenceLabel, PersistenceResource}
 import org.apache.linkis.manager.common.entity.resource._
+import org.apache.linkis.manager.common.errorcode.ManagerCommonErrorCodeSummary._
 import org.apache.linkis.manager.common.exception.{RMErrorException, RMWarnException}
 import org.apache.linkis.manager.common.utils.{ManagerUtils, ResourceUtils}
 import org.apache.linkis.manager.label.builder.factory.LabelBuilderFactoryContext
@@ -62,6 +63,7 @@ import org.springframework.beans.factory.InitializingBean
 import org.springframework.beans.factory.annotation.Autowired
 import org.springframework.stereotype.Component
 
+import java.text.MessageFormat
 import java.util
 import java.util.UUID
 import java.util.concurrent.TimeUnit
@@ -297,7 +299,10 @@ class DefaultResourceManager extends ResourceManager with Logging with Initializ
           val msg =
             s"Resource label: ${label.getStringValue} has no usedResource, please check, refuse request usedResource"
           logger.info(msg)
-          throw new RMErrorException(110022, msg)
+          throw new RMErrorException(
+            REFUSE_REQUEST.getErrorCode,
+            MessageFormat.format(REFUSE_REQUEST.getErrorDesc, label.getStringValue)
+          )
         }
         labelResourceList.put(label.getStringValue, usedResource)
       })
@@ -590,7 +595,7 @@ class DefaultResourceManager extends ResourceManager with Logging with Initializ
     if (labelResource.getCreateTime != null && usedResource.getCreateTime != null) {
       if (labelResource.getCreateTime.getTime > usedResource.getCreateTime.getTime) {
         throw new RMErrorException(
-          10022,
+          RESOURCE_LATER_CREATED.getErrorCode,
           s"no need to clear this labelResource, labelResource:${labelResource} created time is after than usedResource:${usedResource}" +
             s"无需清理该标签的资源,该标签资源的创建时间晚于已用资源的创建时间"
         )
diff --git a/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/rm/utils/UserConfiguration.scala b/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/rm/utils/UserConfiguration.scala
index 71b4b3ca9..c141a809c 100644
--- a/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/rm/utils/UserConfiguration.scala
+++ b/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/rm/utils/UserConfiguration.scala
@@ -26,6 +26,7 @@ import org.apache.linkis.governance.common.protocol.conf.{
 }
 import org.apache.linkis.manager.common.conf.RMConfiguration._
 import org.apache.linkis.manager.common.entity.resource._
+import org.apache.linkis.manager.common.errorcode.ManagerCommonErrorCodeSummary._
 import org.apache.linkis.manager.common.exception.RMWarnException
 import org.apache.linkis.manager.label.builder.factory.LabelBuilderFactoryContext
 import org.apache.linkis.manager.label.entity.engine.{EngineTypeLabel, UserCreatorLabel}
@@ -163,7 +164,11 @@ object UserConfiguration extends Logging {
         )
       )
     case ResourceType.Special => new SpecialResource(new java.util.HashMap[String, AnyVal]())
-    case _ => throw new RMWarnException(11003, "not supported resource result type ")
+    case _ =>
+      throw new RMWarnException(
+        NOT_RESOURCE_RESULT_TYPE.getErrorCode,
+        NOT_RESOURCE_RESULT_TYPE.getErrorDesc
+      )
   }
 
 }
diff --git a/linkis-computation-governance/linkis-manager/linkis-manager-common/src/main/java/org/apache/linkis/manager/common/errorcode/ManagerCommonErrorCodeSummary.java b/linkis-computation-governance/linkis-manager/linkis-manager-common/src/main/java/org/apache/linkis/manager/common/errorcode/ManagerCommonErrorCodeSummary.java
new file mode 100644
index 000000000..4afbfe25b
--- /dev/null
+++ b/linkis-computation-governance/linkis-manager/linkis-manager-common/src/main/java/org/apache/linkis/manager/common/errorcode/ManagerCommonErrorCodeSummary.java
@@ -0,0 +1,137 @@
+/*
+ * 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.manager.common.errorcode;
+
+public enum ManagerCommonErrorCodeSummary {
+  RESOURCE_LATER_CREATED(
+      10022,
+      "The tag resource was created later than the used resource was created(无需清理该标签的资源,该标签资源的创建时间晚于已用资源的创建时间)",
+      "The tag resource was created later than the used resource was created(无需清理该标签的资源,该标签资源的创建时间晚于已用资源的创建时间)"),
+  OPERATION_MULTIPLIED(
+      11002,
+      "Unsupported operation: multiplied(不支持的操作:multiplied)",
+      "Unsupported operation: multiplied(不支持的操作:multiplied)"),
+  NOT_RESOURCE_POLICY(
+      11003,
+      "Not supported resource result policy (不支持的资源结果策略)",
+      "Not supported resource result policy (不支持的资源结果策略)"),
+  NOT_RESOURCE_RESULT_TYPE(
+      11003,
+      "Not supported resource result type(不支持的资源结果类型)",
+      "Not supported resource result type (不支持的资源结果类型)"),
+  NOT_RESOURCE_TYPE(
+      11003, "Not supported resource type(不支持的资源类型):", "Not supported resource type(不支持的资源类型):"),
+  NOT_RESOURCE_STRING(
+      11003,
+      "Not supported resource serializable string(不支持资源可序列化字符串) ",
+      "Not supported resource serializable string(不支持资源可序列化字符串) "),
+  FAILED_REQUEST_RESOURCE(
+      11006,
+      "Failed to request external resource(请求外部资源失败):",
+      "Failed to request external resource(请求外部资源失败):"),
+  YARN_QUEUE_EXCEPTION(
+      11006,
+      "Get the Yarn queue information exception(获取Yarn队列信息异常)",
+      "Get the Yarn queue information exception(获取Yarn队列信息异常)"),
+  YARN_APPLICATION_EXCEPTION(
+      11006,
+      "Get the Yarn Application information exception.(获取Yarn Application信息异常)",
+      "Get the Yarn Application information exception.(获取Yarn Application信息异常)"),
+  YARN_NOT_EXISTS_QUEUE(
+      11006,
+      "Queue  is not exists in YARN(YARN 中不存在队列)",
+      "Queue  is not exists in YARN(YARN 中不存在队列)"),
+  ONLY_SUPPORT_FAIRORCAPA(
+      11006,
+      "Only support fairScheduler or capacityScheduler, schedulerType(仅支持 fairScheduler 或 capacityScheduler、schedulerType):",
+      "Only support fairScheduler or capacityScheduler, schedulerType(仅支持 fairScheduler 或 capacityScheduler、schedulerType):"),
+  GET_YARN_EXCEPTION(
+      11007,
+      "Get active Yarn resourcemanager from : ${haAddress} exception.(从 ${haAddress} 获取主Yarn resourcemanager异常)",
+      "Get active Yarn resourcemanager from : ${haAddress} exception.(从 ${haAddress} 获取主Yarn resourcemanager异常)"),
+  NO_RESOURCE(
+      11201,
+      "The resource tag has no resource, please check the resource in the database. Label(资源标签没有资源,请检查数据库中的资源.标签):",
+      "The resource tag has no resource, please check the resource in the database. Label(资源标签没有资源,请检查数据库中的资源.标签):"),
+  NO_RESOURCE_AVAILABLE(
+      11201,
+      "No resource available found for em(没有为 em 找到可用的资源)",
+      "No resource available found for em(没有为 em 找到可用的资源)"),
+  NO_FOUND_RESOURCE_TYPE(
+      110012,
+      "No ExternalResourceRequester found for resource type(找不到资源类型的 ExternalResourceRequester):",
+      "No ExternalResourceRequester found for resource type(找不到资源类型的 ExternalResourceRequester):"),
+  NO_SUITABLE_CLUSTER(
+      110013,
+      "No suitable ExternalResourceProvider found for cluster(没有为集群找到合适的 ExternalResourceProvider):",
+      "No suitable ExternalResourceProvider found for cluster(没有为集群找到合适的 ExternalResourceProvider):"),
+  REFUSE_REQUEST(
+      110022,
+      "Resource label:{0} has no usedResource, please check, refuse request usedResource(资源标签:{0}没有usedResource,请检查,拒绝请求usedResource)",
+      "Resource label:{0} has no usedResource, please check, refuse request usedResource(资源标签:{0}没有usedResource,请检查,拒绝请求usedResource)"),
+
+  ONLY_ADMIN_READ(
+      120010,
+      "Only admin can read all user's resource.(只有管理员可以读取所有用户的资源.)",
+      "Only admin can read all user's resource.(只有管理员可以读取所有用户的资源.)"),
+  ONLY_ADMIN_RESET(
+      120011,
+      "Only admin can reset user's resource.(只有管理员可以重置用户的资源.)",
+      "Only admin can reset user's resource.(只有管理员可以重置用户的资源.)");
+  /** (errorCode)错误码 */
+  private int errorCode;
+  /** (errorDesc)错误描述 */
+  private String errorDesc;
+  /** Possible reasons for the error(错误可能出现的原因) */
+  private String comment;
+
+  ManagerCommonErrorCodeSummary(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-computation-governance/linkis-manager/linkis-manager-common/src/main/scala/org/apache/linkis/manager/common/entity/resource/Resource.scala b/linkis-computation-governance/linkis-manager/linkis-manager-common/src/main/scala/org/apache/linkis/manager/common/entity/resource/Resource.scala
index 5463b8d33..de5e97c54 100644
--- a/linkis-computation-governance/linkis-manager/linkis-manager-common/src/main/scala/org/apache/linkis/manager/common/entity/resource/Resource.scala
+++ b/linkis-computation-governance/linkis-manager/linkis-manager-common/src/main/scala/org/apache/linkis/manager/common/entity/resource/Resource.scala
@@ -19,6 +19,7 @@ package org.apache.linkis.manager.common.entity.resource
 
 import org.apache.linkis.common.utils.{ByteTimeUtils, Logging}
 import org.apache.linkis.manager.common.entity.resource.ResourceType._
+import org.apache.linkis.manager.common.errorcode.ManagerCommonErrorCodeSummary._
 import org.apache.linkis.manager.common.exception.ResourceWarnException
 
 import org.apache.commons.lang3.StringUtils
@@ -93,7 +94,11 @@ object Resource extends Logging {
       new DriverAndYarnResource(new LoadInstanceResource(0, 0, 0), new YarnResource(0, 0, 0))
     case Special => new SpecialResource(new java.util.HashMap[String, AnyVal]())
     case Default => new LoadResource(0, 0)
-    case _ => throw new ResourceWarnException(11003, "not supported resource result policy ")
+    case _ =>
+      throw new ResourceWarnException(
+        NOT_RESOURCE_POLICY.getErrorCode,
+        NOT_RESOURCE_POLICY.getErrorDesc
+      )
   }
 
   def getZeroResource(resource: Resource): Resource = resource match {
@@ -114,7 +119,10 @@ object Resource extends Logging {
       }
     case s: SpecialResource => new SpecialResource(new java.util.HashMap[String, AnyVal]())
     case r: Resource =>
-      throw new ResourceWarnException(11003, "not supported resource type " + r.getClass)
+      throw new ResourceWarnException(
+        NOT_RESOURCE_TYPE.getErrorCode,
+        NOT_RESOURCE_TYPE.getErrorDesc + r.getClass
+      )
   }
 
 }
@@ -475,7 +483,10 @@ class DriverAndYarnResource(
   }
 
   override def multiplied(r: Resource): Resource = {
-    throw new ResourceWarnException(11002, "Unsupported operation: multiplied")
+    throw new ResourceWarnException(
+      OPERATION_MULTIPLIED.getErrorCode,
+      OPERATION_MULTIPLIED.getErrorDesc
+    )
   }
 
   override def multiplied(rate: Float): Resource = {
@@ -490,7 +501,10 @@ class DriverAndYarnResource(
   }
 
   override def divide(r: Resource): Resource =
-    throw new ResourceWarnException(11002, "Unsupported operation: multiplied")
+    throw new ResourceWarnException(
+      OPERATION_MULTIPLIED.getErrorCode,
+      OPERATION_MULTIPLIED.getErrorDesc
+    )
 
   override def divide(rate: Int): Resource = if (isModuleOperate) {
     new DriverAndYarnResource(this.loadInstanceResource.divide(rate), this.yarnResource)
@@ -579,7 +593,10 @@ class SpecialResource(val resources: java.util.Map[String, AnyVal]) extends Reso
         case f: Float => f + v2.asInstanceOf[Float]
         case s: Short => s + v2.asInstanceOf[Short]
         case _ =>
-          throw new ResourceWarnException(11003, "not supported resource type: " + v1.getClass)
+          throw new ResourceWarnException(
+            NOT_RESOURCE_TYPE.getErrorCode,
+            NOT_RESOURCE_TYPE.getErrorDesc + v1.getClass
+          )
       }
   )
 
@@ -593,7 +610,10 @@ class SpecialResource(val resources: java.util.Map[String, AnyVal]) extends Reso
         case f: Float => f - v2.asInstanceOf[Float]
         case s: Short => s - v2.asInstanceOf[Short]
         case _ =>
-          throw new ResourceWarnException(11003, "not supported resource type: " + v1.getClass)
+          throw new ResourceWarnException(
+            NOT_RESOURCE_TYPE.getErrorCode,
+            NOT_RESOURCE_TYPE.getErrorDesc + v1.getClass
+          )
       }
   )
 
@@ -607,7 +627,10 @@ class SpecialResource(val resources: java.util.Map[String, AnyVal]) extends Reso
         case f: Float => f * v2.asInstanceOf[Float]
         case s: Short => s * v2.asInstanceOf[Short]
         case _ =>
-          throw new ResourceWarnException(11003, "not supported resource type: " + v1.getClass)
+          throw new ResourceWarnException(
+            NOT_RESOURCE_TYPE.getErrorCode,
+            NOT_RESOURCE_TYPE.getErrorDesc + v1.getClass
+          )
       }
   )
 
@@ -621,7 +644,10 @@ class SpecialResource(val resources: java.util.Map[String, AnyVal]) extends Reso
         case (k, f: Float) => k -> f * rate
         case (k, s: Short) => k -> (s * rate).toShort
         case (k, v) =>
-          throw new ResourceWarnException(11003, "not supported resource type: " + v.getClass)
+          throw new ResourceWarnException(
+            NOT_RESOURCE_TYPE.getErrorCode,
+            NOT_RESOURCE_TYPE.getErrorDesc + v.getClass
+          )
       }
       .toMap[String, AnyVal]
   )
@@ -636,7 +662,10 @@ class SpecialResource(val resources: java.util.Map[String, AnyVal]) extends Reso
         case f: Float => f / v2.asInstanceOf[Float]
         case s: Short => s / v2.asInstanceOf[Short]
         case _ =>
-          throw new ResourceWarnException(11003, "not supported resource type: " + v1.getClass)
+          throw new ResourceWarnException(
+            NOT_RESOURCE_TYPE.getErrorCode,
+            NOT_RESOURCE_TYPE.getErrorDesc + v1.getClass
+          )
       }
   )
 
@@ -650,7 +679,10 @@ class SpecialResource(val resources: java.util.Map[String, AnyVal]) extends Reso
         case (k, f: Float) => k -> f / rate
         case (k, s: Short) => k -> (s / rate).toShort
         case (k, v) =>
-          throw new ResourceWarnException(11003, "not supported resource type: " + v.getClass)
+          throw new ResourceWarnException(
+            NOT_RESOURCE_TYPE.getErrorCode,
+            NOT_RESOURCE_TYPE.getErrorDesc + v.getClass
+          )
       }
       .toMap[String, AnyVal]
   )
@@ -665,7 +697,10 @@ class SpecialResource(val resources: java.util.Map[String, AnyVal]) extends Reso
         case (k, f: Float) => f <= rs.get(k).asInstanceOf[Float]
         case (k, s: Short) => s <= rs.get(k).asInstanceOf[Short]
         case (k, v) =>
-          throw new ResourceWarnException(11003, "not supported resource type: " + v.getClass)
+          throw new ResourceWarnException(
+            NOT_RESOURCE_TYPE.getErrorCode,
+            NOT_RESOURCE_TYPE.getErrorDesc + v.getClass
+          )
       }
     case _ => true
   }
@@ -686,7 +721,10 @@ class SpecialResource(val resources: java.util.Map[String, AnyVal]) extends Reso
         case (k, f: Float) => f > rs.get(k).asInstanceOf[Float]
         case (k, s: Short) => s > rs.get(k).asInstanceOf[Short]
         case (k, v) =>
-          throw new ResourceWarnException(11003, "not supported resource type: " + v.getClass)
+          throw new ResourceWarnException(
+            NOT_RESOURCE_TYPE.getErrorCode,
+            NOT_RESOURCE_TYPE.getErrorDesc + v.getClass
+          )
       }
     case _ => true
   }
@@ -701,7 +739,10 @@ class SpecialResource(val resources: java.util.Map[String, AnyVal]) extends Reso
         case (k, f: Float) => f != rs.get(k).asInstanceOf[Float]
         case (k, s: Short) => s != rs.get(k).asInstanceOf[Short]
         case (k, v) =>
-          throw new ResourceWarnException(11003, "not supported resource type: " + v.getClass)
+          throw new ResourceWarnException(
+            NOT_RESOURCE_TYPE.getErrorCode,
+            NOT_RESOURCE_TYPE.getErrorDesc + v.getClass
+          )
       }
     case _ => true
   }
@@ -716,7 +757,10 @@ class SpecialResource(val resources: java.util.Map[String, AnyVal]) extends Reso
         case (k, f: Float) => f < rs.get(k).asInstanceOf[Float]
         case (k, s: Short) => s < rs.get(k).asInstanceOf[Short]
         case (k, v) =>
-          throw new ResourceWarnException(11003, "not supported resource type: " + v.getClass)
+          throw new ResourceWarnException(
+            NOT_RESOURCE_TYPE.getErrorCode,
+            NOT_RESOURCE_TYPE.getErrorDesc + v.getClass
+          )
       }
     case _ => true
   }
@@ -778,8 +822,8 @@ object ResourceSerializer
             new SpecialResource(resources.extract[Map[String, AnyVal]])
           case JObject(list) =>
             throw new ResourceWarnException(
-              11003,
-              "not supported resource serializable string " + list
+              NOT_RESOURCE_STRING.getErrorCode,
+              NOT_RESOURCE_STRING.getErrorDesc + list
             )
         },
         {
@@ -808,7 +852,10 @@ object ResourceSerializer
           case s: SpecialResource =>
             ("resources", Serialization.write(JavaConversions.mapAsScalaMap(s.resources).toMap))
           case r: Resource =>
-            throw new ResourceWarnException(11003, "not supported resource type " + r.getClass)
+            throw new ResourceWarnException(
+              NOT_RESOURCE_TYPE.getErrorCode,
+              NOT_RESOURCE_TYPE.getErrorDesc + r.getClass
+            )
         }
       )
     )


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