[KYUUBI #659] Add UDF engine_name
<!--
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/NetEase/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.
-->
Add UDF engine_name to show the runtime information about execution engine
### _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.readthedocs.io/en/latest/tools/testing.html#running-tests) locally before make a pull request
Closes #747 from zhaomin1423/udf.
Closes #659
8c3b3c16 [Min Zhao] Merge branch 'master' into udf
f184a661 [Min Zhao] Merge branch 'master' into udf
9ea0c190 [Min Zhao] get appName from driver side
09f07523 [Min Zhao] get conf by SparkEnv.get.conf in the engine_name udf
6c000784 [Min Zhao] update the description of the engine_name udf
63ef2762 [Min Zhao] update class to org.apache.kyuubi.engine.spark.udf.KyuubiDefinedFunctionSuite in functions.md
db02884c [Min Zhao] [KYUUBI #659] Add UDF engine_name
Lead-authored-by: Min Zhao <zhaomin1423@163.com>
Co-authored-by: Min Zhao <49054376+zhaomin1423@users.noreply.github.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
(cherry picked from commit 8bab5b8d6a)
Signed-off-by: Cheng Pan <chengpan@apache.org>
This commit is contained in:
parent
d58f970aec
commit
4de2976fd4
@ -30,5 +30,6 @@ Kyuubi provides several auxiliary SQL functions as supplement to Spark's [Built-
|
||||
Name | Description | Return Type | Since
|
||||
--- | --- | --- | ---
|
||||
kyuubi_version | Return the version of Kyuubi Server | string | 1.3.0
|
||||
engine_name | Return the spark application name for the associated query engine | string | 1.3.0
|
||||
system_user | Return the system user name for the associated query engine | string | 1.3.0
|
||||
|
||||
|
||||
@ -29,6 +29,8 @@ object KDFRegistry {
|
||||
|
||||
val registeredFunctions = new ArrayBuffer[KyuubiDefinedFunction]()
|
||||
|
||||
val appName = SparkSession.active.sparkContext.appName
|
||||
|
||||
val kyuubi_version: KyuubiDefinedFunction = create(
|
||||
"kyuubi_version",
|
||||
udf(() => KYUUBI_VERSION).asNonNullable(),
|
||||
@ -36,6 +38,14 @@ object KDFRegistry {
|
||||
"string",
|
||||
"1.3.0")
|
||||
|
||||
val engine_name: KyuubiDefinedFunction = create(
|
||||
"engine_name",
|
||||
udf(() => appName).asNonNullable(),
|
||||
"Return the spark application name for the associated query engine",
|
||||
"string",
|
||||
"1.3.0"
|
||||
)
|
||||
|
||||
val system_user: KyuubiDefinedFunction = create(
|
||||
"system_user",
|
||||
udf(() => System.getProperty("user.name")).asNonNullable(),
|
||||
|
||||
@ -86,7 +86,6 @@ class KyuubiDefinedFunctionSuite extends KyuubiFunSuite {
|
||||
KDFRegistry.registeredFunctions.foreach { func =>
|
||||
newOutput += s"${func.name} | ${func.description} | ${func.returnType} | ${func.since}"
|
||||
}
|
||||
|
||||
newOutput += ""
|
||||
TestUtils.verifyOutput(markdown, newOutput, getClass.getCanonicalName)
|
||||
}
|
||||
|
||||
@ -21,6 +21,7 @@ import java.sql.{Date, SQLException, SQLTimeoutException, Timestamp}
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
import org.apache.commons.lang3.StringUtils
|
||||
import org.apache.hive.service.rpc.thrift.{TExecuteStatementReq, TFetchResultsReq, TOpenSessionReq, TStatusCode}
|
||||
|
||||
import org.apache.kyuubi.KYUUBI_VERSION
|
||||
@ -362,6 +363,14 @@ trait JDBCTests extends BasicJDBCTests {
|
||||
}
|
||||
}
|
||||
|
||||
test("kyuubi defined function - engine_name") {
|
||||
withJdbcStatement() { statement =>
|
||||
val rs = statement.executeQuery("SELECT engine_name()")
|
||||
assert(rs.next())
|
||||
assert(StringUtils.isNotBlank(rs.getString(1)))
|
||||
}
|
||||
}
|
||||
|
||||
// dockerfile use kyuubi as user which is not same with non-k8s env.
|
||||
ignore("kyuubi defined function - system_user") {
|
||||
withJdbcStatement() { statement =>
|
||||
|
||||
Loading…
Reference in New Issue
Block a user