Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[KYUUBI #5076] Add KDF engine_url #5102

Closed
wants to merge 1 commit into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions docs/extensions/engines/spark/functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -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 |

Original file line number Diff line number Diff line change
Expand Up @@ -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}
Expand Down Expand Up @@ -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
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 {

Expand Down Expand Up @@ -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))
cxzl25 marked this conversation as resolved.
Show resolved Hide resolved
.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,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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}
Expand All @@ -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
Expand Down Expand Up @@ -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
Expand Down
Loading