You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@linkis.apache.org by le...@apache.org on 2022/06/15 13:54:45 UTC

[incubator-linkis] branch dev-1.2.0 updated: Entrance support isolation by routing labels (#2291)

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

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


The following commit(s) were added to refs/heads/dev-1.2.0 by this push:
     new df2e38fad Entrance support isolation by routing labels  (#2291)
df2e38fad is described below

commit df2e38fad5bc6877453977c13d3319c089c5e642
Author: peacewong <wp...@gmail.com>
AuthorDate: Wed Jun 15 21:54:40 2022 +0800

    Entrance support isolation by routing labels  (#2291)
    
    * Add the null value judgment of route label
    
    * Support service isolation through route label
    
    * Add a method to get instances by app Name
    
    * optimize code
    
    * add InstanceErrorException
    
    * add entrance route label feature
---
 .../conf/linkis-cg-entrance.properties             |  3 ++
 .../linkis-entrance/pom.xml                        |  4 +-
 .../label/client/EurekaInstanceLabelClient.scala   |  9 +++-
 .../instance/label/InsLabelAutoConfiguration.java  |  5 +--
 .../linkis/instance/label/conf/InsLabelConf.java   |  2 +-
 .../instance/label/dao/InsLabelRelationDao.java    |  3 ++
 .../label/dao/impl/InsLabelRelationMapper.xml      |  7 +++-
 .../instance/label/dao/impl/InstanceInfoMapper.xml |  2 +-
 .../label/dao/impl/InstanceLabelMapper.xml         |  2 +-
 .../label/exception/InstanceErrorException.java    | 36 ++++++++++++++++
 .../instance/label/restful/InstanceRestful.java    |  7 ++--
 .../label/service/InsLabelAccessService.java       | 12 ++++--
 .../instance/label/service/InsLabelService.java    | 13 +++---
 .../label/service/impl/DefaultInsLabelService.java | 49 +++++++++-------------
 .../impl/DefaultInsLabelServiceAdapter.java        | 28 +++++++++++--
 .../label/service/impl/EurekaInsLabelService.java  |  5 +++
 .../gateway/config/GatewayConfiguration.scala      |  3 ++
 .../ujes/route/AbstractLabelGatewayRouter.scala    | 46 +++++++++++++++-----
 .../ujes/route/DefaultLabelGatewayRouter.scala     | 32 +++++++-------
 19 files changed, 189 insertions(+), 79 deletions(-)

diff --git a/assembly-combined-package/assembly-combined/conf/linkis-cg-entrance.properties b/assembly-combined-package/assembly-combined/conf/linkis-cg-entrance.properties
index 212246424..cfba4b7b7 100644
--- a/assembly-combined-package/assembly-combined/conf/linkis-cg-entrance.properties
+++ b/assembly-combined-package/assembly-combined/conf/linkis-cg-entrance.properties
@@ -19,5 +19,8 @@ wds.linkis.server.restful.scan.packages=org.apache.linkis.entrance.restful
 wds.linkis.server.socket.mode=false
 #wds.linkis.entrance.config.log.path=hdfs:///tmp/linkis/
 wds.linkis.resultSet.store.path=hdfs:///tmp/linkis
+
+## enable entrance label registration
+#spring.eureka.instance.metadata-map.route=et1
 ##Spring
 spring.server.port=9104
\ No newline at end of file
diff --git a/linkis-computation-governance/linkis-entrance/pom.xml b/linkis-computation-governance/linkis-entrance/pom.xml
index fa5b474a2..d3d14c900 100644
--- a/linkis-computation-governance/linkis-entrance/pom.xml
+++ b/linkis-computation-governance/linkis-entrance/pom.xml
@@ -107,11 +107,11 @@
             <version>${linkis.version}</version>
         </dependency>
 
-        <!--<dependency>
+        <dependency>
             <groupId>org.apache.linkis</groupId>
             <artifactId>linkis-instance-label-client</artifactId>
             <version>${linkis.version}</version>
-        </dependency>-->
+        </dependency>
 
     </dependencies>
 
diff --git a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-client/src/main/scala/org/apache/linkis/instance/label/client/EurekaInstanceLabelClient.scala b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-client/src/main/scala/org/apache/linkis/instance/label/client/EurekaInstanceLabelClient.scala
index 863507588..dc9f71648 100644
--- a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-client/src/main/scala/org/apache/linkis/instance/label/client/EurekaInstanceLabelClient.scala
+++ b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-client/src/main/scala/org/apache/linkis/instance/label/client/EurekaInstanceLabelClient.scala
@@ -17,10 +17,14 @@
  
 package org.apache.linkis.instance.label.client
 
+import java.util
+
 import org.apache.linkis.common.utils.Logging
 import org.apache.linkis.protocol.label.{InsLabelAttachRequest, InsLabelRemoveRequest}
 import org.apache.linkis.rpc.Sender
 import javax.annotation.PostConstruct
+import org.apache.commons.lang3.StringUtils
+import org.apache.linkis.manager.label.constant.LabelKeyConstant
 import org.springframework.beans.factory.annotation.Autowired
 import org.springframework.cloud.client.serviceregistry.Registration
 import org.springframework.context.event.{ContextClosedEvent, EventListener}
@@ -35,12 +39,15 @@ class EurekaInstanceLabelClient extends Logging {
   private var registration: Registration = _
 
 
+
   @PostConstruct
   def init(): Unit = {
     info("EurekaInstanceLabelClient init")
     val metadata = registration.getMetadata
-    if (null != metadata) {
+    if (null != metadata && metadata.containsKey(LabelKeyConstant.ROUTE_KEY) && StringUtils.isNoneBlank(metadata.get(LabelKeyConstant.ROUTE_KEY))) {
       info(s"Start to register label for instance $metadata")
+      val labels = new util.HashMap[String, Object]()
+      labels.put(LabelKeyConstant.ROUTE_KEY, metadata.get(LabelKeyConstant.ROUTE_KEY))
       val insLabelAttachRequest = new InsLabelAttachRequest
       insLabelAttachRequest.setLabels(metadata.asInstanceOf[java.util.Map[String, Object]])
       insLabelAttachRequest.setServiceInstance(Sender.getThisServiceInstance)
diff --git a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/InsLabelAutoConfiguration.java b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/InsLabelAutoConfiguration.java
index 75fa12125..a0053e72d 100644
--- a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/InsLabelAutoConfiguration.java
+++ b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/InsLabelAutoConfiguration.java
@@ -33,7 +33,6 @@ import org.springframework.context.annotation.Configuration;
 import org.springframework.context.annotation.Scope;
 import org.springframework.core.annotation.AnnotationUtils;
 
-import com.netflix.discovery.EurekaClient;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -74,8 +73,8 @@ public class InsLabelAutoConfiguration {
     }
 
     /** Configuration in eureka environment */
-    @Configuration
-    @ConditionalOnClass({EurekaClient.class})
+    /* @Configuration
+    @ConditionalOnClass({EurekaClient.class})*/
     public static class EurekaClientConfiguration {
         @ConditionalOnMissingBean({EurekaInsLabelService.class})
         @Bean
diff --git a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/conf/InsLabelConf.java b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/conf/InsLabelConf.java
index a7e3a9357..8517fd56f 100644
--- a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/conf/InsLabelConf.java
+++ b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/conf/InsLabelConf.java
@@ -44,7 +44,7 @@ public class InsLabelConf {
             CommonVars.apply("wds.linkis.instance.label.cache.maximum.size", 1000);
 
     public static final CommonVars<String> CACHE_NAMES =
-            CommonVars.apply("wds.linkis.instance.label.cache.names", "instance,label");
+            CommonVars.apply("wds.linkis.instance.label.cache.names", "instance,label,appInstance");
 
     public static final CommonVars<String> EUREKA_URL =
             CommonVars.apply("wds.linkis.eureka.defaultZone", "http://localhost:20303");
diff --git a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/dao/InsLabelRelationDao.java b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/dao/InsLabelRelationDao.java
index 6a2a443a1..2f8627c61 100644
--- a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/dao/InsLabelRelationDao.java
+++ b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/dao/InsLabelRelationDao.java
@@ -17,6 +17,7 @@
 
 package org.apache.linkis.instance.label.dao;
 
+import org.apache.linkis.common.ServiceInstance;
 import org.apache.linkis.instance.label.entity.InsPersistenceLabel;
 import org.apache.linkis.instance.label.entity.InstanceInfo;
 
@@ -75,6 +76,8 @@ public interface InsLabelRelationDao {
      */
     List<InstanceInfo> listAllInstanceWithLabel();
 
+    List<ServiceInstance> getInstancesByNames(String appName);
+
     /**
      * Drop relationships by instance and label ids
      *
diff --git a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/dao/impl/InsLabelRelationMapper.xml b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/dao/impl/InsLabelRelationMapper.xml
index 87c8f0656..e18684023 100644
--- a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/dao/impl/InsLabelRelationMapper.xml
+++ b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/dao/impl/InsLabelRelationMapper.xml
@@ -145,7 +145,7 @@
     </delete>
 
     <insert id="insertRelations">
-        INSERT INTO `linkis_ps_instance_label_relation`
+        REPLACE INTO `linkis_ps_instance_label_relation`
         (`service_instance`, `label_id`)
         VALUES
         <foreach collection="labelIds" item="item" open="" close="" separator=",">
@@ -195,4 +195,9 @@
     <select id="listAllInstanceWithLabel" resultMap="instanceInfoCascadeMap">
        SELECT * FROM `linkis_ps_instance_info`
     </select>
+
+    <select id="getInstancesByNames" resultMap="instanceInfoCascadeMap">
+        SELECT * FROM `linkis_ps_instance_info` WHERE `name`= #{appName}
+    </select>
+
 </mapper>
\ No newline at end of file
diff --git a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/dao/impl/InstanceInfoMapper.xml b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/dao/impl/InstanceInfoMapper.xml
index ef4add08b..d09d9dc03 100644
--- a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/dao/impl/InstanceInfoMapper.xml
+++ b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/dao/impl/InstanceInfoMapper.xml
@@ -29,7 +29,7 @@
     </resultMap>
 
     <insert id="insertOne" parameterType="InstanceInfo">
-        <![CDATA[INSERT INTO `linkis_ps_instance_info`(`instance`, `name`)
+        <![CDATA[REPLACE INTO `linkis_ps_instance_info`(`instance`, `name`)
         VALUES(#{instance}, #{applicationName})]]>
     </insert>
 
diff --git a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/dao/impl/InstanceLabelMapper.xml b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/dao/impl/InstanceLabelMapper.xml
index a8ed0a1ae..f642e8dc8 100644
--- a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/dao/impl/InstanceLabelMapper.xml
+++ b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/dao/impl/InstanceLabelMapper.xml
@@ -54,7 +54,7 @@
 
     <insert id="insertBatch" keyProperty="id" useGeneratedKeys="true">
         <foreach collection="list" item="item" separator="" open="" close="">
-            <![CDATA[INSERT INTO `linkis_ps_instance_label`(]]>
+            <![CDATA[REPLACE INTO `linkis_ps_instance_label`(]]>
             <include refid="label_insert_columns"/>
             <![CDATA[) VALUES(#{item.labelKey}, #{item.stringValue},
              #{item.feature}, #{item.labelValueSize});]]>
diff --git a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/exception/InstanceErrorException.java b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/exception/InstanceErrorException.java
new file mode 100644
index 000000000..2eecad0d2
--- /dev/null
+++ b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/exception/InstanceErrorException.java
@@ -0,0 +1,36 @@
+/*
+ * 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.instance.label.exception;
+
+import org.apache.linkis.common.exception.ErrorException;
+
+public class InstanceErrorException extends ErrorException {
+
+    public InstanceErrorException(int errCode, String desc) {
+        super(errCode, desc);
+    }
+
+    public InstanceErrorException(String desc, Throwable t) {
+        this(desc);
+        this.initCause(t);
+    }
+
+    public InstanceErrorException(String desc) {
+        super(14100, desc);
+    }
+}
diff --git a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/restful/InstanceRestful.java b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/restful/InstanceRestful.java
index 901bbd887..f67404bc3 100644
--- a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/restful/InstanceRestful.java
+++ b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/restful/InstanceRestful.java
@@ -20,6 +20,7 @@ package org.apache.linkis.instance.label.restful;
 import org.apache.linkis.common.ServiceInstance;
 import org.apache.linkis.common.conf.Configuration;
 import org.apache.linkis.instance.label.entity.InstanceInfo;
+import org.apache.linkis.instance.label.exception.InstanceErrorException;
 import org.apache.linkis.instance.label.service.impl.DefaultInsLabelService;
 import org.apache.linkis.instance.label.utils.EntityParser;
 import org.apache.linkis.instance.label.vo.InstanceInfoVo;
@@ -62,7 +63,7 @@ public class InstanceRestful {
     public Message listAllInstanceWithLabel(HttpServletRequest req) throws Exception {
         String userName = ModuleUserUtils.getOperationUser(req);
         if (!Configuration.isAdmin(userName)) {
-            throw new Exception(
+            throw new InstanceErrorException(
                     String.format(
                             "Only admin can view all instances(只有管理员才能查看所有实例). The user [%s] is not admin.",
                             userName));
@@ -81,7 +82,7 @@ public class InstanceRestful {
             throws Exception {
         String userName = ModuleUserUtils.getOperationUser(req);
         if (!Configuration.isAdmin(userName)) {
-            throw new Exception(
+            throw new InstanceErrorException(
                     String.format(
                             "Only admin can modify instance label(只有管理员才能修改标签). The user [%s] is not admin",
                             userName));
@@ -115,7 +116,7 @@ public class InstanceRestful {
             }
         }
         if (labelKeySet.size() != labels.size()) {
-            throw new Exception(
+            throw new InstanceErrorException(
                     "Failed to update label, include repeat label(更新label失败,包含重复label)");
         }
         insLabelService.refreshLabelsToInstance(labels, instance);
diff --git a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/service/InsLabelAccessService.java b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/service/InsLabelAccessService.java
index 366e51510..f4fc2a806 100644
--- a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/service/InsLabelAccessService.java
+++ b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/service/InsLabelAccessService.java
@@ -19,6 +19,7 @@ package org.apache.linkis.instance.label.service;
 
 import org.apache.linkis.common.ServiceInstance;
 import org.apache.linkis.instance.label.entity.InstanceInfo;
+import org.apache.linkis.instance.label.exception.InstanceErrorException;
 import org.apache.linkis.manager.label.entity.Label;
 
 import java.util.List;
@@ -31,9 +32,11 @@ public interface InsLabelAccessService {
      * @param label label entity
      * @param serviceInstance service instance
      */
-    void attachLabelToInstance(Label<?> label, ServiceInstance serviceInstance);
+    void attachLabelToInstance(Label<?> label, ServiceInstance serviceInstance)
+            throws InstanceErrorException;
 
-    void attachLabelsToInstance(List<? extends Label<?>> labels, ServiceInstance serviceInstance);
+    void attachLabelsToInstance(List<? extends Label<?>> labels, ServiceInstance serviceInstance)
+            throws InstanceErrorException;
 
     /**
      * Refresh all the labels of instance (to init the relationship of instance and labels)
@@ -41,7 +44,8 @@ public interface InsLabelAccessService {
      * @param labels
      * @param serviceInstance
      */
-    void refreshLabelsToInstance(List<? extends Label<?>> labels, ServiceInstance serviceInstance);
+    void refreshLabelsToInstance(List<? extends Label<?>> labels, ServiceInstance serviceInstance)
+            throws InstanceErrorException;
     /**
      * Remove all relationship between label and instance
      *
@@ -94,6 +98,8 @@ public interface InsLabelAccessService {
      */
     List<InstanceInfo> listAllInstanceWithLabel();
 
+    List<ServiceInstance> getInstancesByNames(String appName);
+
     void removeInstance(ServiceInstance serviceInstance);
 
     void updateInstance(InstanceInfo instanceInfo);
diff --git a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/service/InsLabelService.java b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/service/InsLabelService.java
index 42376ab83..aac6a78da 100644
--- a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/service/InsLabelService.java
+++ b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/service/InsLabelService.java
@@ -35,12 +35,12 @@ public interface InsLabelService {
      * @param serviceInstance service instance
      */
     @CacheEvict(
-            cacheNames = {"label"},
+            cacheNames = {"instance", "appInstance", "label"},
             allEntries = true)
     void attachLabelToInstance(Label<?> label, ServiceInstance serviceInstance);
 
     @CacheEvict(
-            cacheNames = {"label"},
+            cacheNames = {"instance", "appInstance", "label"},
             allEntries = true)
     void attachLabelsToInstance(List<? extends Label<?>> labels, ServiceInstance serviceInstance);
 
@@ -51,7 +51,7 @@ public interface InsLabelService {
      * @param serviceInstance
      */
     @CacheEvict(
-            cacheNames = {"instance"},
+            cacheNames = {"instance", "appInstance", "label"},
             allEntries = true)
     void refreshLabelsToInstance(List<? extends Label<?>> labels, ServiceInstance serviceInstance);
 
@@ -61,7 +61,7 @@ public interface InsLabelService {
      * @param serviceInstance service instance
      */
     @CacheEvict(
-            cacheNames = {"instance"},
+            cacheNames = {"instance", "appInstance", "label"},
             allEntries = true)
     void removeLabelsFromInstance(ServiceInstance serviceInstance);
 
@@ -93,8 +93,11 @@ public interface InsLabelService {
 
     void removeInstance(ServiceInstance serviceInstance);
 
+    @Cacheable({"appInstance"})
+    List<ServiceInstance> getInstancesByNames(String appName);
+
     @CacheEvict(
-            cacheNames = {"instance", "label"},
+            cacheNames = {"instance", "label", "appInstance"},
             allEntries = true)
     void evictCache();
 }
diff --git a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/service/impl/DefaultInsLabelService.java b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/service/impl/DefaultInsLabelService.java
index ce3f39302..835ceaec8 100644
--- a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/service/impl/DefaultInsLabelService.java
+++ b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/service/impl/DefaultInsLabelService.java
@@ -26,8 +26,8 @@ import org.apache.linkis.instance.label.dao.InsLabelRelationDao;
 import org.apache.linkis.instance.label.dao.InstanceInfoDao;
 import org.apache.linkis.instance.label.dao.InstanceLabelDao;
 import org.apache.linkis.instance.label.entity.InsPersistenceLabel;
-import org.apache.linkis.instance.label.entity.InsPersistenceLabelValue;
 import org.apache.linkis.instance.label.entity.InstanceInfo;
+import org.apache.linkis.instance.label.exception.InstanceErrorException;
 import org.apache.linkis.instance.label.service.InsLabelAccessService;
 import org.apache.linkis.instance.label.service.annotation.AdapterMode;
 import org.apache.linkis.instance.label.vo.InsPersistenceLabelSearchVo;
@@ -97,14 +97,16 @@ public class DefaultInsLabelService implements InsLabelAccessService {
 
     @Override
     @Transactional(rollbackFor = Exception.class)
-    public void attachLabelToInstance(Label<?> label, ServiceInstance serviceInstance) {
+    public void attachLabelToInstance(Label<?> label, ServiceInstance serviceInstance)
+            throws InstanceErrorException {
         attachLabelsToInstance(Collections.singletonList(label), serviceInstance);
     }
 
     @Override
     @Transactional(rollbackFor = Exception.class)
     public void attachLabelsToInstance(
-            List<? extends Label<?>> labels, ServiceInstance serviceInstance) {
+            List<? extends Label<?>> labels, ServiceInstance serviceInstance)
+            throws InstanceErrorException {
         List<InsPersistenceLabel> insLabels = toInsPersistenceLabels(labels);
         List<InsPersistenceLabel> labelsNeedInsert = filterLabelNeededInsert(insLabels, true);
         if (!labelsNeedInsert.isEmpty()) {
@@ -132,7 +134,8 @@ public class DefaultInsLabelService implements InsLabelAccessService {
     @Override
     @Transactional(rollbackFor = Exception.class)
     public void refreshLabelsToInstance(
-            List<? extends Label<?>> labels, ServiceInstance serviceInstance) {
+            List<? extends Label<?>> labels, ServiceInstance serviceInstance)
+            throws InstanceErrorException {
         List<InsPersistenceLabel> insLabels = toInsPersistenceLabels(labels);
         // Label candidate to be removed
         List<InsPersistenceLabel> labelsCandidateRemoved =
@@ -189,25 +192,8 @@ public class DefaultInsLabelService implements InsLabelAccessService {
             List<? extends Label<?>> labels, Label.ValueRelation relation) {
         List<InsPersistenceLabel> insLabels = toInsPersistenceLabels(labels);
         if (!insLabels.isEmpty()) {
-            List<Map<String, String>> valueContent = new ArrayList<>();
-            AtomicBoolean searchByValues = new AtomicBoolean(false);
-            insLabels.forEach(
-                    insLabel -> {
-                        // It means that the labels provided is not regular,
-                        // so we should search instances by key-value map of labels
-                        if (StringUtils.isBlank(insLabel.getStringValue())) {
-                            searchByValues.set(true);
-                        }
-                        valueContent.add(insLabel.getValue());
-                    });
-            List<InstanceInfo> instanceInfoList = new ArrayList<>();
-            if ((relation != Label.ValueRelation.ALL || searchByValues.get())
-                    && valueContent.size() > 0) {
-                instanceInfoList =
-                        insLabelRelationDao.searchInsDirectByValues(valueContent, relation.name());
-            } else if (relation == Label.ValueRelation.ALL && !searchByValues.get()) {
-                instanceInfoList = insLabelRelationDao.searchInsDirectByLabels(insLabels);
-            }
+            List<InstanceInfo> instanceInfoList =
+                    insLabelRelationDao.searchInsDirectByLabels(insLabels);
             return instanceInfoList.stream()
                     .map(instanceInfo -> (ServiceInstance) instanceInfo)
                     .collect(Collectors.toList());
@@ -259,7 +245,7 @@ public class DefaultInsLabelService implements InsLabelAccessService {
                                             + insLabel.toString()
                                             + "]");
                             instanceLabelDao.remove(insLabel);
-                            instanceLabelDao.doRemoveKeyValues(insLabel.getId());
+                            // instanceLabelDao.doRemoveKeyValues(insLabel.getId());
                         }
                     }
                 });
@@ -271,6 +257,11 @@ public class DefaultInsLabelService implements InsLabelAccessService {
         return instances;
     }
 
+    @Override
+    public List<ServiceInstance> getInstancesByNames(String appName) {
+        return insLabelRelationDao.getInstancesByNames(appName);
+    }
+
     @Override
     public void removeInstance(ServiceInstance serviceInstance) {
         instanceInfoDao.removeInstance(serviceInstance);
@@ -368,7 +359,7 @@ public class DefaultInsLabelService implements InsLabelAccessService {
                                 Integer labelId = storedLabel.getId();
                                 labelNeedInsert.setId(labelId);
                                 if (needLock) {
-                                    // Update to lock the record
+                                    // Update time
                                     return instanceLabelDao.updateForLock(labelId) >= 0;
                                 }
                                 return true;
@@ -392,7 +383,7 @@ public class DefaultInsLabelService implements InsLabelAccessService {
                 insLabels,
                 subInsLabels -> instanceLabelDao.insertBatch(subInsLabels),
                 InsLabelConf.DB_PERSIST_BATCH_SIZE.getValue());
-        List<InsPersistenceLabelValue> labelValues =
+        /*List<InsPersistenceLabelValue> labelValues =
                 insLabels.stream()
                         .flatMap(
                                 insLabel -> {
@@ -412,7 +403,7 @@ public class DefaultInsLabelService implements InsLabelAccessService {
         batchOperation(
                 labelValues,
                 subLabelValues -> instanceLabelDao.doInsertKeyValues(subLabelValues),
-                InsLabelConf.DB_PERSIST_BATCH_SIZE.getValue());
+                InsLabelConf.DB_PERSIST_BATCH_SIZE.getValue());*/
     }
 
     /**
@@ -420,12 +411,12 @@ public class DefaultInsLabelService implements InsLabelAccessService {
      *
      * @param serviceInstance service instance
      */
-    private void doInsertInstance(ServiceInstance serviceInstance) {
+    private void doInsertInstance(ServiceInstance serviceInstance) throws InstanceErrorException {
         // ON DUPLICATE KEY
         try {
             instanceDao.insertOne(new InstanceInfo(serviceInstance));
         } catch (Exception e) {
-
+            throw new InstanceErrorException("Failed to insert service instance", e);
         }
     }
 
diff --git a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/service/impl/DefaultInsLabelServiceAdapter.java b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/service/impl/DefaultInsLabelServiceAdapter.java
index 61492e3b4..33f033faa 100644
--- a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/service/impl/DefaultInsLabelServiceAdapter.java
+++ b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/service/impl/DefaultInsLabelServiceAdapter.java
@@ -18,6 +18,7 @@
 package org.apache.linkis.instance.label.service.impl;
 
 import org.apache.linkis.common.ServiceInstance;
+import org.apache.linkis.instance.label.exception.InstanceErrorException;
 import org.apache.linkis.instance.label.service.InsLabelAccessService;
 import org.apache.linkis.instance.label.service.InsLabelServiceAdapter;
 import org.apache.linkis.manager.label.entity.Label;
@@ -67,7 +68,11 @@ public class DefaultInsLabelServiceAdapter implements InsLabelServiceAdapter {
         execOnServiceChain(
                 "attachLabelToInstance",
                 insLabelAccessService -> {
-                    insLabelAccessService.attachLabelToInstance(label, serviceInstance);
+                    try {
+                        insLabelAccessService.attachLabelToInstance(label, serviceInstance);
+                    } catch (InstanceErrorException e) {
+                        LOG.error("Failed to attachLabelToInstance", e);
+                    }
                     return true;
                 },
                 true,
@@ -80,7 +85,11 @@ public class DefaultInsLabelServiceAdapter implements InsLabelServiceAdapter {
         execOnServiceChain(
                 "attachLabelsToInstance",
                 insLabelAccessService -> {
-                    insLabelAccessService.attachLabelsToInstance(labels, serviceInstance);
+                    try {
+                        insLabelAccessService.attachLabelsToInstance(labels, serviceInstance);
+                    } catch (InstanceErrorException e) {
+                        LOG.error("Failed to attachLabelToInstance", e);
+                    }
                     return true;
                 },
                 true,
@@ -93,7 +102,11 @@ public class DefaultInsLabelServiceAdapter implements InsLabelServiceAdapter {
         execOnServiceChain(
                 "refreshLabelsToInstance",
                 insLabelAccessService -> {
-                    insLabelAccessService.refreshLabelsToInstance(labels, serviceInstance);
+                    try {
+                        insLabelAccessService.refreshLabelsToInstance(labels, serviceInstance);
+                    } catch (InstanceErrorException e) {
+                        LOG.error("Failed to attachLabelToInstance", e);
+                    }
                     return true;
                 },
                 true,
@@ -155,6 +168,15 @@ public class DefaultInsLabelServiceAdapter implements InsLabelServiceAdapter {
                 true);
     }
 
+    @Override
+    public List<ServiceInstance> getInstancesByNames(String appName) {
+        return execOnServiceChain(
+                "getInstancesByNames",
+                insLabelAccessService -> insLabelAccessService.getInstancesByNames(appName),
+                false,
+                false);
+    }
+
     @Override
     public void evictCache() {
         // Empty
diff --git a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/service/impl/EurekaInsLabelService.java b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/service/impl/EurekaInsLabelService.java
index 06ce1fdba..e0e40a8c9 100644
--- a/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/service/impl/EurekaInsLabelService.java
+++ b/linkis-public-enhancements/linkis-publicservice/linkis-instance-label/linkis-instance-label-server/src/main/java/org/apache/linkis/instance/label/service/impl/EurekaInsLabelService.java
@@ -75,6 +75,11 @@ public class EurekaInsLabelService implements InsLabelAccessService {
         return null;
     }
 
+    @Override
+    public List<ServiceInstance> getInstancesByNames(String appName) {
+        return null;
+    }
+
     @Override
     public void removeInstance(ServiceInstance serviceInstance) {}
 
diff --git a/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-core/src/main/scala/org/apache/linkis/gateway/config/GatewayConfiguration.scala b/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-core/src/main/scala/org/apache/linkis/gateway/config/GatewayConfiguration.scala
index cb69cc584..5f1644a95 100644
--- a/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-core/src/main/scala/org/apache/linkis/gateway/config/GatewayConfiguration.scala
+++ b/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-core/src/main/scala/org/apache/linkis/gateway/config/GatewayConfiguration.scala
@@ -80,4 +80,7 @@ object GatewayConfiguration {
   val THIS_GATEWAY_SCHEMA = CommonVars("wds.linkis.gateway.this.schema", "")
 
   val ENABLE_WATER_MARK = CommonVars("wds.linkis.web.enable.water.mark", true)
+
+  val ROUTER_SERVER_LIST = CommonVars("wds.linkis.entrance.name", ENTRANCE_SPRING_NAME.getValue, "List of supported routing services")
+
 }
diff --git a/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/route/AbstractLabelGatewayRouter.scala b/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/route/AbstractLabelGatewayRouter.scala
index 3913382ea..a94eccf7d 100644
--- a/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/route/AbstractLabelGatewayRouter.scala
+++ b/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/route/AbstractLabelGatewayRouter.scala
@@ -17,16 +17,22 @@
  
 package org.apache.linkis.gateway.ujes.route
 
-import org.apache.commons.lang.StringUtils
+import org.apache.commons.lang3.StringUtils
 import org.apache.linkis.common.ServiceInstance
 import org.apache.linkis.common.utils.Logging
 import org.apache.linkis.gateway.http.GatewayContext
 import org.apache.linkis.gateway.route.AbstractGatewayRouter
 import org.apache.linkis.instance.label.service.InsLabelService
 import org.apache.linkis.manager.label.entity.route.RouteLabel
-
 import java.util
+
 import javax.annotation.Resource
+import org.apache.linkis.gateway.config.GatewayConfiguration
+import org.apache.linkis.gateway.exception.GatewayErrorException
+import org.apache.linkis.manager.label.utils.LabelUtils
+import org.apache.linkis.rpc.interceptor.ServiceInstanceUtils
+
+import scala.collection.JavaConverters._
 
 abstract class AbstractLabelGatewayRouter extends AbstractGatewayRouter with Logging {
 
@@ -47,20 +53,38 @@ abstract class AbstractLabelGatewayRouter extends AbstractGatewayRouter with Log
     if (StringUtils.isNotBlank(serviceInstance.getApplicationName) && StringUtils.isNotBlank(serviceInstance.getInstance)) {
       return serviceInstance
     }
-    // TODO: It's probably better to throw exception here
-    if (null == routeLabels || routeLabels.isEmpty) {
+    val applicationName = serviceInstance.getApplicationName
+    if (! GatewayConfiguration.ROUTER_SERVER_LIST.getValue.contains(applicationName)) {
+      // Ignore the router using application name
       return null
     }
 
-    val candidateServices = insLabelService.searchInstancesByLabels(routeLabels)
-    val canSelectInstances = if (null == candidateServices || candidateServices.isEmpty) {
-      val labelRelatedInstances = Option(insLabelService.searchLabelRelatedInstances(serviceInstance))
-      removeAllFromRegistry(serviceInstance.getApplicationName,
-        labelRelatedInstances.getOrElse(new util.ArrayList[ServiceInstance]()))
+
+    val canSelectInstances = if (null == routeLabels || routeLabels.isEmpty) {
+      getDefaultInstances(applicationName)
     } else {
-      candidateServices
+      val candidateServices = insLabelService.searchInstancesByLabels(routeLabels)
+      if (null == candidateServices || candidateServices.isEmpty) {
+        throw new GatewayErrorException(11011, s"Cannot route to the corresponding service, URL: ${gatewayContext.getRequest.getRequestURI} RouteLabel: ${LabelUtils.Jackson.toJson(routeLabels, null)}")
+      } else {
+        candidateServices
+      }
     }
-    selectInstance(gatewayContext, canSelectInstances)
+
+    val instance = selectInstance(gatewayContext, canSelectInstances)
+    if (null == instance || StringUtils.isBlank(instance.getInstance)) {
+      throw new GatewayErrorException(11011, s"There are no services available in the registry URL: ${gatewayContext.getRequest.getRequestURI}")
+    }
+    instance
+  }
+
+  protected def getDefaultInstances(applicationName: String): util.List[ServiceInstance] = {
+    val instances = ServiceInstanceUtils.getRPCServerLoader.getServiceInstances(applicationName)
+    val allInstances = new util.ArrayList[ServiceInstance]()
+    if (null != instances && instances.nonEmpty) allInstances.addAll(instances.toList.asJava)
+    val labelInstances = insLabelService.getInstancesByNames(applicationName)
+    allInstances.removeAll(labelInstances)
+    allInstances
   }
 
   /**
diff --git a/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/route/DefaultLabelGatewayRouter.scala b/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/route/DefaultLabelGatewayRouter.scala
index 48b84fe33..4107b6b7a 100644
--- a/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/route/DefaultLabelGatewayRouter.scala
+++ b/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/route/DefaultLabelGatewayRouter.scala
@@ -5,17 +5,18 @@
  * 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.gateway.ujes.route
+
 import java.util
 
 import org.apache.linkis.common.ServiceInstance
@@ -29,7 +30,8 @@ import scala.collection.JavaConversions._
 import scala.util.Random
 
 class DefaultLabelGatewayRouter(var routeLabelParsers: util.List[RouteLabelParser]) extends AbstractLabelGatewayRouter {
-  if(Option(routeLabelParsers).isEmpty){
+
+  if (Option(routeLabelParsers).isEmpty) {
      routeLabelParsers = new util.ArrayList[RouteLabelParser]()
   }
   override protected def parseToRouteLabels(gatewayContext: GatewayContext): util.List[RouteLabel] = {
@@ -41,14 +43,14 @@ class DefaultLabelGatewayRouter(var routeLabelParsers: util.List[RouteLabelParse
   }
 
   override protected def selectInstance(gatewayContext: GatewayContext, candidates: util.List[ServiceInstance]): ServiceInstance = {
-    if(candidates.size() <= 0){
+    if (candidates.size() <= 0) {
       return null
     }
-    if(gatewayContext.getGatewayRoute.getServiceInstance != null && StringUtils.isNotBlank(gatewayContext.getGatewayRoute.getServiceInstance.getApplicationName)){
-      val applicationName:String =gatewayContext.getGatewayRoute.getServiceInstance.getApplicationName
-      val filterCandidates =candidates.filter(serviceInstance => serviceInstance.getApplicationName.equalsIgnoreCase(applicationName))
+    if (gatewayContext.getGatewayRoute.getServiceInstance != null && StringUtils.isNotBlank(gatewayContext.getGatewayRoute.getServiceInstance.getApplicationName)) {
+      val applicationName: String = gatewayContext.getGatewayRoute.getServiceInstance.getApplicationName
+      val filterCandidates = candidates.filter(serviceInstance => serviceInstance.getApplicationName.equalsIgnoreCase(applicationName))
       roulette(filterCandidates)
-    }else {
+    } else {
       roulette(candidates)
     }
   }
@@ -60,15 +62,15 @@ class DefaultLabelGatewayRouter(var routeLabelParsers: util.List[RouteLabelParse
    */
   private def roulette(serviceInstances: util.List[ServiceInstance]): ServiceInstance = {
     //Fetch from registry, make sure that the instances are available and the serviceId is right
-    if(serviceInstances.size()<=0){
-      throw new GatewayErrorException(11011, "ServiceInstances is empty, please check eureka service!")
+    if (serviceInstances.size() <= 0) {
+      throw new GatewayErrorException(11011, "There is no route label service with the corresponding app name")
     }
     val serviceId = serviceInstances.get(0).getApplicationName
     val filteredInstances = retainAllInRegistry(serviceId, serviceInstances)
-    filteredInstances match {
-      case _ if filteredInstances.size() == 1 => filteredInstances.get(0)
-      case _ if filteredInstances.size() > 1 => filteredInstances.get(Random.nextInt(filteredInstances.size()))
-      case _ => throw new GatewayErrorException(11012, s"Cannot find an instance in the routing chain of serviceId [" +
+    if (filteredInstances.size() > 0) {
+      filteredInstances.get(Random.nextInt(filteredInstances.size()))
+    } else {
+      throw new GatewayErrorException(11012, s"Cannot find an instance in the routing chain of serviceId [" +
         serviceId + "], please retry")
     }
   }


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