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/03/31 12:20:27 UTC

[incubator-linkis] branch dev-1.1.2 updated: Script custom variable run type and suffix constraint configuration. (#1848)

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

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


The following commit(s) were added to refs/heads/dev-1.1.2 by this push:
     new 1752b13  Script custom variable run type and suffix constraint configuration. (#1848)
1752b13 is described below

commit 1752b13ff1510a6aec61428712a6fac6c73638ef
Author: weixiao <le...@gmail.com>
AuthorDate: Thu Mar 31 20:20:17 2022 +0800

    Script custom variable run type and suffix constraint configuration. (#1848)
    
    * Script custom variable run type and suffix constraint configuration.
    Co-authored-by: leo_jie <we...@aliyun.com>
---
 .../linkis/common/utils/CodeAndRunTypeUtils.scala  | 69 ++++++++++++++++++++++
 .../common/utils/CodeAndRunTypeUtilsTest.scala     | 53 +++++++++++++++++
 .../linkis/storage/script/ScriptFsWriter.scala     |  6 +-
 .../script/compaction/PYScriptCompaction.scala     |  9 +--
 .../script/compaction/QLScriptCompaction.scala     | 10 +---
 .../script/compaction/ScalaScriptCompaction.scala  |  7 +--
 .../script/compaction/ShellScriptCompaction.scala  | 11 ++--
 .../storage/script/parser/PYScriptParser.scala     |  8 +--
 .../storage/script/parser/QLScriptParser.scala     | 10 +---
 .../storage/script/parser/ScalaScriptParser.scala  |  9 +--
 ...YScriptParser.scala => ShellScriptParser.scala} | 23 ++++----
 .../governance/common/paser/CodeParser.scala       | 20 +++++--
 .../interceptor/impl/CustomVariableUtils.scala     | 30 ++++------
 13 files changed, 183 insertions(+), 82 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
new file mode 100644
index 0000000..437e545
--- /dev/null
+++ b/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/utils/CodeAndRunTypeUtils.scala
@@ -0,0 +1,69 @@
+/*
+ * 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.common.utils
+
+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 RUN_TYPE_SQL = "sql"
+  val RUN_TYPE_PYTHON = "python"
+  val RUN_TYPE_JAVA = "java"
+  val RUN_TYPE_SCALA = "scala"
+  val RUN_TYPE_SHELL = "shell"
+
+  private var codeTypeAndRunTypeRelationMap: Map[String, List[String]] = null
+
+  private def codeTypeAndRunTypeRelationMapParser(configV: String): Map[String, List[String]] = {
+    val confDelimiter = ","
+    if (configV == null || "".equals(configV)) Map()
+    else configV.split(confDelimiter).filter(x => x != null && !"".equals(x)).map(x => {
+      val confArr = x.split("=>")
+      if (confArr.length == 2) (confArr(0), for (x <- confArr(1).split("\\|").toList) yield x.trim) else null
+    }).filter(x => x != null).toMap
+  }
+
+  def getCodeTypeAndRunTypeRelationMap: Map[String, List[String]] = {
+    if (codeTypeAndRunTypeRelationMap == null) {
+      CONF_LOCK.synchronized {
+        if (codeTypeAndRunTypeRelationMap == null) {
+          codeTypeAndRunTypeRelationMap = codeTypeAndRunTypeRelationMapParser(CODE_TYPE_AND_RUN_TYPE_RELATION.getValue)
+        }
+      }
+    }
+    codeTypeAndRunTypeRelationMap
+  }
+
+  def getRunTypeAndCodeTypeRelationMap: Map[String, String] = {
+    val codeTypeAndRunTypeRelationMap = getCodeTypeAndRunTypeRelationMap
+    if (codeTypeAndRunTypeRelationMap.isEmpty) Map()
+    else codeTypeAndRunTypeRelationMap.flatMap(x => x._2.map(y => (y, x._1)))
+  }
+
+  def getSuffixBelongToRunTypeOrNot(suffix: String, runType: String): Boolean = {
+    val codeTypeAndRunTypeRelationMap = getCodeTypeAndRunTypeRelationMap
+    if (codeTypeAndRunTypeRelationMap.isEmpty) return false
+    val suffixListOfRunType = codeTypeAndRunTypeRelationMap.getOrElse(runType, List())
+    if (suffixListOfRunType.isEmpty) return false
+    if (suffixListOfRunType.contains(suffix)) return true
+    false
+  }
+}
diff --git a/linkis-commons/linkis-common/src/test/scala/org/apache/linkis/common/utils/CodeAndRunTypeUtilsTest.scala b/linkis-commons/linkis-common/src/test/scala/org/apache/linkis/common/utils/CodeAndRunTypeUtilsTest.scala
new file mode 100644
index 0000000..299be38
--- /dev/null
+++ b/linkis-commons/linkis-common/src/test/scala/org/apache/linkis/common/utils/CodeAndRunTypeUtilsTest.scala
@@ -0,0 +1,53 @@
+/*
+ * 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.common.utils
+import org.junit.{Assert, Test}
+
+@Test
+class CodeAndRunTypeUtilsTest extends Assert {
+
+  @Test
+  def testGetCodeTypeAndRunTypeRelationMap(): Unit = {
+    val codeTypeAndRunTypeRelationMap = CodeAndRunTypeUtils.getCodeTypeAndRunTypeRelationMap
+    assert(codeTypeAndRunTypeRelationMap.nonEmpty)
+    assert(codeTypeAndRunTypeRelationMap.keySet.contains("sql"))
+    assert(codeTypeAndRunTypeRelationMap("python").size == 3)
+  }
+
+  @Test
+  def testGetRunTypeAndCodeTypeRelationMap(): Unit = {
+    val runTypeAndCodeTypeRelationMap = CodeAndRunTypeUtils.getRunTypeAndCodeTypeRelationMap
+    assert(runTypeAndCodeTypeRelationMap.nonEmpty)
+    assert(CodeAndRunTypeUtils.RUN_TYPE_SHELL.equals(runTypeAndCodeTypeRelationMap("sh")))
+    assert(CodeAndRunTypeUtils.RUN_TYPE_SQL.equals(runTypeAndCodeTypeRelationMap("psql")))
+  }
+
+  @Test
+  def testGetSuffixBelongToRunTypeOrNot(): Unit = {
+    val shell = CodeAndRunTypeUtils.getSuffixBelongToRunTypeOrNot("sh", CodeAndRunTypeUtils.RUN_TYPE_SHELL)
+    assert(shell)
+    val sql = CodeAndRunTypeUtils.getSuffixBelongToRunTypeOrNot("jdbc", CodeAndRunTypeUtils.RUN_TYPE_SQL)
+    assert(sql)
+    val hql = CodeAndRunTypeUtils.getSuffixBelongToRunTypeOrNot("hql", CodeAndRunTypeUtils.RUN_TYPE_SQL)
+    assert(hql)
+    val python = CodeAndRunTypeUtils.getSuffixBelongToRunTypeOrNot("py", CodeAndRunTypeUtils.RUN_TYPE_PYTHON)
+    assert(python)
+    val scala = CodeAndRunTypeUtils.getSuffixBelongToRunTypeOrNot("java", CodeAndRunTypeUtils.RUN_TYPE_SCALA)
+    assert(!scala)
+  }
+}
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/ScriptFsWriter.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/ScriptFsWriter.scala
index 424e76f..b847ab4 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/ScriptFsWriter.scala
+++ b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/ScriptFsWriter.scala
@@ -22,7 +22,7 @@ import java.io.{InputStream, OutputStream}
 import org.apache.linkis.common.io.{FsPath, FsWriter, MetaData}
 import org.apache.linkis.storage.LineRecord
 import org.apache.linkis.storage.script.compaction.{PYScriptCompaction, QLScriptCompaction, ScalaScriptCompaction, ShellScriptCompaction}
-import org.apache.linkis.storage.script.parser.{PYScriptParser, QLScriptParser, ScalaScriptParser}
+import org.apache.linkis.storage.script.parser.{PYScriptParser, QLScriptParser, ScalaScriptParser, ShellScriptParser}
 import org.apache.linkis.storage.script.writer.StorageScriptFsWriter
 
 abstract class ScriptFsWriter extends FsWriter {
@@ -41,11 +41,11 @@ object ScriptFsWriter {
 
 
 object ParserFactory {
-  def listParsers(): Array[Parser] = Array(PYScriptParser(), QLScriptParser(), ScalaScriptParser())
+  def listParsers(): Array[Parser] = Array(PYScriptParser(), QLScriptParser(), ScalaScriptParser(), ShellScriptParser())
 }
 
 object Compaction {
-  def listCompactions(): Array[Compaction] = Array(PYScriptCompaction(), QLScriptCompaction(), ScalaScriptCompaction(),ShellScriptCompaction())
+  def listCompactions(): Array[Compaction] = Array(PYScriptCompaction(), QLScriptCompaction(), ScalaScriptCompaction(), ShellScriptCompaction())
 }
 
 trait Parser {
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/compaction/PYScriptCompaction.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/compaction/PYScriptCompaction.scala
index 0a1f733..ac8218a 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/compaction/PYScriptCompaction.scala
+++ b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/compaction/PYScriptCompaction.scala
@@ -17,15 +17,12 @@
  
 package org.apache.linkis.storage.script.compaction
 
+import org.apache.linkis.common.utils.CodeAndRunTypeUtils
+
 
 class PYScriptCompaction private extends CommonScriptCompaction {
 
-  override def belongTo(suffix: String): Boolean = {
-    suffix match {
-      case "python"|"py"|"sh" => true
-      case _ => false
-    }
-  }
+  override def belongTo(suffix: String): Boolean = CodeAndRunTypeUtils.getSuffixBelongToRunTypeOrNot(suffix, CodeAndRunTypeUtils.RUN_TYPE_PYTHON)
 
   override def prefix: String = "#@set"
 
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/compaction/QLScriptCompaction.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/compaction/QLScriptCompaction.scala
index 9f11fa2..18ac80c 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/compaction/QLScriptCompaction.scala
+++ b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/compaction/QLScriptCompaction.scala
@@ -17,16 +17,12 @@
  
 package org.apache.linkis.storage.script.compaction
 
+import org.apache.linkis.common.utils.CodeAndRunTypeUtils
+
 
 class QLScriptCompaction private extends CommonScriptCompaction{
 
-  override def belongTo(suffix: String): Boolean = {
-    suffix match {
-      case "sql" => true
-      case "hql" => true
-      case _ => false
-    }
-  }
+  override def belongTo(suffix: String): Boolean = CodeAndRunTypeUtils.getSuffixBelongToRunTypeOrNot(suffix, CodeAndRunTypeUtils.RUN_TYPE_SQL)
 
   override def prefix: String = "--@set"
 
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/compaction/ScalaScriptCompaction.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/compaction/ScalaScriptCompaction.scala
index 7ead967..c8f0f39 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/compaction/ScalaScriptCompaction.scala
+++ b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/compaction/ScalaScriptCompaction.scala
@@ -17,14 +17,13 @@
  
 package org.apache.linkis.storage.script.compaction
 
+import org.apache.linkis.common.utils.CodeAndRunTypeUtils
+
 
 class ScalaScriptCompaction private extends CommonScriptCompaction{
   override def prefix: String = "//@set"
 
-  override def belongTo(suffix: String): Boolean = suffix match {
-    case "scala" => true
-    case _ => false
-  }
+  override def belongTo(suffix: String): Boolean = CodeAndRunTypeUtils.getSuffixBelongToRunTypeOrNot(suffix, CodeAndRunTypeUtils.RUN_TYPE_SCALA)
 
   override def prefixConf: String = "//conf@set"
 }
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/compaction/ShellScriptCompaction.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/compaction/ShellScriptCompaction.scala
index 0067d29..723b675 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/compaction/ShellScriptCompaction.scala
+++ b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/compaction/ShellScriptCompaction.scala
@@ -17,21 +17,18 @@
  
 package org.apache.linkis.storage.script.compaction
 
+import org.apache.linkis.common.utils.CodeAndRunTypeUtils
+
 
 class ShellScriptCompaction private extends CommonScriptCompaction{
   override def prefixConf: String = "#conf@set"
 
   override def prefix: String = "#@set"
 
-  override def belongTo(suffix: String): Boolean ={
-    suffix match {
-      case "sh"=>true
-      case _=>false
-    }
-  }
+  override def belongTo(suffix: String): Boolean = CodeAndRunTypeUtils.getSuffixBelongToRunTypeOrNot(suffix, CodeAndRunTypeUtils.RUN_TYPE_SHELL)
 }
 object ShellScriptCompaction{
-  val shellScriptCompaction:ShellScriptCompaction=new ShellScriptCompaction
+  val shellScriptCompaction: ShellScriptCompaction = new ShellScriptCompaction
 
   def apply(): CommonScriptCompaction = shellScriptCompaction
 }
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/parser/PYScriptParser.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/parser/PYScriptParser.scala
index 21e420a..e167905 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/parser/PYScriptParser.scala
+++ b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/parser/PYScriptParser.scala
@@ -17,16 +17,12 @@
  
 package org.apache.linkis.storage.script.parser
 
+import org.apache.linkis.common.utils.CodeAndRunTypeUtils
 
 class PYScriptParser private extends CommonScriptParser {
   override def prefix: String = "#@set"
 
-  override def belongTo(suffix: String): Boolean = {
-    suffix match {
-      case "python"|"py"|"sh" => true
-      case _ => false
-    }
-  }
+  override def belongTo(suffix: String): Boolean = CodeAndRunTypeUtils.getSuffixBelongToRunTypeOrNot(suffix, CodeAndRunTypeUtils.RUN_TYPE_PYTHON)
 
   override def prefixConf: String = "#conf@set"
 }
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/parser/QLScriptParser.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/parser/QLScriptParser.scala
index 39db8f6..99f8d7f 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/parser/QLScriptParser.scala
+++ b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/parser/QLScriptParser.scala
@@ -17,17 +17,13 @@
  
 package org.apache.linkis.storage.script.parser
 
+import org.apache.linkis.common.utils.CodeAndRunTypeUtils
+
 
 class QLScriptParser private extends CommonScriptParser {
   override def prefix: String = "--@set"
 
-  override def belongTo(suffix: String): Boolean = {
-    suffix match {
-      case "sql" => true
-      case "hql" => true
-      case _ => false
-    }
-  }
+  override def belongTo(suffix: String): Boolean = CodeAndRunTypeUtils.getSuffixBelongToRunTypeOrNot(suffix, CodeAndRunTypeUtils.RUN_TYPE_SQL)
 
   override def prefixConf: String = "--conf@set"
 }
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/parser/ScalaScriptParser.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/parser/ScalaScriptParser.scala
index ed18c9c..fa40389 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/parser/ScalaScriptParser.scala
+++ b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/parser/ScalaScriptParser.scala
@@ -17,17 +17,14 @@
  
 package org.apache.linkis.storage.script.parser
 
+import org.apache.linkis.common.utils.CodeAndRunTypeUtils
+
 
 class ScalaScriptParser private extends CommonScriptParser {
   //todo To be determined(待定)
   override def prefix: String = "//@set"
 
-  override def belongTo(suffix: String): Boolean = {
-    suffix match {
-      case "scala" => true
-      case _ => false
-    }
-  }
+  override def belongTo(suffix: String): Boolean = CodeAndRunTypeUtils.getSuffixBelongToRunTypeOrNot(suffix, CodeAndRunTypeUtils.RUN_TYPE_SCALA)
 
   override def prefixConf: String = "//conf@set"
 }
diff --git a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/parser/PYScriptParser.scala b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/parser/ShellScriptParser.scala
similarity index 69%
copy from linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/parser/PYScriptParser.scala
copy to linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/parser/ShellScriptParser.scala
index 21e420a..ddbf6d9 100644
--- a/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/parser/PYScriptParser.scala
+++ b/linkis-commons/linkis-storage/src/main/scala/org/apache/linkis/storage/script/parser/ShellScriptParser.scala
@@ -5,35 +5,32 @@
  * 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.storage.script.parser
 
+import org.apache.linkis.common.utils.CodeAndRunTypeUtils
 
-class PYScriptParser private extends CommonScriptParser {
+class ShellScriptParser private extends CommonScriptParser {
   override def prefix: String = "#@set"
 
-  override def belongTo(suffix: String): Boolean = {
-    suffix match {
-      case "python"|"py"|"sh" => true
-      case _ => false
-    }
-  }
+  override def belongTo(suffix: String): Boolean = CodeAndRunTypeUtils.getSuffixBelongToRunTypeOrNot(suffix, CodeAndRunTypeUtils.RUN_TYPE_SHELL)
 
   override def prefixConf: String = "#conf@set"
 }
 
-object PYScriptParser {
-  val pYScriptParser: PYScriptParser = new PYScriptParser
+object ShellScriptParser {
+  val shellScriptParser: ShellScriptParser = new ShellScriptParser
 
-  def apply(): CommonScriptParser = pYScriptParser
+  def apply(): CommonScriptParser = shellScriptParser
 
 }
+
diff --git a/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/paser/CodeParser.scala b/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/paser/CodeParser.scala
index cc595a7..d5be0b5 100644
--- a/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/paser/CodeParser.scala
+++ b/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/paser/CodeParser.scala
@@ -20,9 +20,11 @@ package org.apache.linkis.governance.common.paser
 import org.apache.linkis.common.utils.Logging
 import org.apache.linkis.governance.common.conf.GovernanceCommonConf
 import org.apache.linkis.governance.common.paser.CodeType.CodeType
+import org.apache.linkis.common.utils.CodeAndRunTypeUtils
 import org.apache.commons.lang.StringUtils
 import org.slf4j.{Logger, LoggerFactory}
 
+import scala.collection.JavaConversions._
 import scala.collection.mutable
 import scala.collection.mutable.ArrayBuffer
 import java.util
@@ -300,12 +302,18 @@ object CodeType extends Enumeration {
   type CodeType = Value
   val Python, SQL, Scala, Shell, Other, Remain, JSON = Value
 
-  def getType(codeType: String): CodeType = codeType.toLowerCase() match {
-    case "python" | "pyspark" | "py" => Python
-    case "sql" | "hql" | "psql" => SQL
-    case "scala" => Scala
-    case "shell" | "sh" => Shell
-    case _ => Other
+  def getType(codeType: String): CodeType = {
+    val runTypeAndCodeTypeRelationMap: Map[String, String] = CodeAndRunTypeUtils.getRunTypeAndCodeTypeRelationMap
+    if (runTypeAndCodeTypeRelationMap.isEmpty || ! runTypeAndCodeTypeRelationMap.contains(codeType.toLowerCase)) return Other
+
+    val runType = runTypeAndCodeTypeRelationMap(codeType.toLowerCase)
+    runType match {
+      case CodeAndRunTypeUtils.RUN_TYPE_PYTHON => Python
+      case CodeAndRunTypeUtils.RUN_TYPE_SQL => SQL
+      case CodeAndRunTypeUtils.RUN_TYPE_SCALA => Scala
+      case CodeAndRunTypeUtils.RUN_TYPE_SHELL => Shell
+      case _ => Other
+    }
   }
 }
 
diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/CustomVariableUtils.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/CustomVariableUtils.scala
index 4991292..1f09fa6 100644
--- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/CustomVariableUtils.scala
+++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/CustomVariableUtils.scala
@@ -20,7 +20,7 @@ package org.apache.linkis.entrance.interceptor.impl
 import org.apache.commons.lang.StringUtils
 import org.apache.commons.lang.time.DateUtils
 import org.apache.linkis.common.conf.Configuration
-import org.apache.linkis.common.utils.{Logging, Utils}
+import org.apache.linkis.common.utils.{CodeAndRunTypeUtils, Logging, Utils}
 import org.apache.linkis.entrance.interceptor.exception.VarSubstitutionException
 import org.apache.linkis.entrance.interceptor.impl.CustomVariableUtils.{dateFormatLocal, dateFormatStdLocal}
 import org.apache.linkis.governance.common.entity.job.JobRequest
@@ -40,10 +40,6 @@ import scala.util.control.Exception._
 //TODO: optimize code, 拆分Utils类
 object CustomVariableUtils extends Logging {
   //hql sql jdbc to sql python to py
-  private val SQL_TYPE = "sql"
-  private val PY_TYPE = "python"
-  private val JAVA_TYPE: String = "java"
-  private val SCALA_TYPE: String = "scala"
   private val R_TYPE: String = "r"
   private val RUN_DATE = "run_date"
   private val TEAM: String = "team"
@@ -91,14 +87,14 @@ object CustomVariableUtils extends Logging {
    */
   def replaceCustomVar(jobRequest: JobRequest, runType: String): (Boolean, String) = {
     val code: String = jobRequest.getExecutionCode
-    var codeType = SQL_TYPE
-    runType match {
-      case "hql" | "sql" | "jdbc" | "hive" | "psql" => codeType = SQL_TYPE
-      case "python" | "py" => codeType = PY_TYPE
-      case "java" => codeType = JAVA_TYPE
-      case "scala" => codeType = SCALA_TYPE
-      case "sh" | "shell" => codeType = SQL_TYPE
-      case _ => return (false, code)
+    val codeTypeAndRunTypeRelationMap = CodeAndRunTypeUtils.getCodeTypeAndRunTypeRelationMap
+    if (codeTypeAndRunTypeRelationMap.isEmpty) {
+      return (false, code)
+    }
+    val allowedRunTypeMap: Map[String, String] = CodeAndRunTypeUtils.getRunTypeAndCodeTypeRelationMap
+    val codeType = allowedRunTypeMap.getOrDefault(runType, null)
+    if (codeType == null) {
+      return (false, code)
     }
 
     var run_date: CustomDateType = null
@@ -337,13 +333,13 @@ object CustomVariableUtils extends Logging {
     var errString:String = null
 
     codeType match {
-      case SQL_TYPE => varString = """\s*--@set\s*.+\s*"""
+      case CodeAndRunTypeUtils.RUN_TYPE_SQL => varString = """\s*--@set\s*.+\s*"""
         errString = """\s*--@.*"""
-      case PY_TYPE => varString = """\s*#@set\s*.+\s*"""
+      case CodeAndRunTypeUtils.RUN_TYPE_PYTHON | CodeAndRunTypeUtils.RUN_TYPE_SHELL => varString = """\s*#@set\s*.+\s*"""
         errString = """\s*#@"""
-      case SCALA_TYPE => varString = """\s*//@set\s*.+\s*"""
+      case CodeAndRunTypeUtils.RUN_TYPE_SCALA => varString = """\s*//@set\s*.+\s*"""
         errString = """\s*//@.+"""
-      case JAVA_TYPE => varString = """\s*!!@set\s*.+\s*"""
+      case CodeAndRunTypeUtils.RUN_TYPE_JAVA => varString = """\s*!!@set\s*.+\s*"""
     }
 
     val customRegex = varString.r.unanchored

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