This is an automated email from the ASF dual-hosted git repository.
chengpan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kyuubi.git
The following commit(s) were added to refs/heads/master by this push:
new 316205056 [KYUUBI #5076] Add KDF engine_url
316205056 is described below
commit 31620505666c0171068cc73e0339676a484eb877
Author: senmiaoliu <[email protected]>
AuthorDate: Mon Jul 31 13:52:21 2023 +0800
[KYUUBI #5076] Add KDF engine_url
### _Why are the changes needed?_
close #5076
### _How was this patch tested?_
- [x] 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.readthedocs.io/en/master/contributing/code/testing.html#running-tests)
locally before make a pull request
Closes #5102 from lsm1/features/kyuubi_5076.
Closes #5076
ce7cfe678 [senmiaoliu] kdf support engine url
Authored-by: senmiaoliu <[email protected]>
Signed-off-by: Cheng Pan <[email protected]>
---
docs/extensions/engines/spark/functions.md | 1 +
.../apache/kyuubi/engine/spark/SparkSQLEngine.scala | 3 ++-
.../apache/kyuubi/engine/spark/udf/KDFRegistry.scala | 12 +++++++++++-
.../kyuubi/operation/KyuubiOperationPerUserSuite.scala | 18 ++++++++++++++++--
4 files changed, 30 insertions(+), 4 deletions(-)
diff --git a/docs/extensions/engines/spark/functions.md
b/docs/extensions/engines/spark/functions.md
index 66f22aea8..78c269243 100644
--- a/docs/extensions/engines/spark/functions.md
+++ b/docs/extensions/engines/spark/functions.md
@@ -27,4 +27,5 @@ Kyuubi provides several auxiliary SQL functions as supplement
to Spark's [Built-
| engine_id | Return the spark application id for the associated query
engine | string | 1.4.0 |
| system_user | Return the system user name for the associated query engine
| string | 1.3.0 |
| session_user | Return the session username for the associated query engine
| string | 1.4.0 |
+| engine_url | Return the engine url for the associated query engine
| string | 1.8.0 |
diff --git
a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala
b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala
index b94367e9e..5badf695e 100644
---
a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala
+++
b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala
@@ -36,7 +36,7 @@ import org.apache.kyuubi.{KyuubiException, Logging, Utils}
import org.apache.kyuubi.Utils._
import org.apache.kyuubi.config.{KyuubiConf, KyuubiReservedKeys}
import org.apache.kyuubi.config.KyuubiConf._
-import
org.apache.kyuubi.config.KyuubiReservedKeys.KYUUBI_ENGINE_SUBMIT_TIME_KEY
+import
org.apache.kyuubi.config.KyuubiReservedKeys.{KYUUBI_ENGINE_SUBMIT_TIME_KEY,
KYUUBI_ENGINE_URL}
import org.apache.kyuubi.engine.ShareLevel
import org.apache.kyuubi.engine.spark.SparkSQLEngine.{countDownLatch,
currentEngine}
import org.apache.kyuubi.engine.spark.events.{EngineEvent, EngineEventsStore,
SparkEventHandlerRegister}
@@ -290,6 +290,7 @@ object SparkSQLEngine extends Logging {
KyuubiSparkUtil.initializeSparkSession(
session,
kyuubiConf.get(ENGINE_INITIALIZE_SQL) ++
kyuubiConf.get(ENGINE_SESSION_INITIALIZE_SQL))
+ session.sparkContext.setLocalProperty(KYUUBI_ENGINE_URL,
KyuubiSparkUtil.engineUrl)
session
}
diff --git
a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/udf/KDFRegistry.scala
b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/udf/KDFRegistry.scala
index f4612a3d0..a2d50d151 100644
---
a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/udf/KDFRegistry.scala
+++
b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/udf/KDFRegistry.scala
@@ -25,7 +25,7 @@ import org.apache.spark.sql.expressions.UserDefinedFunction
import org.apache.spark.sql.functions.udf
import org.apache.kyuubi.{KYUUBI_VERSION, Utils}
-import org.apache.kyuubi.config.KyuubiReservedKeys.KYUUBI_SESSION_USER_KEY
+import org.apache.kyuubi.config.KyuubiReservedKeys.{KYUUBI_ENGINE_URL,
KYUUBI_SESSION_USER_KEY}
object KDFRegistry {
@@ -73,6 +73,16 @@ object KDFRegistry {
"string",
"1.4.0")
+ val engine_url: KyuubiDefinedFunction = create(
+ "engine_url",
+ udf { () =>
+ Option(TaskContext.get()).map(_.getLocalProperty(KYUUBI_ENGINE_URL))
+ .getOrElse(throw new RuntimeException("Unable to get engine url"))
+ },
+ "Return the engine url for the associated query engine",
+ "string",
+ "1.8.0")
+
def create(
name: String,
udf: UserDefinedFunction,
diff --git
a/kyuubi-server/src/test/scala/org/apache/kyuubi/operation/KyuubiOperationPerUserSuite.scala
b/kyuubi-server/src/test/scala/org/apache/kyuubi/operation/KyuubiOperationPerUserSuite.scala
index fb6f4efa7..a67534164 100644
---
a/kyuubi-server/src/test/scala/org/apache/kyuubi/operation/KyuubiOperationPerUserSuite.scala
+++
b/kyuubi-server/src/test/scala/org/apache/kyuubi/operation/KyuubiOperationPerUserSuite.scala
@@ -17,7 +17,7 @@
package org.apache.kyuubi.operation
-import java.util.UUID
+import java.util.{Properties, UUID}
import org.apache.hadoop.fs.{FileSystem, FileUtil, Path}
import org.apache.hive.service.rpc.thrift.{TExecuteStatementReq, TGetInfoReq,
TGetInfoType, TStatusCode}
@@ -26,7 +26,8 @@ import org.scalatest.time.SpanSugar._
import org.apache.kyuubi.{KYUUBI_VERSION, Utils, WithKyuubiServer,
WithSimpleDFSService}
import org.apache.kyuubi.config.KyuubiConf
import org.apache.kyuubi.config.KyuubiConf.KYUUBI_ENGINE_ENV_PREFIX
-import org.apache.kyuubi.jdbc.hive.KyuubiStatement
+import org.apache.kyuubi.jdbc.KyuubiHiveDriver
+import org.apache.kyuubi.jdbc.hive.{KyuubiConnection, KyuubiStatement}
import org.apache.kyuubi.metrics.{MetricsConstants, MetricsSystem}
import org.apache.kyuubi.session.{KyuubiSessionImpl, SessionHandle}
import org.apache.kyuubi.util.SemanticVersion
@@ -68,6 +69,19 @@ class KyuubiOperationPerUserSuite
}
}
+ test("kyuubi defined function - engine_url") {
+ withSessionConf(Map.empty)(Map.empty)(Map(
+ "spark.ui.enabled" -> "true")) {
+ val driver = new KyuubiHiveDriver()
+ val connection = driver.connect(jdbcUrlWithConf, new Properties())
+ .asInstanceOf[KyuubiConnection]
+ val stmt = connection.createStatement()
+ val rs = stmt.executeQuery("SELECT engine_url()")
+ assert(rs.next())
+ assert(rs.getString(1).nonEmpty)
+ }
+ }
+
test("ensure two connections in user mode share the same engine") {
var r1: String = null
var r2: String = null