You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@linkis.apache.org by pe...@apache.org on 2022/06/27 10:00:22 UTC

[incubator-linkis] branch dev-1.2.0 updated: Optimized to automatically refresh all LinkisManager caches and add trino ec type (#2323)

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

peacewong 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 f0f8a7894 Optimized to automatically refresh all LinkisManager caches and add trino ec type (#2323)
f0f8a7894 is described below

commit f0f8a7894aa67a88312cd229f4bfe91a7e4a00f0
Author: peacewong <wp...@gmail.com>
AuthorDate: Mon Jun 27 18:00:16 2022 +0800

    Optimized to automatically refresh all LinkisManager caches and add trino ec type (#2323)
    
    * Added trino engine type close #2217
    
    * Modify the configuration to clean up the broadcast message close #2077
    
    * fix build error
---
 .../linkis/common/utils/CodeAndRunTypeUtils.scala  |  2 +-
 .../linkis/storage/conf/LinkisStorageConf.scala    |  2 +-
 .../protocol/conf/RemoveCacheConfRequest.scala}    | 33 +++-----------
 .../protocol/conf/RequestQueryGlobalConfig.scala   | 13 ++++--
 .../linkis/manager/am/conf/AMConfiguration.scala   |  4 +-
 .../cache/ConfCacheRemoveBroadcastListener.scala   | 51 ++++++++++++++++++++++
 .../am/service/cache/DefaultConfCacheService.scala | 49 ---------------------
 .../manager/label/conf/LabelCommonConfig.java      |  3 ++
 .../manager/label/entity/engine/EngineType.scala   |  2 +
 .../manager/label/entity/engine/RunType.scala      |  2 +
 .../label/utils/EngineTypeLabelCreator.java        |  3 ++
 .../protocol/conf/RemoveCacheConfRequest.java      | 45 -------------------
 .../restful/api/ConfigurationRestfulApi.java       |  1 +
 .../service/ConfigurationService.scala             | 19 ++++----
 14 files changed, 91 insertions(+), 138 deletions(-)

diff --git a/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/utils/CodeAndRunTypeUtils.scala b/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/utils/CodeAndRunTypeUtils.scala
index 673f32e55..3930f548b 100644
--- a/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/utils/CodeAndRunTypeUtils.scala
+++ b/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/utils/CodeAndRunTypeUtils.scala
@@ -23,7 +23,7 @@ import org.apache.linkis.common.conf.CommonVars
 object CodeAndRunTypeUtils {
   private val CONF_LOCK = new Object()
 
-  val CODE_TYPE_AND_RUN_TYPE_RELATION = CommonVars("wds.linkis.codeType.runType.relation", "sql=>sql|hql|jdbc|hive|psql|fql,python=>python|py|pyspark,java=>java,scala=>scala,shell=>sh|shell")
+  val CODE_TYPE_AND_RUN_TYPE_RELATION = CommonVars("wds.linkis.codeType.runType.relation", "sql=>sql|hql|jdbc|hive|psql|fql|tsql,python=>python|py|pyspark,java=>java,scala=>scala,shell=>sh|shell")
 
   val RUN_TYPE_SQL = "sql"
   val RUN_TYPE_PYTHON = "python"
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/conf/LinkisStorageConf.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/conf/LinkisStorageConf.scala
index ad2844fd9..9d45a773d 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/conf/LinkisStorageConf.scala
+++ b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/conf/LinkisStorageConf.scala
@@ -34,7 +34,7 @@ object LinkisStorageConf {
 
   val ROW_BYTE_MAX_LEN = ByteTimeUtils.byteStringAsBytes(ROW_BYTE_MAX_LEN_STR)
 
-  val FILE_TYPE = CommonVars("wds.linkis.storage.file.type", "dolphin,sql,scala,py,hql,python,out,log,text,sh,jdbc,ngql,psql,fql").getValue
+  val FILE_TYPE = CommonVars("wds.linkis.storage.file.type", "dolphin,sql,scala,py,hql,python,out,log,text,sh,jdbc,ngql,psql,fql,tsql").getValue
 
   private var fileTypeArr: Array[String] = null
 
diff --git a/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/entity/engine/RunType.scala b/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/protocol/conf/RemoveCacheConfRequest.scala
similarity index 52%
copy from linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/entity/engine/RunType.scala
copy to linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/protocol/conf/RemoveCacheConfRequest.scala
index e6af7bc70..216341830 100644
--- a/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/entity/engine/RunType.scala
+++ b/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/protocol/conf/RemoveCacheConfRequest.scala
@@ -5,40 +5,19 @@
  * 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.label.entity.engine
 
-object RunType extends Enumeration {
+package org.apache.linkis.governance.common.protocol.conf
 
-  type RunType = Value
-  val SQL = Value("sql")
-  val HIVE = Value("hql")
-  val SCALA = Value("scala")
-  val PYTHON = Value("python")
-  val JAVA = Value("java")
-  val PYSPARK = Value("py")
-  val R = Value("r")
-  val STORAGE = Value("out")
-  val SHELL = Value("shell")
-  val IO_FILE = Value("io_file")
-  val IO_HDFS = Value("io_hdfs")
-  val FPS = Value("fps")
-  val PIPELINE = Value("pipeline")
-  val JDBC = Value("jdbc")
-  val PRESTO_SQL = Value("psql")
-  val JAR = Value("jar")
-  val APPCONN = Value("appconn")
-  val FUNCTION_MDQ_TYPE = Value("function.mdq")
-  val ES_SQL = Value("essql")
-  val ES_JSON = Value("esjson")
+import org.apache.linkis.manager.label.entity.engine.{EngineTypeLabel, UserCreatorLabel}
+import org.apache.linkis.protocol.BroadcastProtocol
 
-}
+case class RemoveCacheConfRequest(userCreatorLabel: UserCreatorLabel, engineTypeLabel: EngineTypeLabel) extends BroadcastProtocol
diff --git a/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/protocol/conf/RequestQueryGlobalConfig.scala b/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/protocol/conf/RequestQueryGlobalConfig.scala
index 141228e25..4b33a3715 100644
--- a/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/protocol/conf/RequestQueryGlobalConfig.scala
+++ b/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/protocol/conf/RequestQueryGlobalConfig.scala
@@ -26,24 +26,29 @@ trait ConfigProtocol
 
 case class RequestQueryGlobalConfig(username: String) extends CacheableProtocol with RetryableProtocol with ConfigProtocol with RequestProtocol {
   override def toString: String = {
-    RequestQueryGlobalConfig.getClass.getName + "," + username
+    s"RequestQueryGlobalConfig_${username}"
   }
 }
 
 case class RequestQueryEngineConfig(userCreatorLabel: UserCreatorLabel, engineTypeLabel: EngineTypeLabel, filter: String = null) extends CacheableProtocol with RetryableProtocol with ConfigProtocol{
   override def toString: String = {
-    RequestQueryEngineConfig.getClass.getName + "," + userCreatorLabel.getStringValue + "," + engineTypeLabel.getStringValue
+   s"RequestQueryEngineConfig_${userCreatorLabel.getStringValue}_${engineTypeLabel.getStringValue}"
   }
 }
 
 case class RequestQueryEngineConfigWithGlobalConfig(userCreatorLabel: UserCreatorLabel, engineTypeLabel: EngineTypeLabel, filter: String = null) extends CacheableProtocol with RetryableProtocol with ConfigProtocol{
   override def toString: String = {
-    RequestQueryEngineConfigWithGlobalConfig.getClass.getName + "," + userCreatorLabel.getStringValue + "," + engineTypeLabel.getStringValue
+    s"RequestQueryEngineConfigWithGlobalConfig_${userCreatorLabel.getStringValue}_${engineTypeLabel.getStringValue}"
   }
 }
 
 
-case class RequestQueryEngineTypeDefault(engineTypeLabel: EngineTypeLabel) extends CacheableProtocol with RetryableProtocol with ConfigProtocol
+case class RequestQueryEngineTypeDefault(engineTypeLabel: EngineTypeLabel) extends CacheableProtocol with RetryableProtocol with ConfigProtocol {
+  override def toString: String = {
+   s"RequestQueryEngineTypeDefault_${engineTypeLabel.getStringValue}"
+  }
+
+}
 
 case class RequestConfigByLabel(labels: java.util.List[Label[_]], isMerge: Boolean = true) extends CacheableProtocol with RetryableProtocol with ConfigProtocol
 
diff --git a/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/am/conf/AMConfiguration.scala b/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/am/conf/AMConfiguration.scala
index 4baa53bee..2aff4ab86 100644
--- a/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/am/conf/AMConfiguration.scala
+++ b/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/am/conf/AMConfiguration.scala
@@ -55,7 +55,7 @@ object AMConfiguration {
 
   val ENGINECONN_DEBUG_ENABLED = CommonVars("wds.linkis.engineconn.debug.mode.enable", false)
 
-  val MULTI_USER_ENGINE_TYPES = CommonVars("wds.linkis.multi.user.engine.types", "jdbc,es,presto,io_file,appconn,openlookeng")
+  val MULTI_USER_ENGINE_TYPES = CommonVars("wds.linkis.multi.user.engine.types", "jdbc,es,presto,io_file,appconn,openlookeng,trino")
 
   val MULTI_USER_ENGINE_USER = CommonVars("wds.linkis.multi.user.engine.user", getDefaultMultiEngineUser)
 
@@ -77,7 +77,7 @@ object AMConfiguration {
 
   private def getDefaultMultiEngineUser(): String = {
     val jvmUser = Utils.getJvmUser
-    s""" {jdbc:"$jvmUser", es: "$jvmUser", presto:"$jvmUser", appconn:"$jvmUser", openlookeng:"$jvmUser", io_file:"root"}"""
+    s""" {jdbc:"$jvmUser", es: "$jvmUser", presto:"$jvmUser", appconn:"$jvmUser", openlookeng:"$jvmUser", trino:"$jvmUser", io_file:"root"}"""
   }
 
   def isAdmin(userName: String): Boolean = {
diff --git a/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/am/service/cache/ConfCacheRemoveBroadcastListener.scala b/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/am/service/cache/ConfCacheRemoveBroadcastListener.scala
new file mode 100644
index 000000000..b04259a11
--- /dev/null
+++ b/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/am/service/cache/ConfCacheRemoveBroadcastListener.scala
@@ -0,0 +1,51 @@
+/*
+ * 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.am.service.cache
+
+import org.apache.linkis.common.utils.Logging
+import org.apache.linkis.governance.common.protocol.conf.{RemoveCacheConfRequest, RequestQueryEngineConfig, RequestQueryEngineConfigWithGlobalConfig, RequestQueryGlobalConfig}
+import org.apache.linkis.protocol.BroadcastProtocol
+import org.apache.linkis.rpc.interceptor.common.CacheableRPCInterceptor
+import org.apache.linkis.rpc.{BroadcastListener, Sender}
+import org.springframework.beans.factory.annotation.Autowired
+import org.springframework.stereotype.Service
+
+@Service
+class ConfCacheRemoveBroadcastListener extends BroadcastListener with Logging {
+
+  @Autowired
+  private var cacheableRPCInterceptor: CacheableRPCInterceptor = _
+
+  override def onBroadcastEvent(protocol: BroadcastProtocol, sender: Sender): Unit = protocol match {
+    case  removeCacheConfRequest: RemoveCacheConfRequest =>
+      if (removeCacheConfRequest.userCreatorLabel != null) {
+        if (removeCacheConfRequest.engineTypeLabel != null) {
+          val request = RequestQueryEngineConfig(removeCacheConfRequest.userCreatorLabel, removeCacheConfRequest.engineTypeLabel)
+          val globalRequest = RequestQueryEngineConfigWithGlobalConfig(removeCacheConfRequest.userCreatorLabel, request.engineTypeLabel, null)
+          cacheableRPCInterceptor.removeCache(request.toString)
+          cacheableRPCInterceptor.removeCache(globalRequest.toString)
+          logger.info(s"success to clear cache about configuration of ${removeCacheConfRequest.engineTypeLabel.getStringValue}-${removeCacheConfRequest.userCreatorLabel.getStringValue}")
+        } else {
+          val request = RequestQueryGlobalConfig(removeCacheConfRequest.userCreatorLabel.getUser)
+          cacheableRPCInterceptor.removeCache(request.toString)
+          logger.info(s"success to clear cache about global configuration of ${removeCacheConfRequest.userCreatorLabel.getUser}")
+        }
+      }
+    case _ =>
+  }
+}
diff --git a/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/am/service/cache/DefaultConfCacheService.scala b/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/am/service/cache/DefaultConfCacheService.scala
deleted file mode 100644
index f837c480e..000000000
--- a/linkis-computation-governance/linkis-manager/linkis-application-manager/src/main/scala/org/apache/linkis/manager/am/service/cache/DefaultConfCacheService.scala
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.am.service.cache
-
-import org.apache.linkis.common.utils.Logging
-import org.apache.linkis.governance.common.protocol.conf.{RequestQueryEngineConfig, RequestQueryGlobalConfig}
-import org.apache.linkis.manager.common.protocol.conf.RemoveCacheConfRequest
-import org.apache.linkis.rpc.message.annotation.Receiver
-import org.apache.linkis.rpc.interceptor.common.CacheableRPCInterceptor
-import org.springframework.beans.factory.annotation.Autowired
-import org.springframework.stereotype.Service
-
-@Service
-class DefaultConfCacheService extends Logging {
-
-  @Autowired
-  private var cacheableRPCInterceptor: CacheableRPCInterceptor = _
-
-  @Receiver
-  def removeCacheConfiguration(removeCacheConfRequest: RemoveCacheConfRequest): Boolean = {
-    if (removeCacheConfRequest.getUserCreatorLabel != null) {
-      if (removeCacheConfRequest.getEngineTypeLabel != null) {
-        val request = RequestQueryEngineConfig(removeCacheConfRequest.getUserCreatorLabel, removeCacheConfRequest.getEngineTypeLabel)
-        cacheableRPCInterceptor.removeCache(request.toString)
-        info(s"success to clear cache about configuration of ${removeCacheConfRequest.getEngineTypeLabel.getStringValue}-${removeCacheConfRequest.getEngineTypeLabel.getStringValue}")
-      } else {
-        val request = RequestQueryGlobalConfig(removeCacheConfRequest.getUserCreatorLabel.getUser)
-        cacheableRPCInterceptor.removeCache(request.toString)
-        info(s"success to clear cache about global configuration of ${removeCacheConfRequest.getUserCreatorLabel.getUser}")
-      }
-    }
-    true
-  }
-}
diff --git a/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/conf/LabelCommonConfig.java b/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/conf/LabelCommonConfig.java
index e27b7baa5..5b0b23787 100644
--- a/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/conf/LabelCommonConfig.java
+++ b/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/conf/LabelCommonConfig.java
@@ -83,4 +83,7 @@ public class LabelCommonConfig {
 
     public static final CommonVars<String> ENGINE_CONN_SPRING_NAME =
             CommonVars.apply("wds.linkis.engineconn.name", "linkis-cg-engineconn");
+
+    public static final CommonVars<String> TRINO_ENGINE_CONN_VERSION =
+            CommonVars.apply("wds.linkis.trino.engineconn.version", "371");
 }
diff --git a/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/entity/engine/EngineType.scala b/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/entity/engine/EngineType.scala
index 6f220a977..b49f82596 100644
--- a/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/entity/engine/EngineType.scala
+++ b/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/entity/engine/EngineType.scala
@@ -57,6 +57,8 @@ object EngineType extends Enumeration with Logging {
 
   val OPENLOOKENG = Value("openlookeng")
 
+  val TRINO = Value("trino")
+
   val ELASTICSEARCH = Value("elasticsearch")
 
   def mapFsTypeToEngineType(fsType: String): String = {
diff --git a/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/entity/engine/RunType.scala b/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/entity/engine/RunType.scala
index e6af7bc70..33601c725 100644
--- a/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/entity/engine/RunType.scala
+++ b/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/entity/engine/RunType.scala
@@ -41,4 +41,6 @@ object RunType extends Enumeration {
   val ES_SQL = Value("essql")
   val ES_JSON = Value("esjson")
 
+  val TRINO_SQL = Value("tsql")
+
 }
diff --git a/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/utils/EngineTypeLabelCreator.java b/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/utils/EngineTypeLabelCreator.java
index 6c1783190..56e6c3c9d 100644
--- a/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/utils/EngineTypeLabelCreator.java
+++ b/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/utils/EngineTypeLabelCreator.java
@@ -86,6 +86,9 @@ public class EngineTypeLabelCreator {
                     defaultVersion.put(
                             EngineType.OPENLOOKENG().toString(),
                             LabelCommonConfig.OPENLOOKENG_ENGINE_VERSION.getValue());
+                    defaultVersion.put(
+                            EngineType.TRINO().toString(),
+                            LabelCommonConfig.TRINO_ENGINE_CONN_VERSION.getValue());
                     defaultVersion.put("*", "*");
                 }
             }
diff --git a/linkis-computation-governance/linkis-manager/linkis-manager-common/src/main/java/org/apache/linkis/manager/common/protocol/conf/RemoveCacheConfRequest.java b/linkis-computation-governance/linkis-manager/linkis-manager-common/src/main/java/org/apache/linkis/manager/common/protocol/conf/RemoveCacheConfRequest.java
deleted file mode 100644
index a50b1cc6a..000000000
--- a/linkis-computation-governance/linkis-manager/linkis-manager-common/src/main/java/org/apache/linkis/manager/common/protocol/conf/RemoveCacheConfRequest.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.protocol.conf;
-
-import org.apache.linkis.manager.label.entity.engine.EngineTypeLabel;
-import org.apache.linkis.manager.label.entity.engine.UserCreatorLabel;
-import org.apache.linkis.protocol.message.RequestProtocol;
-
-public class RemoveCacheConfRequest implements RequestProtocol {
-
-    private UserCreatorLabel userCreatorLabel;
-
-    private EngineTypeLabel engineTypeLabel;
-
-    public UserCreatorLabel getUserCreatorLabel() {
-        return userCreatorLabel;
-    }
-
-    public void setUserCreatorLabel(UserCreatorLabel userCreatorLabel) {
-        this.userCreatorLabel = userCreatorLabel;
-    }
-
-    public EngineTypeLabel getEngineTypeLabel() {
-        return engineTypeLabel;
-    }
-
-    public void setEngineTypeLabel(EngineTypeLabel engineTypeLabel) {
-        this.engineTypeLabel = engineTypeLabel;
-    }
-}
diff --git a/linkis-public-enhancements/linkis-publicservice/linkis-configuration/src/main/java/org/apache/linkis/configuration/restful/api/ConfigurationRestfulApi.java b/linkis-public-enhancements/linkis-publicservice/linkis-configuration/src/main/java/org/apache/linkis/configuration/restful/api/ConfigurationRestfulApi.java
index 3d085c7d7..409467dbc 100644
--- a/linkis-public-enhancements/linkis-publicservice/linkis-configuration/src/main/java/org/apache/linkis/configuration/restful/api/ConfigurationRestfulApi.java
+++ b/linkis-public-enhancements/linkis-publicservice/linkis-configuration/src/main/java/org/apache/linkis/configuration/restful/api/ConfigurationRestfulApi.java
@@ -307,6 +307,7 @@ public class ConfigurationRestfulApi {
         configKeyValue.setConfigValue(value);
 
         ConfigValue configValue = configKeyService.saveConfigValue(configKeyValue, labelList);
+        configurationService.clearAMCacheConf(username, creator, engineType, version);
         return Message.ok().data("configValue", configValue);
     }
 
diff --git a/linkis-public-enhancements/linkis-publicservice/linkis-configuration/src/main/scala/org/apache/linkis/configuration/service/ConfigurationService.scala b/linkis-public-enhancements/linkis-publicservice/linkis-configuration/src/main/scala/org/apache/linkis/configuration/service/ConfigurationService.scala
index 24d8f1620..536fb2352 100644
--- a/linkis-public-enhancements/linkis-publicservice/linkis-configuration/src/main/scala/org/apache/linkis/configuration/service/ConfigurationService.scala
+++ b/linkis-public-enhancements/linkis-publicservice/linkis-configuration/src/main/scala/org/apache/linkis/configuration/service/ConfigurationService.scala
@@ -25,8 +25,7 @@ import org.apache.linkis.configuration.entity._
 import org.apache.linkis.configuration.exception.ConfigurationException
 import org.apache.linkis.configuration.util.{LabelEntityParser, LabelParameterParser}
 import org.apache.linkis.configuration.validate.ValidatorManager
-import org.apache.linkis.governance.common.protocol.conf.ResponseQueryConfig
-import org.apache.linkis.manager.common.protocol.conf.RemoveCacheConfRequest
+import org.apache.linkis.governance.common.protocol.conf.{RemoveCacheConfRequest, ResponseQueryConfig}
 import org.apache.linkis.manager.label.builder.CombinedLabelBuilder
 import org.apache.linkis.manager.label.builder.factory.LabelBuilderFactoryContext
 import org.apache.linkis.manager.label.entity.engine.{EngineTypeLabel, UserCreatorLabel}
@@ -132,17 +131,19 @@ class ConfigurationService extends Logging {
 
   def clearAMCacheConf(username: String, creator: String, engine: String, version: String): Unit = {
     val sender = Sender.getSender(Configuration.MANAGER_SPRING_NAME.getValue)
-    if(StringUtils.isNotEmpty(username)) {
+    if(StringUtils.isNotBlank(username)) {
       val userCreatorLabel = LabelBuilderFactoryContext.getLabelBuilderFactory.createLabel(classOf[UserCreatorLabel])
       userCreatorLabel.setUser(username)
       userCreatorLabel.setCreator(creator)
-      val request = new RemoveCacheConfRequest
-      request.setUserCreatorLabel(userCreatorLabel)
-      if(StringUtils.isNotEmpty(engine) && StringUtils.isNotEmpty(version)) {
-        val engineTypeLabel = EngineTypeLabelCreator.createEngineTypeLabel(engine)
-        engineTypeLabel.setVersion(version)
-        request.setEngineTypeLabel(engineTypeLabel)
+      val engineTypeLabel: EngineTypeLabel = if (StringUtils.isNotBlank(engine) && StringUtils.isNotBlank(version)) {
+        val label = EngineTypeLabelCreator.createEngineTypeLabel(engine)
+        label.setVersion(version)
+        label
+      } else {
+        null
       }
+      val request = RemoveCacheConfRequest(userCreatorLabel, engineTypeLabel)
+      logger.info(s"Broadcast cleanup message to manager $request")
       sender.ask(request)
     }
   }


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