You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@kyuubi.apache.org by GitBox <gi...@apache.org> on 2022/04/12 11:10:36 UTC

[GitHub] [incubator-kyuubi] yaooqinn opened a new pull request, #2336: Simplify TrinoProcessBuilder with java executable

yaooqinn opened a new pull request, #2336:
URL: https://github.com/apache/incubator-kyuubi/pull/2336

   <!--
   Thanks for sending a pull request!
   
   Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://kyuubi.readthedocs.io/en/latest/community/contributions.html
     2. If the PR is related to an issue in https://github.com/apache/incubator-kyuubi/issues, add '[KYUUBI #XXXX]' in your PR title, e.g., '[KYUUBI #XXXX] Your PR title ...'.
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][KYUUBI #XXXX] Your PR title ...'.
   -->
   
   ### _Why are the changes needed?_
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you add a feature, you can talk about the use case of it.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   
   ### _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
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] yaooqinn commented on a diff in pull request #2336: Simplify TrinoProcessBuilder with java executable

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on code in PR #2336:
URL: https://github.com/apache/incubator-kyuubi/pull/2336#discussion_r849028461


##########
build/dist:
##########
@@ -257,9 +253,8 @@ cp "$KYUUBI_HOME/externals/kyuubi-flink-sql-engine/target/kyuubi-flink-sql-engin
 cp "$KYUUBI_HOME/externals/kyuubi-spark-sql-engine/target/kyuubi-spark-sql-engine_${SCALA_VERSION}-${VERSION}.jar" "$DISTDIR/externals/engines/spark"
 
 # Copy trino engines
-cp -r "$KYUUBI_HOME/externals/kyuubi-trino-engine/bin/" "$DISTDIR/externals/engines/trino/bin/"
-chmod a+x "$DISTDIR/externals/engines/trino/bin/trino-engine.sh"
-cp "$KYUUBI_HOME/externals/kyuubi-trino-engine/target/kyuubi-trino-engine_${SCALA_VERSION}-${VERSION}.jar" "$DISTDIR/externals/engines/trino/jars"
+cp "$KYUUBI_HOME/externals/kyuubi-trino-engine/target/kyuubi-trino-engine_${SCALA_VERSION}-${VERSION}.jar" "$DISTDIR/externals/engines/trino"
+cp -r "$KYUUBI_HOME/externals/kyuubi-trino-engine/target/scala-$SCALA_VERSION/jars/" "$DISTDIR/externals/engines/trino"

Review Comment:
   what's more, most of the dependencies here
   - duplicate with kyuubi servers, making our release tar oversize
   - uncurated, we don't carefully manage the transitive jars, the behavior just inherits from the parent's maven-dependency-jar, which is dangerous for making an apache-approved release.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] yaooqinn commented on a diff in pull request #2336: Simplify TrinoProcessBuilder with java executable

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on code in PR #2336:
URL: https://github.com/apache/incubator-kyuubi/pull/2336#discussion_r849035661


##########
kyuubi-server/src/main/scala/org/apache/kyuubi/engine/trino/TrinoProcessBuilder.scala:
##########
@@ -17,101 +17,71 @@
 
 package org.apache.kyuubi.engine.trino
 
-import java.net.URI
-import java.nio.file.Files
+import java.io.File
 import java.nio.file.Paths
+import java.util.LinkedHashSet
 
-import org.apache.kyuubi.{KYUUBI_VERSION, KyuubiSQLException, Logging, SCALA_COMPILE_VERSION, Utils}
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.kyuubi.{Logging, SCALA_COMPILE_VERSION}
 import org.apache.kyuubi.config.KyuubiConf
-import org.apache.kyuubi.config.KyuubiConf.ENGINE_TRINO_CONNECTION_CATALOG
-import org.apache.kyuubi.config.KyuubiConf.ENGINE_TRINO_CONNECTION_URL
-import org.apache.kyuubi.config.KyuubiConf.ENGINE_TRINO_MAIN_RESOURCE
+import org.apache.kyuubi.config.KyuubiConf.{ENGINE_TRINO_CONNECTION_CATALOG, ENGINE_TRINO_CONNECTION_URL}
+import org.apache.kyuubi.config.KyuubiReservedKeys.KYUUBI_SESSION_USER_KEY
 import org.apache.kyuubi.engine.ProcBuilder
