You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kyuubi.apache.org by ya...@apache.org on 2022/04/07 04:16:50 UTC

[incubator-kyuubi] branch master updated: [KYUUBI #2084][FOLLOWUP] Support arbitrary parameters for KyuubiConf

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

yao pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-kyuubi.git


The following commit(s) were added to refs/heads/master by this push:
     new 0ed865f70 [KYUUBI #2084][FOLLOWUP] Support arbitrary parameters for KyuubiConf
0ed865f70 is described below

commit 0ed865f704e3e7c47a7d2172544c1b1834d3e59d
Author: Kent Yao <ya...@apache.org>
AuthorDate: Thu Apr 7 12:16:41 2022 +0800

    [KYUUBI #2084][FOLLOWUP] Support arbitrary parameters for KyuubiConf
    
    ### _Why are the changes needed?_
    
    In this PR, we narrow the scope of 8f15622d9beef8d3b15602f3180a21ec86c75623 to support arbitrary parameters which configured in kyuubi-defaults.conf only.
    
    We shall avoid propagating all `sys.props` of Kyuubi server to engine side, which is a dangerous behavior
    
    ### _How was this patch tested?_
    - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible
    
    - [ ] Add screenshots for manual tests if appropriate
    
    - [ ] [Run test](https://kyuubi.apache.org/docs/latest/develop_tools/testing.html#running-tests) locally before make a pull request
    
    Closes #2283 from yaooqinn/2084.
    
    Closes #2084
    
    434db31c [Kent Yao] fix flink
    bdd983d4 [Kent Yao] [KYUUBI #2084][FOLLOWUP] Support arbitrary parameters for KyuubiConf
    78bada0b [Kent Yao] [KYUUBI #2084][FOLLOWUP] Support arbitrary parameters for KyuubiConf
    55e6fdf1 [Kent Yao] [KYUUBI #2084][FOLLOWUP] Support arbitrary parameters for KyuubiConf
    b2239bc7 [Kent Yao] [KYUUBI #2084][FOLLOWUP] Support arbitrary parameters for KyuubiConf
    
    Authored-by: Kent Yao <ya...@apache.org>
    Signed-off-by: Kent Yao <ya...@apache.org>
---
 docs/deployment/settings.md                        |  1 -
 .../kyuubi/engine/flink/FlinkSQLEngine.scala       | 11 ++++----
 .../org/apache/kyuubi/config/KyuubiConf.scala      | 31 +++++-----------------
 .../src/test/resources/kyuubi-defaults.conf        |  6 ++---
 .../org/apache/kyuubi/config/KyuubiConfSuite.scala | 13 +++++----
 5 files changed, 20 insertions(+), 42 deletions(-)

diff --git a/docs/deployment/settings.md b/docs/deployment/settings.md
index 20fc77757..2bcc7fa72 100644
--- a/docs/deployment/settings.md
+++ b/docs/deployment/settings.md
@@ -313,7 +313,6 @@ Key | Default | Meaning | Type | Since
 
 Key | Default | Meaning | Type | Since
 --- | --- | --- | --- | ---
-<code>kyuubi.server.conf.ignore.prefix.list</code>|<div style='width: 65pt;word-wrap: break-word;white-space: normal'>java,sun,os,jdk</div>|<div style='width: 170pt;word-wrap: break-word;white-space: normal'>A comma separated list of ignored keys prefix. If kyuubi conf contains any of them, the key and the corresponding value will be removed silently during server setup. Note that this rule is for server-side protection defined via administrators to prevent some essential configs from ta [...]
 <code>kyuubi.server.name</code>|<div style='width: 65pt;word-wrap: break-word;white-space: normal'>&lt;undefined&gt;</div>|<div style='width: 170pt;word-wrap: break-word;white-space: normal'>The name of Kyuubi Server.</div>|<div style='width: 30pt'>string</div>|<div style='width: 20pt'>1.5.0</div>
 
 
diff --git a/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/FlinkSQLEngine.scala b/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/FlinkSQLEngine.scala
index cce82bd98..06ff5559a 100644
--- a/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/FlinkSQLEngine.scala
+++ b/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/FlinkSQLEngine.scala
@@ -31,9 +31,8 @@ import org.apache.flink.table.client.SqlClientException
 import org.apache.flink.table.client.gateway.context.DefaultContext
 import org.apache.flink.util.JarUtils
 
-import org.apache.kyuubi.{KyuubiSQLException, Logging}
-import org.apache.kyuubi.Utils.{addShutdownHook, FLINK_ENGINE_SHUTDOWN_PRIORITY}
-import org.apache.kyuubi.Utils.currentUser
+import org.apache.kyuubi.{KyuubiSQLException, Logging, Utils}
+import org.apache.kyuubi.Utils.{addShutdownHook, currentUser, FLINK_ENGINE_SHUTDOWN_PRIORITY}
 import org.apache.kyuubi.config.KyuubiConf
 import org.apache.kyuubi.engine.flink.FlinkSQLEngine.{countDownLatch, currentEngine}
 import org.apache.kyuubi.service.Serverable
@@ -76,8 +75,10 @@ object FlinkSQLEngine extends Logging {
     try {
       val flinkConfDir = CliFrontend.getConfigurationDirectoryFromEnv
       val flinkConf = GlobalConfiguration.loadConfiguration(flinkConfDir)
-      val flinkConfFromKyuubi = kyuubiConf.getAllWithPrefix("flink", "")
-      flinkConf.addAll(Configuration.fromMap(flinkConfFromKyuubi.asJava))
+      val flinkConfFromSys =
+        Utils.getSystemProperties.filterKeys(_.startsWith("flink."))
+          .map { case (k, v) => (k.stripPrefix("flink."), v) }
+      flinkConf.addAll(Configuration.fromMap(flinkConfFromSys.asJava))
 
       val executionTarget = flinkConf.getString(DeploymentOptions.TARGET)
       // set cluster name for per-job and application mode
diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala
index f9200e591..bb54b37a6 100644
--- a/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala
+++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala
@@ -25,27 +25,21 @@ import java.util.regex.Pattern
 import scala.collection.JavaConverters._
 
 import org.apache.kyuubi.{Logging, Utils}
+import org.apache.kyuubi.config.KyuubiConf._
 import org.apache.kyuubi.engine.{EngineType, ShareLevel}
 import org.apache.kyuubi.service.authentication.{AuthTypes, SaslQOP}
 
 case class KyuubiConf(loadSysDefault: Boolean = true) extends Logging {
-  import KyuubiConf._
 
   private val settings = new ConcurrentHashMap[String, String]()
   private lazy val reader: ConfigProvider = new ConfigProvider(settings)
-
-  if (loadSysDefault) {
-    loadFromMap()
+  private def loadFromMap(props: Map[String, String]): Unit = {
+    settings.putAll(props.asJava)
   }
 
-  private def loadFromMap(props: Map[String, String] = Utils.getSystemProperties): KyuubiConf = {
-
-    val ignorePrefixList = props.get(SERVER_CONF_IGNORE_PREFIX_LIST.key)
-      .map(_.split(",").map(_.trim).toSeq).getOrElse(get(SERVER_CONF_IGNORE_PREFIX_LIST))
-
-    props.withFilter { case (key, _) => !ignorePrefixList.contains(key.split("\\.").apply(0).trim) }
-      .foreach { case (key, value) => set(key, value) }
-    this
+  if (loadSysDefault) {
+    val fromSysDefaults = Utils.getSystemProperties.filterKeys(_.startsWith("kyuubi."))
+    loadFromMap(fromSysDefaults)
   }
 
   def loadFileDefaults(): KyuubiConf = {
@@ -1316,19 +1310,6 @@ object KyuubiConf {
       .stringConf
       .createOptional
 
-  val SERVER_CONF_IGNORE_PREFIX_LIST: ConfigEntry[Seq[String]] =
-    buildConf("kyuubi.server.conf.ignore.prefix.list")
-      .doc("A comma separated list of ignored keys prefix. If kyuubi conf contains any of" +
-        " them, the key and the corresponding value will be removed silently during server" +
-        " setup." +
-        " Note that this rule is for server-side protection defined via administrators to" +
-        " prevent some essential configs from tampering but will not forbid users to set dynamic" +
-        " configurations via SET syntax.")
-      .version("1.6.0")
-      .stringConf
-      .toSequence()
-      .createWithDefault(Seq("java", "sun", "os", "jdk"))
-
   val ENGINE_SPARK_SHOW_PROGRESS: ConfigEntry[Boolean] =
     buildConf("kyuubi.session.engine.spark.showProgress")
       .doc("When true, show the progress bar in the spark engine log.")
diff --git a/kyuubi-common/src/test/resources/kyuubi-defaults.conf b/kyuubi-common/src/test/resources/kyuubi-defaults.conf
index 0007c56ac..f11b02cb2 100644
--- a/kyuubi-common/src/test/resources/kyuubi-defaults.conf
+++ b/kyuubi-common/src/test/resources/kyuubi-defaults.conf
@@ -24,7 +24,5 @@ ___userb___.spark.user.test b
 
 ___userc___.spark.user.test c
 
-jdk.vendor.version=Zulu 8.60.0.21-CA-linux64
-sun.os.patch.level=unknown
-os.name=Linux
-java.vm.name=OpenJDK 64-Bit Server VM
+abc=xyz
+xyz=abc
diff --git a/kyuubi-common/src/test/scala/org/apache/kyuubi/config/KyuubiConfSuite.scala b/kyuubi-common/src/test/scala/org/apache/kyuubi/config/KyuubiConfSuite.scala
index b9e7e3d38..ee2d562e4 100644
--- a/kyuubi-common/src/test/scala/org/apache/kyuubi/config/KyuubiConfSuite.scala
+++ b/kyuubi-common/src/test/scala/org/apache/kyuubi/config/KyuubiConfSuite.scala
@@ -94,13 +94,12 @@ class KyuubiConfSuite extends KyuubiFunSuite {
     assert(conf.getUserDefaults("userc").getOption("spark.user.test").get === "c")
   }
 
-  test("Ignore configurations with a specific prefix") {
-    val conf = KyuubiConf().loadFileDefaults()
-    assert(conf.getOption("jdk.vendor.version").isEmpty)
-    assert(conf.getOption("sun.os.patch.level").isEmpty)
-    assert(conf.getOption("os.name").isEmpty)
-    assert(conf.getOption("java.vm.name").isEmpty)
-
+  test("support arbitrary config from kyuubi-defaults") {
+    val conf = KyuubiConf()
+    assert(conf.getOption("user.name").isEmpty)
+    conf.loadFileDefaults()
+    assert(conf.getOption("abc").get === "xyz")
+    assert(conf.getOption("xyz").get === "abc")
   }
 
   test("time config test") {