-import org.apache.kyuubi.engine.trino.TrinoProcessBuilder._
 import org.apache.kyuubi.operation.log.OperationLog
 
 class TrinoProcessBuilder(
     override val proxyUser: String,
     override val conf: KyuubiConf,
     val extraEngineLog: Option[OperationLog] = None) extends ProcBuilder with Logging {
 
-  private[trino] lazy val trinoConf: Map[String, String] = {
-    assert(
-      conf.get(ENGINE_TRINO_CONNECTION_URL).isDefined,
-      throw KyuubiSQLException(
-        s"Trino server url can not be null! Please set ${ENGINE_TRINO_CONNECTION_URL.key}"))
-    assert(
-      conf.get(ENGINE_TRINO_CONNECTION_CATALOG).isDefined,
-      throw KyuubiSQLException(
-        s"Trino default catalog can not be null!" +
-          s" Please set ${ENGINE_TRINO_CONNECTION_CATALOG.key}"))
+  override protected def module: String = "kyuubi-trino-engine"
 
-    conf.getAll.filter { case (k, v) =>
-      !k.startsWith("spark.") && !k.startsWith("hadoop.")
-    } + (USER -> proxyUser)
-  }
+  override protected def mainClass: String = "org.apache.kyuubi.engine.trino.TrinoSqlEngine"
 
-  override protected val executable: String = {
-    val trinoHomeOpt = env.get("TRINO_ENGINE_HOME").orElse {
-      val cwd = Utils.getCodeSourceLocation(getClass)
-        .split("kyuubi-server")
-      assert(cwd.length > 1)
-      Option(
-        Paths.get(cwd.head)
-          .resolve("externals")
-          .resolve(module)
-          .toFile)
-        .map(_.getAbsolutePath)
-    }
+  override protected def commands: Array[String] = {
+    require(
+      conf.get(ENGINE_TRINO_CONNECTION_URL).nonEmpty,
+      s"Trino server url can not be null! Please set ${ENGINE_TRINO_CONNECTION_URL.key}")
+    require(
+      conf.get(ENGINE_TRINO_CONNECTION_CATALOG).nonEmpty,
+      s"Trino default catalog can not be null! Please set ${ENGINE_TRINO_CONNECTION_CATALOG.key}")
+    val buffer = new ArrayBuffer[String]()
+    buffer += executable
 
-    trinoHomeOpt.map { dir =>
-      Paths.get(dir, "bin", TRINO_ENGINE_BINARY_FILE).toAbsolutePath.toFile.getCanonicalPath
-    }.getOrElse {
-      throw KyuubiSQLException("TRINO_ENGINE_HOME is not set! " +
-        "For more detail information on installing and configuring Trino, please visit " +
-        "https://kyuubi.apache.org/docs/latest/deployment/settings.html#environments")
-    }
-  }
+    // TODO: How shall we deal with proxyUser,
+    // user.name
+    // kyuubi.session.user
+    // or just leave it, because we can handle it at operation layer
+    buffer += s"-D$KYUUBI_SESSION_USER_KEY=$proxyUser"
 
-  override protected def mainResource: Option[String] = {
-    val jarName = s"${module}_$SCALA_COMPILE_VERSION-$KYUUBI_VERSION.jar"
-    // 1. get the main resource jar for user specified config first
-    conf.get(ENGINE_TRINO_MAIN_RESOURCE).filter { userSpecified =>
-      // skip check exist if not local file.
-      val uri = new URI(userSpecified)
-      val schema = if (uri.getScheme != null) uri.getScheme else "file"
-      schema match {
-        case "file" => Files.exists(Paths.get(userSpecified))
-        case _ => true
-      }
-    }.orElse {
-      // 2. get the main resource jar from system build default
-      env.get(KyuubiConf.KYUUBI_HOME)
-        .map { Paths.get(_, "externals", "engines", "trino", "jars", jarName) }
-        .filter(Files.exists(_)).map(_.toAbsolutePath.toFile.getCanonicalPath)
-    }.orElse {
-      // 3. get the main resource from dev environment
-      Option(Paths.get("externals", module, "target", jarName))
-        .filter(Files.exists(_)).orElse {
-          Some(Paths.get("..", "externals", module, "target", jarName))
-        }.map(_.toAbsolutePath.toFile.getCanonicalPath)
+    // TODO: add Kyuubi.engineEnv.TRINO_ENGINE_MEMORY or kyuubi.engine.trino.memory to configure
+    // -Xmx5g
+    // java options
+    for ((k, v) <- conf.getAll) {
+      buffer += s"-D$k=$v"
     }
-  }
-
-  override protected def module: String = "kyuubi-trino-engine"
 
-  override protected def mainClass: String = "org.apache.kyuubi.engine.trino.TrinoSqlEngine"
+    buffer += "-cp"
+    val classpathEntries = new LinkedHashSet[String]
+    // trino engine runtime jar
+    mainResource.foreach(classpathEntries.add)
 
-  override protected def childProcEnv: Map[String, String] = conf.getEnvs +
-    ("TRINO_ENGINE_JAR" -> mainResource.get) +
-    ("TRINO_ENGINE_DYNAMIC_ARGS" ->
-      trinoConf.map { case (k, v) => s"-D$k=$v" }.mkString(" "))
+    mainResource.foreach { path =>
+      classpathEntries.add(s"$path${File.separator}*")
+      val trinoDeps = Paths.get(path).getParent
+        .resolve(s"scala-$SCALA_COMPILE_VERSION")
+        .resolve("jars")

Review Comment:
   I made some changes here to make it clear. before I just added the prod/dev jars blindly. PTAL



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] yaooqinn commented on pull request #2336: Simplify TrinoProcessBuilder with java executable

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on PR #2336:
URL: https://github.com/apache/incubator-kyuubi/pull/2336#issuecomment-1096639839

   cc @hddong 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] yaooqinn commented on a diff in pull request #2336: Simplify TrinoProcessBuilder with java executable

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on code in PR #2336:
URL: https://github.com/apache/incubator-kyuubi/pull/2336#discussion_r849025974


##########
kyuubi-server/src/main/scala/org/apache/kyuubi/engine/trino/TrinoProcessBuilder.scala:
##########
@@ -17,101 +17,71 @@
 
 package org.apache.kyuubi.engine.trino
 
-import java.net.URI
-import java.nio.file.Files
+import java.io.File
 import java.nio.file.Paths
+import java.util.LinkedHashSet
 
-import org.apache.kyuubi.{KYUUBI_VERSION, KyuubiSQLException, Logging, SCALA_COMPILE_VERSION, Utils}
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.kyuubi.{Logging, SCALA_COMPILE_VERSION}
 import org.apache.kyuubi.config.KyuubiConf
-import org.apache.kyuubi.config.KyuubiConf.ENGINE_TRINO_CONNECTION_CATALOG
-import org.apache.kyuubi.config.KyuubiConf.ENGINE_TRINO_CONNECTION_URL
-import org.apache.kyuubi.config.KyuubiConf.ENGINE_TRINO_MAIN_RESOURCE
+import org.apache.kyuubi.config.KyuubiConf.{ENGINE_TRINO_CONNECTION_CATALOG, ENGINE_TRINO_CONNECTION_URL}
+import org.apache.kyuubi.config.KyuubiReservedKeys.KYUUBI_SESSION_USER_KEY
 import org.apache.kyuubi.engine.ProcBuilder
-import org.apache.kyuubi.engine.trino.TrinoProcessBuilder._
 import org.apache.kyuubi.operation.log.OperationLog
 
 class TrinoProcessBuilder(
     override val proxyUser: String,
     override val conf: KyuubiConf,
     val extraEngineLog: Option[OperationLog] = None) extends ProcBuilder with Logging {
 
-  private[trino] lazy val trinoConf: Map[String, String] = {
-    assert(
-      conf.get(ENGINE_TRINO_CONNECTION_URL).isDefined,
-      throw KyuubiSQLException(
-        s"Trino server url can not be null! Please set ${ENGINE_TRINO_CONNECTION_URL.key}"))
-    assert(
-      conf.get(ENGINE_TRINO_CONNECTION_CATALOG).isDefined,
-      throw KyuubiSQLException(
-        s"Trino default catalog can not be null!" +
-          s" Please set ${ENGINE_TRINO_CONNECTION_CATALOG.key}"))
+  override protected def module: String = "kyuubi-trino-engine"
 
-    conf.getAll.filter { case (k, v) =>
-      !k.startsWith("spark.") && !k.startsWith("hadoop.")
-    } + (USER -> proxyUser)
-  }
+  override protected def mainClass: String = "org.apache.kyuubi.engine.trino.TrinoSqlEngine"
 
-  override protected val executable: String = {
-    val trinoHomeOpt = env.get("TRINO_ENGINE_HOME").orElse {
-      val cwd = Utils.getCodeSourceLocation(getClass)
-        .split("kyuubi-server")
-      assert(cwd.length > 1)
-      Option(
-        Paths.get(cwd.head)
-          .resolve("externals")
-          .resolve(module)
-          .toFile)
-        .map(_.getAbsolutePath)
-    }
+  override protected def commands: Array[String] = {
+    require(
+      conf.get(ENGINE_TRINO_CONNECTION_URL).nonEmpty,
+      s"Trino server url can not be null! Please set ${ENGINE_TRINO_CONNECTION_URL.key}")
+    require(
+      conf.get(ENGINE_TRINO_CONNECTION_CATALOG).nonEmpty,
+      s"Trino default catalog can not be null! Please set ${ENGINE_TRINO_CONNECTION_CATALOG.key}")
+    val buffer = new ArrayBuffer[String]()
+    buffer += executable
 
-    trinoHomeOpt.map { dir =>
-      Paths.get(dir, "bin", TRINO_ENGINE_BINARY_FILE).toAbsolutePath.toFile.getCanonicalPath
-    }.getOrElse {
-      throw KyuubiSQLException("TRINO_ENGINE_HOME is not set! " +
-        "For more detail information on installing and configuring Trino, please visit " +
-        "https://kyuubi.apache.org/docs/latest/deployment/settings.html#environments")
-    }
-  }
+    // TODO: How shall we deal with proxyUser,
+    // user.name
+    // kyuubi.session.user
+    // or just leave it, because we can handle it at operation layer
+    buffer += s"-D$KYUUBI_SESSION_USER_KEY=$proxyUser"
 
-  override protected def mainResource: Option[String] = {
-    val jarName = s"${module}_$SCALA_COMPILE_VERSION-$KYUUBI_VERSION.jar"
-    // 1. get the main resource jar for user specified config first
-    conf.get(ENGINE_TRINO_MAIN_RESOURCE).filter { userSpecified =>
-      // skip check exist if not local file.
-      val uri = new URI(userSpecified)
-      val schema = if (uri.getScheme != null) uri.getScheme else "file"
-      schema match {
-        case "file" => Files.exists(Paths.get(userSpecified))
-        case _ => true
-      }
-    }.orElse {
-      // 2. get the main resource jar from system build default
-      env.get(KyuubiConf.KYUUBI_HOME)
-        .map { Paths.get(_, "externals", "engines", "trino", "jars", jarName) }
-        .filter(Files.exists(_)).map(_.toAbsolutePath.toFile.getCanonicalPath)
-    }.orElse {
-      // 3. get the main resource from dev environment
-      Option(Paths.get("externals", module, "target", jarName))
-        .filter(Files.exists(_)).orElse {
-          Some(Paths.get("..", "externals", module, "target", jarName))
-        }.map(_.toAbsolutePath.toFile.getCanonicalPath)
+    // TODO: add Kyuubi.engineEnv.TRINO_ENGINE_MEMORY or kyuubi.engine.trino.memory to configure
+    // -Xmx5g
+    // java options
+    for ((k, v) <- conf.getAll) {
+      buffer += s"-D$k=$v"
     }
-  }
-
-  override protected def module: String = "kyuubi-trino-engine"
 
-  override protected def mainClass: String = "org.apache.kyuubi.engine.trino.TrinoSqlEngine"
+    buffer += "-cp"
+    val classpathEntries = new LinkedHashSet[String]
+    // trino engine runtime jar
+    mainResource.foreach(classpathEntries.add)
 
-  override protected def childProcEnv: Map[String, String] = conf.getEnvs +
-    ("TRINO_ENGINE_JAR" -> mainResource.get) +
-    ("TRINO_ENGINE_DYNAMIC_ARGS" ->
-      trinoConf.map { case (k, v) => s"-D$k=$v" }.mkString(" "))
+    mainResource.foreach { path =>
+      classpathEntries.add(s"$path${File.separator}*")

Review Comment:
   nice catch!
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] hddong commented on a diff in pull request #2336: Simplify TrinoProcessBuilder with java executable

Posted by GitBox <gi...@apache.org>.
hddong commented on code in PR #2336:
URL: https://github.com/apache/incubator-kyuubi/pull/2336#discussion_r849034576


##########
build/dist:
##########
@@ -257,9 +253,8 @@ cp "$KYUUBI_HOME/externals/kyuubi-flink-sql-engine/target/kyuubi-flink-sql-engin
 cp "$KYUUBI_HOME/externals/kyuubi-spark-sql-engine/target/kyuubi-spark-sql-engine_${SCALA_VERSION}-${VERSION}.jar" "$DISTDIR/externals/engines/spark"
 
 # Copy trino engines
-cp -r "$KYUUBI_HOME/externals/kyuubi-trino-engine/bin/" "$DISTDIR/externals/engines/trino/bin/"
-chmod a+x "$DISTDIR/externals/engines/trino/bin/trino-engine.sh"
-cp "$KYUUBI_HOME/externals/kyuubi-trino-engine/target/kyuubi-trino-engine_${SCALA_VERSION}-${VERSION}.jar" "$DISTDIR/externals/engines/trino/jars"
+cp "$KYUUBI_HOME/externals/kyuubi-trino-engine/target/kyuubi-trino-engine_${SCALA_VERSION}-${VERSION}.jar" "$DISTDIR/externals/engines/trino"
+cp -r "$KYUUBI_HOME/externals/kyuubi-trino-engine/target/scala-$SCALA_VERSION/jars/" "$DISTDIR/externals/engines/trino"

Review Comment:
   Get it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] yaooqinn commented on a diff in pull request #2336: Simplify TrinoProcessBuilder with java executable

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on code in PR #2336:
URL: https://github.com/apache/incubator-kyuubi/pull/2336#discussion_r849026345


##########
kyuubi-server/src/main/scala/org/apache/kyuubi/engine/trino/TrinoProcessBuilder.scala:
##########
@@ -17,101 +17,71 @@
 
 package org.apache.kyuubi.engine.trino
 
-import java.net.URI
-import java.nio.file.Files
+import java.io.File
 import java.nio.file.Paths
+import java.util.LinkedHashSet
 
-import org.apache.kyuubi.{KYUUBI_VERSION, KyuubiSQLException, Logging, SCALA_COMPILE_VERSION, Utils}
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.kyuubi.{Logging, SCALA_COMPILE_VERSION}
 import org.apache.kyuubi.config.KyuubiConf
-import org.apache.kyuubi.config.KyuubiConf.ENGINE_TRINO_CONNECTION_CATALOG
-import org.apache.kyuubi.config.KyuubiConf.ENGINE_TRINO_CONNECTION_URL
-import org.apache.kyuubi.config.KyuubiConf.ENGINE_TRINO_MAIN_RESOURCE
+import org.apache.kyuubi.config.KyuubiConf.{ENGINE_TRINO_CONNECTION_CATALOG, ENGINE_TRINO_CONNECTION_URL}
+import org.apache.kyuubi.config.KyuubiReservedKeys.KYUUBI_SESSION_USER_KEY
 import org.apache.kyuubi.engine.ProcBuilder
-import org.apache.kyuubi.engine.trino.TrinoProcessBuilder._
 import org.apache.kyuubi.operation.log.OperationLog
 
 class TrinoProcessBuilder(
     override val proxyUser: String,
     override val conf: KyuubiConf,
     val extraEngineLog: Option[OperationLog] = None) extends ProcBuilder with Logging {
 
-  private[trino] lazy val trinoConf: Map[String, String] = {
-    assert(
-      conf.get(ENGINE_TRINO_CONNECTION_URL).isDefined,
-      throw KyuubiSQLException(
-        s"Trino server url can not be null! Please set ${ENGINE_TRINO_CONNECTION_URL.key}"))
-    assert(
-      conf.get(ENGINE_TRINO_CONNECTION_CATALOG).isDefined,
-      throw KyuubiSQLException(
-        s"Trino default catalog can not be null!" +
-          s" Please set ${ENGINE_TRINO_CONNECTION_CATALOG.key}"))
+  override protected def module: String = "kyuubi-trino-engine"
 
-    conf.getAll.filter { case (k, v) =>
-      !k.startsWith("spark.") && !k.startsWith("hadoop.")
-    } + (USER -> proxyUser)
-  }
+  override protected def mainClass: String = "org.apache.kyuubi.engine.trino.TrinoSqlEngine"
 
-  override protected val executable: String = {
-    val trinoHomeOpt = env.get("TRINO_ENGINE_HOME").orElse {
-      val cwd = Utils.getCodeSourceLocation(getClass)
-        .split("kyuubi-server")
-      assert(cwd.length > 1)
-      Option(
-        Paths.get(cwd.head)
-          .resolve("externals")
-          .resolve(module)
-          .toFile)
-        .map(_.getAbsolutePath)
-    }
+  override protected def commands: Array[String] = {
+    require(
+      conf.get(ENGINE_TRINO_CONNECTION_URL).nonEmpty,
+      s"Trino server url can not be null! Please set ${ENGINE_TRINO_CONNECTION_URL.key}")
+    require(
+      conf.get(ENGINE_TRINO_CONNECTION_CATALOG).nonEmpty,
+      s"Trino default catalog can not be null! Please set ${ENGINE_TRINO_CONNECTION_CATALOG.key}")
+    val buffer = new ArrayBuffer[String]()
+    buffer += executable
 
-    trinoHomeOpt.map { dir =>
-      Paths.get(dir, "bin", TRINO_ENGINE_BINARY_FILE).toAbsolutePath.toFile.getCanonicalPath
-    }.getOrElse {
-      throw KyuubiSQLException("TRINO_ENGINE_HOME is not set! " +
-        "For more detail information on installing and configuring Trino, please visit " +
-        "https://kyuubi.apache.org/docs/latest/deployment/settings.html#environments")
-    }
-  }
+    // TODO: How shall we deal with proxyUser,
+    // user.name
+    // kyuubi.session.user
+    // or just leave it, because we can handle it at operation layer
+    buffer += s"-D$KYUUBI_SESSION_USER_KEY=$proxyUser"
 
-  override protected def mainResource: Option[String] = {
-    val jarName = s"${module}_$SCALA_COMPILE_VERSION-$KYUUBI_VERSION.jar"
-    // 1. get the main resource jar for user specified config first
-    conf.get(ENGINE_TRINO_MAIN_RESOURCE).filter { userSpecified =>
-      // skip check exist if not local file.
-      val uri = new URI(userSpecified)
-      val schema = if (uri.getScheme != null) uri.getScheme else "file"
-      schema match {
-        case "file" => Files.exists(Paths.get(userSpecified))
-        case _ => true
-      }
-    }.orElse {
-      // 2. get the main resource jar from system build default
-      env.get(KyuubiConf.KYUUBI_HOME)
-        .map { Paths.get(_, "externals", "engines", "trino", "jars", jarName) }
-        .filter(Files.exists(_)).map(_.toAbsolutePath.toFile.getCanonicalPath)
-    }.orElse {
-      // 3. get the main resource from dev environment
-      Option(Paths.get("externals", module, "target", jarName))
-        .filter(Files.exists(_)).orElse {
-          Some(Paths.get("..", "externals", module, "target", jarName))
-        }.map(_.toAbsolutePath.toFile.getCanonicalPath)
+    // TODO: add Kyuubi.engineEnv.TRINO_ENGINE_MEMORY or kyuubi.engine.trino.memory to configure
+    // -Xmx5g
+    // java options
+    for ((k, v) <- conf.getAll) {
+      buffer += s"-D$k=$v"
     }
-  }
-
-  override protected def module: String = "kyuubi-trino-engine"
 
-  override protected def mainClass: String = "org.apache.kyuubi.engine.trino.TrinoSqlEngine"
+    buffer += "-cp"
+    val classpathEntries = new LinkedHashSet[String]
+    // trino engine runtime jar
+    mainResource.foreach(classpathEntries.add)
 
-  override protected def childProcEnv: Map[String, String] = conf.getEnvs +
-    ("TRINO_ENGINE_JAR" -> mainResource.get) +
-    ("TRINO_ENGINE_DYNAMIC_ARGS" ->
-      trinoConf.map { case (k, v) => s"-D$k=$v" }.mkString(" "))
+    mainResource.foreach { path =>
+      classpathEntries.add(s"$path${File.separator}*")
+      val trinoDeps = Paths.get(path).getParent
+        .resolve(s"scala-$SCALA_COMPILE_VERSION")
+        .resolve("jars")

Review Comment:
   this seems not related to this pr



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] yaooqinn closed pull request #2336: Simplify TrinoProcessBuilder with java executable

Posted by GitBox <gi...@apache.org>.
yaooqinn closed pull request #2336: Simplify TrinoProcessBuilder with java executable
URL: https://github.com/apache/incubator-kyuubi/pull/2336


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] codecov-commenter commented on pull request #2336: Simplify TrinoProcessBuilder with java executable

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on PR #2336:
URL: https://github.com/apache/incubator-kyuubi/pull/2336#issuecomment-1096753156

   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2336?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2336](https://codecov.io/gh/apache/incubator-kyuubi/pull/2336?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (7048f97) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/a56e4b4f0189ffcebaafc7be328e88255d449a33?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (a56e4b4) will **increase** coverage by `62.58%`.
   > The diff coverage is `92.59%`.
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2336       +/-   ##
   =============================================
   + Coverage          0   62.58%   +62.58%     
   - Complexity        0       69       +69     
   =============================================
     Files             0      353      +353     
     Lines             0    16814    +16814     
     Branches          0     2275     +2275     
   =============================================
   + Hits              0    10523    +10523     
   - Misses            0     5314     +5314     
   - Partials          0      977      +977     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2336?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...n/scala/org/apache/kyuubi/engine/ProcBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9lbmdpbmUvUHJvY0J1aWxkZXIuc2NhbGE=) | `82.80% <84.21%> (ø)` | |
   | [...ache/kyuubi/engine/trino/TrinoProcessBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9lbmdpbmUvdHJpbm8vVHJpbm9Qcm9jZXNzQnVpbGRlci5zY2FsYQ==) | `93.54% <96.29%> (ø)` | |
   | [...org/apache/kyuubi/engine/trino/schema/RowSet.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS10cmluby1lbmdpbmUvc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvZW5naW5lL3RyaW5vL3NjaGVtYS9Sb3dTZXQuc2NhbGE=) | `92.59% <100.00%> (ø)` | |
   | [...kyuubi/engine/trino/session/TrinoSessionImpl.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS10cmluby1lbmdpbmUvc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvZW5naW5lL3RyaW5vL3Nlc3Npb24vVHJpbm9TZXNzaW9uSW1wbC5zY2FsYQ==) | `97.61% <100.00%> (ø)` | |
   | [...ache/kyuubi/engine/flink/FlinkProcessBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9lbmdpbmUvZmxpbmsvRmxpbmtQcm9jZXNzQnVpbGRlci5zY2FsYQ==) | `76.47% <100.00%> (ø)` | |
   | [...apache/kyuubi/engine/hive/HiveProcessBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9lbmdpbmUvaGl2ZS9IaXZlUHJvY2Vzc0J1aWxkZXIuc2NhbGE=) | `90.90% <100.00%> (ø)` | |
   | [...ache/kyuubi/engine/spark/SparkProcessBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9lbmdpbmUvc3BhcmsvU3BhcmtQcm9jZXNzQnVpbGRlci5zY2FsYQ==) | `86.11% <100.00%> (ø)` | |
   | [...ache/kyuubi/sql/RepartitionBeforeWritingBase.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZW5zaW9ucy9zcGFyay9reXV1YmktZXh0ZW5zaW9uLXNwYXJrLWNvbW1vbi9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9zcWwvUmVwYXJ0aXRpb25CZWZvcmVXcml0aW5nQmFzZS5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | [...rg/apache/kyuubi/zookeeper/EmbeddedZookeeper.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXpvb2tlZXBlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS96b29rZWVwZXIvRW1iZWRkZWRab29rZWVwZXIuc2NhbGE=) | `86.66% <0.00%> (ø)` | |
   | [...server/mysql/authentication/MySQLAuthPackets.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2336/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9zZXJ2ZXIvbXlzcWwvYXV0aGVudGljYXRpb24vTXlTUUxBdXRoUGFja2V0cy5zY2FsYQ==) | `82.27% <0.00%> (ø)` | |
   | ... and [350 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2336/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2336?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2336?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [a56e4b4...7048f97](https://codecov.io/gh/apache/incubator-kyuubi/pull/2336?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] yaooqinn commented on pull request #2336: Simplify TrinoProcessBuilder with java executable

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on PR #2336:
URL: https://github.com/apache/incubator-kyuubi/pull/2336#issuecomment-1097545626

   thanks, merged to master


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] yaooqinn commented on a diff in pull request #2336: Simplify TrinoProcessBuilder with java executable

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on code in PR #2336:
URL: https://github.com/apache/incubator-kyuubi/pull/2336#discussion_r849029308


##########
kyuubi-server/src/main/scala/org/apache/kyuubi/engine/trino/TrinoProcessBuilder.scala:
##########
@@ -17,101 +17,71 @@
 
 package org.apache.kyuubi.engine.trino
 
-import java.net.URI
-import java.nio.file.Files
+import java.io.File
 import java.nio.file.Paths
+import java.util.LinkedHashSet
 
-import org.apache.kyuubi.{KYUUBI_VERSION, KyuubiSQLException, Logging, SCALA_COMPILE_VERSION, Utils}
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.kyuubi.{Logging, SCALA_COMPILE_VERSION}
 import org.apache.kyuubi.config.KyuubiConf
-import org.apache.kyuubi.config.KyuubiConf.ENGINE_TRINO_CONNECTION_CATALOG
-import org.apache.kyuubi.config.KyuubiConf.ENGINE_TRINO_CONNECTION_URL
-import org.apache.kyuubi.config.KyuubiConf.ENGINE_TRINO_MAIN_RESOURCE
+import org.apache.kyuubi.config.KyuubiConf.{ENGINE_TRINO_CONNECTION_CATALOG, ENGINE_TRINO_CONNECTION_URL}
+import org.apache.kyuubi.config.KyuubiReservedKeys.KYUUBI_SESSION_USER_KEY
 import org.apache.kyuubi.engine.ProcBuilder
-import org.apache.kyuubi.engine.trino.TrinoProcessBuilder._
 import org.apache.kyuubi.operation.log.OperationLog
 
 class TrinoProcessBuilder(
     override val proxyUser: String,
     override val conf: KyuubiConf,
     val extraEngineLog: Option[OperationLog] = None) extends ProcBuilder with Logging {
 
-  private[trino] lazy val trinoConf: Map[String, String] = {
-    assert(
-      conf.get(ENGINE_TRINO_CONNECTION_URL).isDefined,
-      throw KyuubiSQLException(
-        s"Trino server url can not be null! Please set ${ENGINE_TRINO_CONNECTION_URL.key}"))
-    assert(
-      conf.get(ENGINE_TRINO_CONNECTION_CATALOG).isDefined,
-      throw KyuubiSQLException(
-        s"Trino default catalog can not be null!" +
-          s" Please set ${ENGINE_TRINO_CONNECTION_CATALOG.key}"))
+  override protected def module: String = "kyuubi-trino-engine"
 
-    conf.getAll.filter { case (k, v) =>
-      !k.startsWith("spark.") && !k.startsWith("hadoop.")
-    } + (USER -> proxyUser)
-  }
+  override protected def mainClass: String = "org.apache.kyuubi.engine.trino.TrinoSqlEngine"
 
-  override protected val executable: String = {
-    val trinoHomeOpt = env.get("TRINO_ENGINE_HOME").orElse {
-      val cwd = Utils.getCodeSourceLocation(getClass)
-        .split("kyuubi-server")
-      assert(cwd.length > 1)
-      Option(
-        Paths.get(cwd.head)
-          .resolve("externals")
-          .resolve(module)
-          .toFile)
-        .map(_.getAbsolutePath)
-    }
+  override protected def commands: Array[String] = {
+    require(
+      conf.get(ENGINE_TRINO_CONNECTION_URL).nonEmpty,
+      s"Trino server url can not be null! Please set ${ENGINE_TRINO_CONNECTION_URL.key}")
+    require(
+      conf.get(ENGINE_TRINO_CONNECTION_CATALOG).nonEmpty,
+      s"Trino default catalog can not be null! Please set ${ENGINE_TRINO_CONNECTION_CATALOG.key}")
+    val buffer = new ArrayBuffer[String]()
+    buffer += executable
 
-    trinoHomeOpt.map { dir =>
-      Paths.get(dir, "bin", TRINO_ENGINE_BINARY_FILE).toAbsolutePath.toFile.getCanonicalPath
-    }.getOrElse {
-      throw KyuubiSQLException("TRINO_ENGINE_HOME is not set! " +
-        "For more detail information on installing and configuring Trino, please visit " +
-        "https://kyuubi.apache.org/docs/latest/deployment/settings.html#environments")
-    }
-  }
+    // TODO: How shall we deal with proxyUser,
+    // user.name
+    // kyuubi.session.user
+    // or just leave it, because we can handle it at operation layer
+    buffer += s"-D$KYUUBI_SESSION_USER_KEY=$proxyUser"
 
-  override protected def mainResource: Option[String] = {
-    val jarName = s"${module}_$SCALA_COMPILE_VERSION-$KYUUBI_VERSION.jar"
-    // 1. get the main resource jar for user specified config first
-    conf.get(ENGINE_TRINO_MAIN_RESOURCE).filter { userSpecified =>
-      // skip check exist if not local file.
-      val uri = new URI(userSpecified)
-      val schema = if (uri.getScheme != null) uri.getScheme else "file"
-      schema match {
-        case "file" => Files.exists(Paths.get(userSpecified))
-        case _ => true
-      }
-    }.orElse {
-      // 2. get the main resource jar from system build default
-      env.get(KyuubiConf.KYUUBI_HOME)
-        .map { Paths.get(_, "externals", "engines", "trino", "jars", jarName) }
-        .filter(Files.exists(_)).map(_.toAbsolutePath.toFile.getCanonicalPath)
-    }.orElse {
-      // 3. get the main resource from dev environment
-      Option(Paths.get("externals", module, "target", jarName))
-        .filter(Files.exists(_)).orElse {
-          Some(Paths.get("..", "externals", module, "target", jarName))
-        }.map(_.toAbsolutePath.toFile.getCanonicalPath)
+    // TODO: add Kyuubi.engineEnv.TRINO_ENGINE_MEMORY or kyuubi.engine.trino.memory to configure
+    // -Xmx5g
+    // java options
+    for ((k, v) <- conf.getAll) {
+      buffer += s"-D$k=$v"
     }
-  }
-
-  override protected def module: String = "kyuubi-trino-engine"
 
-  override protected def mainClass: String = "org.apache.kyuubi.engine.trino.TrinoSqlEngine"
+    buffer += "-cp"
+    val classpathEntries = new LinkedHashSet[String]
+    // trino engine runtime jar
+    mainResource.foreach(classpathEntries.add)
 
-  override protected def childProcEnv: Map[String, String] = conf.getEnvs +
-    ("TRINO_ENGINE_JAR" -> mainResource.get) +
-    ("TRINO_ENGINE_DYNAMIC_ARGS" ->
-      trinoConf.map { case (k, v) => s"-D$k=$v" }.mkString(" "))
+    mainResource.foreach { path =>
+      classpathEntries.add(s"$path${File.separator}*")
+      val trinoDeps = Paths.get(path).getParent
+        .resolve(s"scala-$SCALA_COMPILE_VERSION")
+        .resolve("jars")

Review Comment:
   sorry, this is just for test



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] hddong commented on a diff in pull request #2336: Simplify TrinoProcessBuilder with java executable

Posted by GitBox <gi...@apache.org>.
hddong commented on code in PR #2336:
URL: https://github.com/apache/incubator-kyuubi/pull/2336#discussion_r849022998


##########
kyuubi-server/src/main/scala/org/apache/kyuubi/engine/trino/TrinoProcessBuilder.scala:
##########
@@ -17,101 +17,71 @@
 
 package org.apache.kyuubi.engine.trino
 
-import java.net.URI
-import java.nio.file.Files
+import java.io.File
 import java.nio.file.Paths
+import java.util.LinkedHashSet
 
-import org.apache.kyuubi.{KYUUBI_VERSION, KyuubiSQLException, Logging, SCALA_COMPILE_VERSION, Utils}
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.kyuubi.{Logging, SCALA_COMPILE_VERSION}
 import org.apache.kyuubi.config.KyuubiConf
-import org.apache.kyuubi.config.KyuubiConf.ENGINE_TRINO_CONNECTION_CATALOG
-import org.apache.kyuubi.config.KyuubiConf.ENGINE_TRINO_CONNECTION_URL
-import org.apache.kyuubi.config.KyuubiConf.ENGINE_TRINO_MAIN_RESOURCE
+import org.apache.kyuubi.config.KyuubiConf.{ENGINE_TRINO_CONNECTION_CATALOG, ENGINE_TRINO_CONNECTION_URL}
+import org.apache.kyuubi.config.KyuubiReservedKeys.KYUUBI_SESSION_USER_KEY
 import org.apache.kyuubi.engine.ProcBuilder
-import org.apache.kyuubi.engine.trino.TrinoProcessBuilder._
 import org.apache.kyuubi.operation.log.OperationLog
 
 class TrinoProcessBuilder(
     override val proxyUser: String,
     override val conf: KyuubiConf,
     val extraEngineLog: Option[OperationLog] = None) extends ProcBuilder with Logging {
 
-  private[trino] lazy val trinoConf: Map[String, String] = {
-    assert(
-      conf.get(ENGINE_TRINO_CONNECTION_URL).isDefined,
-      throw KyuubiSQLException(
-        s"Trino server url can not be null! Please set ${ENGINE_TRINO_CONNECTION_URL.key}"))
-    assert(
-      conf.get(ENGINE_TRINO_CONNECTION_CATALOG).isDefined,
-      throw KyuubiSQLException(
-        s"Trino default catalog can not be null!" +
-          s" Please set ${ENGINE_TRINO_CONNECTION_CATALOG.key}"))
+  override protected def module: String = "kyuubi-trino-engine"
 
-    conf.getAll.filter { case (k, v) =>
-      !k.startsWith("spark.") && !k.startsWith("hadoop.")
-    } + (USER -> proxyUser)
-  }
+  override protected def mainClass: String = "org.apache.kyuubi.engine.trino.TrinoSqlEngine"
 
-  override protected val executable: String = {
-    val trinoHomeOpt = env.get("TRINO_ENGINE_HOME").orElse {
-      val cwd = Utils.getCodeSourceLocation(getClass)
-        .split("kyuubi-server")
-      assert(cwd.length > 1)
-      Option(
-        Paths.get(cwd.head)
-          .resolve("externals")
-          .resolve(module)
-          .toFile)
-        .map(_.getAbsolutePath)
-    }
+  override protected def commands: Array[String] = {
+    require(
+      conf.get(ENGINE_TRINO_CONNECTION_URL).nonEmpty,
+      s"Trino server url can not be null! Please set ${ENGINE_TRINO_CONNECTION_URL.key}")
+    require(
+      conf.get(ENGINE_TRINO_CONNECTION_CATALOG).nonEmpty,
+      s"Trino default catalog can not be null! Please set ${ENGINE_TRINO_CONNECTION_CATALOG.key}")
+    val buffer = new ArrayBuffer[String]()
+    buffer += executable
 
-    trinoHomeOpt.map { dir =>
-      Paths.get(dir, "bin", TRINO_ENGINE_BINARY_FILE).toAbsolutePath.toFile.getCanonicalPath
-    }.getOrElse {
-      throw KyuubiSQLException("TRINO_ENGINE_HOME is not set! " +
-        "For more detail information on installing and configuring Trino, please visit " +
-        "https://kyuubi.apache.org/docs/latest/deployment/settings.html#environments")
-    }
-  }
+    // TODO: How shall we deal with proxyUser,
+    // user.name
+    // kyuubi.session.user
+    // or just leave it, because we can handle it at operation layer
+    buffer += s"-D$KYUUBI_SESSION_USER_KEY=$proxyUser"
 
-  override protected def mainResource: Option[String] = {
-    val jarName = s"${module}_$SCALA_COMPILE_VERSION-$KYUUBI_VERSION.jar"
-    // 1. get the main resource jar for user specified config first
-    conf.get(ENGINE_TRINO_MAIN_RESOURCE).filter { userSpecified =>
-      // skip check exist if not local file.
-      val uri = new URI(userSpecified)
-      val schema = if (uri.getScheme != null) uri.getScheme else "file"
-      schema match {
-        case "file" => Files.exists(Paths.get(userSpecified))
-        case _ => true
-      }
-    }.orElse {
-      // 2. get the main resource jar from system build default
-      env.get(KyuubiConf.KYUUBI_HOME)
-        .map { Paths.get(_, "externals", "engines", "trino", "jars", jarName) }
-        .filter(Files.exists(_)).map(_.toAbsolutePath.toFile.getCanonicalPath)
-    }.orElse {
-      // 3. get the main resource from dev environment
-      Option(Paths.get("externals", module, "target", jarName))
-        .filter(Files.exists(_)).orElse {
-          Some(Paths.get("..", "externals", module, "target", jarName))
-        }.map(_.toAbsolutePath.toFile.getCanonicalPath)
+    // TODO: add Kyuubi.engineEnv.TRINO_ENGINE_MEMORY or kyuubi.engine.trino.memory to configure
+    // -Xmx5g
+    // java options
+    for ((k, v) <- conf.getAll) {
+      buffer += s"-D$k=$v"
     }
-  }
-
-  override protected def module: String = "kyuubi-trino-engine"
 
-  override protected def mainClass: String = "org.apache.kyuubi.engine.trino.TrinoSqlEngine"
+    buffer += "-cp"
+    val classpathEntries = new LinkedHashSet[String]
+    // trino engine runtime jar
+    mainResource.foreach(classpathEntries.add)
 
-  override protected def childProcEnv: Map[String, String] = conf.getEnvs +
-    ("TRINO_ENGINE_JAR" -> mainResource.get) +
-    ("TRINO_ENGINE_DYNAMIC_ARGS" ->
-      trinoConf.map { case (k, v) => s"-D$k=$v" }.mkString(" "))
+    mainResource.foreach { path =>
+      classpathEntries.add(s"$path${File.separator}*")

Review Comment:
   This classpath seems like `${KYUUBI_HOME}/externals/engines/trino/kyuubi-trino-engine_2.12-1.6.0-SNAPSHOT.jar/*`, missing `getParent`?



##########
kyuubi-server/src/main/scala/org/apache/kyuubi/engine/trino/TrinoProcessBuilder.scala:
##########
@@ -17,101 +17,71 @@
 
 package org.apache.kyuubi.engine.trino
 
-import java.net.URI
-import java.nio.file.Files
+import java.io.File
 import java.nio.file.Paths
+import java.util.LinkedHashSet
 
-import org.apache.kyuubi.{KYUUBI_VERSION, KyuubiSQLException, Logging, SCALA_COMPILE_VERSION, Utils}
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.kyuubi.{Logging, SCALA_COMPILE_VERSION}
 import org.apache.kyuubi.config.KyuubiConf
-import org.apache.kyuubi.config.KyuubiConf.ENGINE_TRINO_CONNECTION_CATALOG
-import org.apache.kyuubi.config.KyuubiConf.ENGINE_TRINO_CONNECTION_URL
-import org.apache.kyuubi.config.KyuubiConf.ENGINE_TRINO_MAIN_RESOURCE
+import org.apache.kyuubi.config.KyuubiConf.{ENGINE_TRINO_CONNECTION_CATALOG, ENGINE_TRINO_CONNECTION_URL}
+import org.apache.kyuubi.config.KyuubiReservedKeys.KYUUBI_SESSION_USER_KEY
 import org.apache.kyuubi.engine.ProcBuilder
-import org.apache.kyuubi.engine.trino.TrinoProcessBuilder._
 import org.apache.kyuubi.operation.log.OperationLog
 
 class TrinoProcessBuilder(
     override val proxyUser: String,
     override val conf: KyuubiConf,
     val extraEngineLog: Option[OperationLog] = None) extends ProcBuilder with Logging {
 
-  private[trino] lazy val trinoConf: Map[String, String] = {
-    assert(
-      conf.get(ENGINE_TRINO_CONNECTION_URL).isDefined,
-      throw KyuubiSQLException(
-        s"Trino server url can not be null! Please set ${ENGINE_TRINO_CONNECTION_URL.key}"))
-    assert(
-      conf.get(ENGINE_TRINO_CONNECTION_CATALOG).isDefined,
-      throw KyuubiSQLException(
-        s"Trino default catalog can not be null!" +
-          s" Please set ${ENGINE_TRINO_CONNECTION_CATALOG.key}"))
+  override protected def module: String = "kyuubi-trino-engine"
 
-    conf.getAll.filter { case (k, v) =>
-      !k.startsWith("spark.") && !k.startsWith("hadoop.")
-    } + (USER -> proxyUser)
-  }
+  override protected def mainClass: String = "org.apache.kyuubi.engine.trino.TrinoSqlEngine"
 
-  override protected val executable: String = {
-    val trinoHomeOpt = env.get("TRINO_ENGINE_HOME").orElse {
-      val cwd = Utils.getCodeSourceLocation(getClass)
-        .split("kyuubi-server")
-      assert(cwd.length > 1)
-      Option(
-        Paths.get(cwd.head)
-          .resolve("externals")
-          .resolve(module)
-          .toFile)
-        .map(_.getAbsolutePath)
-    }
+  override protected def commands: Array[String] = {
+    require(
+      conf.get(ENGINE_TRINO_CONNECTION_URL).nonEmpty,
+      s"Trino server url can not be null! Please set ${ENGINE_TRINO_CONNECTION_URL.key}")
+    require(
+      conf.get(ENGINE_TRINO_CONNECTION_CATALOG).nonEmpty,
+      s"Trino default catalog can not be null! Please set ${ENGINE_TRINO_CONNECTION_CATALOG.key}")
+    val buffer = new ArrayBuffer[String]()
+    buffer += executable
 
-    trinoHomeOpt.map { dir =>
-      Paths.get(dir, "bin", TRINO_ENGINE_BINARY_FILE).toAbsolutePath.toFile.getCanonicalPath
-    }.getOrElse {
-      throw KyuubiSQLException("TRINO_ENGINE_HOME is not set! " +
-        "For more detail information on installing and configuring Trino, please visit " +
-        "https://kyuubi.apache.org/docs/latest/deployment/settings.html#environments")
-    }
-  }
+    // TODO: How shall we deal with proxyUser,
+    // user.name
+    // kyuubi.session.user
+    // or just leave it, because we can handle it at operation layer
+    buffer += s"-D$KYUUBI_SESSION_USER_KEY=$proxyUser"
 
-  override protected def mainResource: Option[String] = {
-    val jarName = s"${module}_$SCALA_COMPILE_VERSION-$KYUUBI_VERSION.jar"
-    // 1. get the main resource jar for user specified config first
-    conf.get(ENGINE_TRINO_MAIN_RESOURCE).filter { userSpecified =>
-      // skip check exist if not local file.
-      val uri = new URI(userSpecified)
-      val schema = if (uri.getScheme != null) uri.getScheme else "file"
-      schema match {
-        case "file" => Files.exists(Paths.get(userSpecified))
-        case _ => true
-      }
-    }.orElse {
-      // 2. get the main resource jar from system build default
-      env.get(KyuubiConf.KYUUBI_HOME)
-        .map { Paths.get(_, "externals", "engines", "trino", "jars", jarName) }
-        .filter(Files.exists(_)).map(_.toAbsolutePath.toFile.getCanonicalPath)
-    }.orElse {
-      // 3. get the main resource from dev environment
-      Option(Paths.get("externals", module, "target", jarName))
-        .filter(Files.exists(_)).orElse {
-          Some(Paths.get("..", "externals", module, "target", jarName))
-        }.map(_.toAbsolutePath.toFile.getCanonicalPath)
+    // TODO: add Kyuubi.engineEnv.TRINO_ENGINE_MEMORY or kyuubi.engine.trino.memory to configure
+    // -Xmx5g
+    // java options
+    for ((k, v) <- conf.getAll) {
+      buffer += s"-D$k=$v"
     }
-  }
-
-  override protected def module: String = "kyuubi-trino-engine"
 
-  override protected def mainClass: String = "org.apache.kyuubi.engine.trino.TrinoSqlEngine"
+    buffer += "-cp"
+    val classpathEntries = new LinkedHashSet[String]
+    // trino engine runtime jar
+    mainResource.foreach(classpathEntries.add)
 
-  override protected def childProcEnv: Map[String, String] = conf.getEnvs +
-    ("TRINO_ENGINE_JAR" -> mainResource.get) +
-    ("TRINO_ENGINE_DYNAMIC_ARGS" ->
-      trinoConf.map { case (k, v) => s"-D$k=$v" }.mkString(" "))
+    mainResource.foreach { path =>
+      classpathEntries.add(s"$path${File.separator}*")
+      val trinoDeps = Paths.get(path).getParent
+        .resolve(s"scala-$SCALA_COMPILE_VERSION")
+        .resolve("jars")

Review Comment:
   Related to [build/dist](https://github.com/apache/incubator-kyuubi/pull/2336/files#diff-b65bcfaa42cfe8af6eb1dba6b26df9ef944859901f355ec553592521fd8ade0d), seems path `scala-$SCALA_COMPILE_VERSION/jars` not exists.



##########
kyuubi-server/src/test/scala/org/apache/kyuubi/engine/trino/TrinoProcessBuilderSuite.scala:
##########
@@ -18,22 +18,25 @@
 package org.apache.kyuubi.engine.trino
 
 import org.apache.kyuubi.KyuubiFunSuite
-import org.apache.kyuubi.KyuubiSQLException
 import org.apache.kyuubi.config.KyuubiConf
-import org.apache.kyuubi.config.KyuubiConf.ENGINE_TRINO_CONNECTION_URL
+import org.apache.kyuubi.config.KyuubiConf._
 
 class TrinoProcessBuilderSuite extends KyuubiFunSuite {
-  private def conf = KyuubiConf().set("kyuubi.on", "off")
 
   test("trino process builder") {
+    val conf = KyuubiConf()
+      .set(ENGINE_TRINO_CONNECTION_URL, "dummy_url")
+      .set(ENGINE_TRINO_CONNECTION_CATALOG, "dumm_catalog")

Review Comment:
   nit: `dummy_catalog `



##########
build/dist:
##########
@@ -257,9 +253,8 @@ cp "$KYUUBI_HOME/externals/kyuubi-flink-sql-engine/target/kyuubi-flink-sql-engin
 cp "$KYUUBI_HOME/externals/kyuubi-spark-sql-engine/target/kyuubi-spark-sql-engine_${SCALA_VERSION}-${VERSION}.jar" "$DISTDIR/externals/engines/spark"
 
 # Copy trino engines
-cp -r "$KYUUBI_HOME/externals/kyuubi-trino-engine/bin/" "$DISTDIR/externals/engines/trino/bin/"
-chmod a+x "$DISTDIR/externals/engines/trino/bin/trino-engine.sh"
-cp "$KYUUBI_HOME/externals/kyuubi-trino-engine/target/kyuubi-trino-engine_${SCALA_VERSION}-${VERSION}.jar" "$DISTDIR/externals/engines/trino/jars"
+cp "$KYUUBI_HOME/externals/kyuubi-trino-engine/target/kyuubi-trino-engine_${SCALA_VERSION}-${VERSION}.jar" "$DISTDIR/externals/engines/trino"
+cp -r "$KYUUBI_HOME/externals/kyuubi-trino-engine/target/scala-$SCALA_VERSION/jars/" "$DISTDIR/externals/engines/trino"

Review Comment:
   The command will put those jars under `engines/trino` directly.  Shell we make them put `trino/jars` or `trino/scala-$SCALA_VERSION/jars/`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] yaooqinn commented on a diff in pull request #2336: Simplify TrinoProcessBuilder with java executable

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on code in PR #2336:
URL: https://github.com/apache/incubator-kyuubi/pull/2336#discussion_r849026792


##########
build/dist:
##########
@@ -257,9 +253,8 @@ cp "$KYUUBI_HOME/externals/kyuubi-flink-sql-engine/target/kyuubi-flink-sql-engin
 cp "$KYUUBI_HOME/externals/kyuubi-spark-sql-engine/target/kyuubi-spark-sql-engine_${SCALA_VERSION}-${VERSION}.jar" "$DISTDIR/externals/engines/spark"
 
 # Copy trino engines
-cp -r "$KYUUBI_HOME/externals/kyuubi-trino-engine/bin/" "$DISTDIR/externals/engines/trino/bin/"
-chmod a+x "$DISTDIR/externals/engines/trino/bin/trino-engine.sh"
-cp "$KYUUBI_HOME/externals/kyuubi-trino-engine/target/kyuubi-trino-engine_${SCALA_VERSION}-${VERSION}.jar" "$DISTDIR/externals/engines/trino/jars"
+cp "$KYUUBI_HOME/externals/kyuubi-trino-engine/target/kyuubi-trino-engine_${SCALA_VERSION}-${VERSION}.jar" "$DISTDIR/externals/engines/trino"
+cp -r "$KYUUBI_HOME/externals/kyuubi-trino-engine/target/scala-$SCALA_VERSION/jars/" "$DISTDIR/externals/engines/trino"

Review Comment:
   I don't agree, it make huge gaps between engines



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] hddong commented on pull request #2336: Simplify TrinoProcessBuilder with java executable

Posted by GitBox <gi...@apache.org>.
hddong commented on PR #2336:
URL: https://github.com/apache/incubator-kyuubi/pull/2336#issuecomment-1097510044

   Thanks, LGTM


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] ulysses-you commented on pull request #2336: Simplify TrinoProcessBuilder with java executable

Posted by GitBox <gi...@apache.org>.
ulysses-you commented on PR #2336:
URL: https://github.com/apache/incubator-kyuubi/pull/2336#issuecomment-1096759424

   great code refactor


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org