Merge pull request #77 from yaooqinn/i76

updating some outdated documentations
This commit is contained in:
Kent Yao 2018-05-26 15:15:59 +08:00 committed by GitHub
commit 3448923242
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
9 changed files with 103 additions and 191 deletions

1
.gitignore vendored
View File

@ -31,4 +31,5 @@ pid/
local/
out/
hs_err_pid*
spark-warehouse/

View File

@ -3,29 +3,30 @@
<img style="zoom: 0.3141592653589" src="docs/imgs/kyuubi.png" />
**Kyuubi** is an enhanced edition of the [Apache Spark](http://spark.apache.org)'s primordial
[Thrift JDBC/ODBC Server](http://spark.apache.org/docs/latest/sql-programming-guide.html#running-the-thrift-jdbcodbc-server). It is mainly designed for directly running SQL towards a cluster with all components including HDFS, YARN, Hive MetaStore, and itself secured.
[Thrift JDBC/ODBC Server](http://spark.apache.org/docs/latest/sql-programming-guide.html#running-the-thrift-jdbcodbc-server). It is mainly designed for directly running SQL towards a cluster with all components including HDFS, YARN, Hive MetaStore, and itself secured. Kyuubi is a Spark SQL thrift service with end-to-end multi tenant guaranteed. Please go to [Kyuubi Architecture](https://yaooqinn.github.io/kyuubi/docs/architecture.html) to learn more if you are interested.
Basicaly, the Thrift JDBC/ODBC Server as a similar ad-hoc SQL query service of [Apache Hive](https://hive.apache.org)'s [HiveServer2](https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Overview) for Spark SQL, acts as a distributed query engine using its JDBC/ODBC or command-line interface.
Basically, the Thrift JDBC/ODBC Server as a similar ad-hoc SQL query service of [Apache Hive](https://hive.apache.org)'s [HiveServer2](https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Overview) for Spark SQL, acts as a distributed query engine using its JDBC/ODBC or command-line interface.
In this mode, end-users or applications can interact with Spark SQL directly to run SQL queries, without the need to write any code. We can make pretty business reports with massive data using some BI tools which supported JDBC/ODBC connections, such as [Tableau](https://www.tableau.com), [NetEase YouData](https://youdata.163.com) and so on. Benefitting from Apache Spark's capability, we can archive much more performance improvement than Apache Hive as a SQL on Hadoop service.
But unfortunately, due to the limitations of Spark's own architectureto be used as an enterprise-class product, there are a number of problems compared with HiveServer2such as multi-tenant isolation, authentication/authorization, high concurrency, high availability, and so on. And the Apache Spark community's support for this module has been in a state of prolonged stagnation.
**Kyuubi** has enhanced the Thrift JDBC/ODBC Server in some ways for these existing problems, as shown in the following table.
**Kyuubi** has enhanced the Thrift JDBC/ODBC Server in some ways for solving these existing problems, as shown in the following table.
|---|**Thrift Server**|**Kyuubi**|Comments|
|---|---|---|---|
|Multi SparkContext Instances| ✘ | ✔ |Apache Spark has several [issues](https://www.jianshu.com/p/e1cfcaece8f1) to have multiple SparkContext instances in one single JVM. Option `spark.driver.allowMultipleContexts=true` only enables SparkContext to be instantiated many times but these instance can only share and use the scheduler and execution environments of the last initialized one, which is kind of like a shallow copy of a Java object. The patches of Kyuubi provides a way of isolating these components by user to avoid overlapping.|
|Dynamic SparkContext Initialization| ✘ | ✔ |Each SparkContext initialization is delayed to the phase of first session of a particular user's creation in Kyuubi, while Thrift JDBC/ODBC Server create one only when it starts.|
|Dynamic SparkContext Recycling| ✘ | ✔ | In Thrift JDBC/ODBC Server, SparkContext is a resident variable. Kyuubi will cache SparkContext instances for a while after session closed before the server terminating them.|
|Dynamic Yarn Queue| ✘ | ✔ |We use `spark.yarn.queue` to specifying the queue that Spark on Yarn applications run into. Once Thrift JDBC/ODBC Server started, it becomes unchangeable, while HiveServer2 could switch queue by`set mapred.job.queue.name=thequeue`. Kyuubi adopts a compromise method which could identify and use `spark.yarn.queue` in the connection string.|
|Dynamic Configuring| only `spark.sql.*` | ✔ |Kyuubi supports all Spark/Hive/Hadoop configurations, such as `spark.executor.cores/memory`, to be set in the connection string which will be used to initialize SparkContext. |
|Authorization| ✘ | ✘ |[Spark Authorizer](https://github.com/yaooqinn/spark-authorizer) will be add to Kyuubi soon.|
|Impersonation|`--proxy-user singleuser`| ✔ |Kyuubi fully support `hive.server2.proxy.user` and `hive.server2.doAs`|
|Multi Tenancy| ✘ | ✔ |Based on the above featuresKyuubi is able to run as a multi-tenant server on a LCE supported Yarn cluster.|
|SQL Operation Log| ✘ | ✔ |Kyuubi redirect sql operation log to local file which has an interface for the client to fetch.|
|High Availability| ✘ | ✔ |Based on ZooKeeper |
|cluster deploy mode| ✘ | ✘ |yarn cluster mode will be supported soon|
|Type Mapping| ✘ | ✔ |Kyuubi support Spark result/schema to be directly converted to Thrift result/schemas bypassing Hive format results|
|Features|Thrift Server|Kyuubi|Comments|
|:---:|:---:|:---:|:---|
|multiple `SparkContext`| ✘ | ✔ |Spark has several [issues](https://www.jianshu.com/p/e1cfcaece8f1) to have multiple `SparkContext` instances in one single JVM. Option `spark.driver.allowMultipleContexts=true` only enables `SparkContext` to be instantiated many times but these instances can only share and use the scheduler and execution environments of the last initialized one, which is kind of like a shallow copy of a Java object. Kyuubi provides a way of isolating these components by user to avoid overlapping.|
|["lazy" `SparkContext`](https://yaooqinn.github.io/kyuubi/docs/architecture.html#1.2.1)| ✘ | ✔ |Each `SparkContext` initialization is delayed to the phase of first session of a particular user's creation in Kyuubi, while Thrift JDBC/ODBC Server create one only when it starts.|
|[`SparkContext` cache](https://yaooqinn.github.io/kyuubi/docs/architecture.html#1.2.2)| ✘ | ✔ | In Thrift JDBC/ODBC Server, `SparkContext` is a resident variable. Kyuubi will cache `SparkContext` instances for a while after session closed before the server terminating them.|
|dynamic queue| ✘ | ✔ |We use `spark.yarn.queue` to specifying the queue that Spark on Yarn applications run into. Once Thrift JDBC/ODBC Server started, it becomes unchangeable, while HiveServer2 could switch queue by`set mapred.job.queue.name=thequeue`. Kyuubi adopts a compromise method which could identify and use `spark.yarn.queue` in the connection string.|
|[session level configurations](https://yaooqinn.github.io/kyuubi/docs/architecture.html#1.2.1)|`spark.sql.*`| ✔ |Kyuubi supports all Spark/Hive/Hadoop configurations, such as `spark.executor.cores/memory`, to be set in the connection string which will be used to initialize `SparkContext`. |
|authentication| ✔ | ✔ |Please refer to the [Authentication/Security Guide](https://yaooqinn.github.io/kyuubi/docs/authentication.html) |
|authorization| ✘ | ✘ |[Spark Authorizer](https://github.com/yaooqinn/spark-authorizer) will be add to Kyuubi soon.|
|impersonation| ✘ | ✔ |Kyuubi fully support `hive.server2.proxy.user` and `hive.server2.doAs`|
|multi tenancy| ✘ | ✔ |Based on the above featuresKyuubi is able to run as a multi-tenant server on a LCE supported Yarn cluster.|
|operation log| ✘ | ✔ |Kyuubi redirect sql operation log to local file which has an interface for the client to fetch.|
|[high availability](https://yaooqinn.github.io/kyuubi/docs/architecture.html#1.4)| ✘ | ✔ |Based on ZooKeeper dynamic service discovery |
|cluster mode| ✘ | ✘ |yarn cluster mode will be supported soon|
|type mapping| ✘ | ✔ |Kyuubi support Spark result/schema to be directly converted to Thrift result/schemas bypassing Hive format results|
## Getting Started
@ -35,39 +36,8 @@ Please refer to the [Building Kyuubi](https://yaooqinn.github.io/kyuubi/docs/bui
### Start Kyuubi
#### 1. As a normal spark application
For test cases, your can run Kyuubi Server as a normal spark application.
```bash
$ $SPARK_HOME/bin/spark-submit \
--class yaooqinn.kyuubi.server.KyuubiServer \
--master yarn \
--deploy-mode client \
--driver-memory 10g \
--conf spark.kyuubi.frontend.bind.port=10009 \
$KYUUBI_HOME/target/kyuubi-<version>.jar
```
**NOTE:** No multi tenancy guarantee if without [Patch](https://github.com/yaooqinn/kyuubi/tree/master/patches)
#### 2. As a long running service
Using `nohup` and `&` could run Kyuubi as a long running service
```bash
$ nohup $SPARK_HOME/bin/spark-submit \
--class yaooqinn.kyuubi.server.KyuubiServer \
--master yarn \
--deploy-mode client \
--driver-memory 10g \
--conf spark.kyuubi.frontend.bind.port=10009 \
$KYUUBI_HOME/target/kyuubi-<version>.jar &
```
**NOTE:** No multi tenancy guarantee if without [Patch](https://github.com/yaooqinn/kyuubi/tree/master/patches)
#### 3. With built-in startup script
The more recommended way is through the built-in startup script `bin/start-kyuubi.sh`
First of all, export `SPARK_HOME` in $KYUUBI_HOME/bin/kyuubi-env.sh`
We can start Kyuubi with the built-in startup script `bin/start-kyuubi.sh`.
First of all, export `SPARK_HOME` in `$KYUUBI_HOME/bin/kyuubi-env.sh`
```bash
export SPARK_HOME=/the/path/to/a/runable/spark/binary/dir
@ -81,7 +51,6 @@ $ bin/start-kyuubi.sh \
--driver-memory 10g \
--conf spark.kyuubi.frontend.bind.port=10009
```
**NOTE:** Full multi tenancy guarantee by default on YARN.
### Run Spark SQL on Kyuubi
@ -92,7 +61,6 @@ Now you can use [beeline](https://cwiki.apache.org/confluence/display/Hive/HiveS
```bash
bin/stop-kyuubi.sh
```
**Notes:** Obviouslywithout the patches we supplied, Kyuubi is mostly same with the Thrift JDBC/ODBC Server as an non-multi-tenancy server.
## Multi Tenancy Support
@ -109,16 +77,12 @@ Suppose that you already have a secured HDFS cluster for deploying Spark, Hive o
+ Queues(Optional), please refer to [Capacity Scheduler](https://hadoop.apache.org/docs/r2.7.2/hadoop-yarn/hadoop-yarn-site/CapacityScheduler.html) or [Fair Scheduler](https://hadoop.apache.org/docs/r2.7.2/hadoop-yarn/hadoop-yarn-site/FairScheduler.html) to see more.
#### Spark on Yarn
- Setup for [Spark On Yarn](http://spark.apache.org/docs/latest/running-on-yarn.html)
- Setup for [Spark On Yarn](http://spark.apache.org/docs/latest/running-on-yarn.html) Ensure that `HADOOP_CONF_DIR` or `YARN_CONF_DIR` points to the directory which contains the (client side) configuration files for the Hadoop cluster.
#### Configure Hive
- Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` and `hdfs-site.xml` files in `$SPARK_HOME/conf`.
#### Patch Spark (Deprecated)
- Apply a simple patch from [Patches Directory](https://github.com/yaooqinn/kyuubi/tree/master/patches) to specified Spark version
- [Build Spark](http://spark.apache.org/docs/latest/building-spark.html) of your own.
## Configuration
Please refer to the [Configuration Guide](https://yaooqinn.github.io/kyuubi/docs/configurations.html) in the online documentation for an overview on how to configure Kyuubi.
@ -127,6 +91,8 @@ Please refer to the [Configuration Guide](https://yaooqinn.github.io/kyuubi/docs
Please refer to the [Authentication/Security Guide](https://yaooqinn.github.io/kyuubi/docs/authentication.html) in the online documentation for an overview on how to enable security for Kyuubi.
## Additional Documentation
## Additional Documentations
[Building Kyuubi](https://yaooqinn.github.io/kyuubi/docs/building.html)
[Configuration Guide](https://yaooqinn.github.io/kyuubi/docs/configurations.html)
[Authentication/Security Guide](https://yaooqinn.github.io/kyuubi/docs/authentication.html)
[Kyuubi Architecture](https://yaooqinn.github.io/kyuubi/docs/architecture.html)

View File

@ -1,4 +1,4 @@
# Kyuubi Architecture
# Kyuubi Architecture Introduction
- [Unified Interface](#1.1)
- [Runtime Resource Resiliency](#1.2)
@ -17,7 +17,7 @@ It is mainly designed for directly running SQL towards a cluster with all compon
and itself secured. The main purpose of Kyuubi is to realize an architecture that can not only speed up SQL queries using
Spark SQL Engine, and also be compatible with the HiveServer2's behavior as much as possible. Thus, Kyuubi use the same protocol
of HiveServer2, which can be found at [HiveServer2 Thrift API](https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Thrift+API)
as the client-server communication mechanism, and a user session level `SparkContext` instantiating/registering/caching/recycling
as the client-server communication mechanism, and a user session level `SparkContext` instantiating / registering / caching / recycling
mechanism to implement multi-tenant functionality.
<div style="text-align: center">
@ -117,4 +117,9 @@ Kyuubi's internal is very simple to understand, which is shown as the picture be
<img style="zoom: 1.00" src="./imgs/kyuubi_internal.png" />
</div>
[Home Page](https://yaooqinn.github.io/kyuubi/)
## Additional Documentations
[Building Kyuubi](https://yaooqinn.github.io/kyuubi/docs/building.html)
[Configuration Guide](https://yaooqinn.github.io/kyuubi/docs/configurations.html)
[Authentication/Security Guide](https://yaooqinn.github.io/kyuubi/docs/authentication.html)
[Home Page](https://yaooqinn.github.io/kyuubi/)

View File

@ -1,4 +1,4 @@
# Authentication/Security Guide
# Kyuubi Authentication/Security Guide
Kyuubi supports Anonymous (no authentication) with and without SASL, Kerberos (GSSAPI), pass through LDAP between the Thrift client and itself.
## Configuration
@ -46,4 +46,9 @@ $KYUUBI_HOME/bin/start-kyuubi.sh --conf spark.kyuubi.authentication=KERBEROS
$SPARK_HOME/bin/beeline -u "jdbc:hive2://${replace with spark.kyuubi.frontend.bind.host}:10000/;principal=${replace with spark.yarn.principal};hive.server2.proxy.user=yaooqinn"
```
[Home Page](https://yaooqinn.github.io/kyuubi/)
## Additional Documentations
[Building Kyuubi](https://yaooqinn.github.io/kyuubi/docs/building.html)
[Configuration Guide](https://yaooqinn.github.io/kyuubi/docs/configurations.html)
[Kyuubi Architecture](https://yaooqinn.github.io/kyuubi/docs/architecture.html)
[Home Page](https://yaooqinn.github.io/kyuubi/)

View File

@ -37,4 +37,9 @@ With Maven, you can use the -DwildcardSuites flag to run individual Scala tests:
For more information about the ScalaTest Maven Plugin, refer to the [ScalaTest documentation](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin).
[Home Page](https://yaooqinn.github.io/kyuubi/)
## Additional Documentations
[Configuration Guide](https://yaooqinn.github.io/kyuubi/docs/configurations.html)
[Authentication/Security Guide](https://yaooqinn.github.io/kyuubi/docs/authentication.html)
[Kyuubi Architecture](https://yaooqinn.github.io/kyuubi/docs/architecture.html)
[Home Page](https://yaooqinn.github.io/kyuubi/)

View File

@ -1,4 +1,4 @@
# Configuration Guide
# Kyuubi Configuration Guide
Kyuubi provides several kinds of properties to configure the system:
@ -97,14 +97,23 @@ spark.kyuubi.operation.incremental.collect|false|Whether to use incremental resu
---
## Spark Configurations
All properties of Spark can be set as servel level ones. Some of them only work for Kyuubi server itself and become unchangable, such as `spark.driver.memory` specifying the heap memory of server. Session level Spark properties take Server lever ones as default values and can be changed with session connection strings. And obviously, all sql properties of Spark can be set via `set` statement at runtime, such as `set spark.sql.autoBroadcastJoinThreshold=-1`
Spark properties become session level options, which are used to generate a SparkContext instances and passed to Kyuubi Server by JDBC/ODBC connection strings. Setting them in `$SPARK_HOME/conf/spark-defaults.conf` supplies with default values for each session.
#### Session Level
Spark properties which becomes session level options, which are used to generate a `SparkContext` instances and passed to Kyuubi Server by JDBC/ODBC connection strings. Setting them in `$SPARK_HOME/conf/spark-defaults.conf` supplies with default values for each session.
#### Server Level
Name|Default|Description
---|---|---
spark.driver.memory| 1g | Amount of memory to use for the Kyuubi Server instance. Set this through the --driver-memory command line option or in your default properties file.
spark.driver.extraJavaOptions| (none) | A string of extra JVM options to pass to the Kyuubi Server instance. For instance, GC settings or other logging. Set this through the --driver-java-options command line option or in your default properties file.
Spark use netty as RPC between driver and executor, Kyuubi Server may need much bigger directory memory size.
```properties
spark.driver.extraJavaOptions -XX:PermSize=1024m -XX:MaxPermSize=1024m -XX:MaxDirectMemorySize=4096m
```
Spark properties for [Driver](http://spark.apache.org/docs/latest/configuration.html#runtime-environment) like those above controls Kyuubi Server's own behaviors, while other properies could be set in JDBC/ODBC connection strings.
Please refer to the [Configuration Guide](http://spark.apache.org/docs/latest/configuration.html) in the online documentation for an overview on how to configure Spark.
@ -117,4 +126,9 @@ These configurations are used for SparkSessin to talk to Hive MetaStore Server c
## Hadoop Configurations
Please refer to the [Apache Hadoop](http://hadoop.apache.org)'s online documentation for an overview on how to configure Hadoop.
[Home Page](https://yaooqinn.github.io/kyuubi/)
## Additional Documentations
[Building Kyuubi](https://yaooqinn.github.io/kyuubi/docs/building.html)
[Authentication/Security Guide](https://yaooqinn.github.io/kyuubi/docs/authentication.html)
[Kyuubi Architecture](https://yaooqinn.github.io/kyuubi/docs/architecture.html)
[Home Page](https://yaooqinn.github.io/kyuubi/)

View File

@ -1,60 +0,0 @@
From b7ae7c5a7ef64d8209bc3ce4f8303d41470f8bc6 Mon Sep 17 00:00:00 2001
From: Kent Yao <yaooqinn@hotmail.com>
Date: Fri, 19 Jan 2018 16:50:35 +0800
Subject: [PATCH] support multi sc for diff users towards spark 2.1
---
core/src/main/scala/org/apache/spark/SparkEnv.scala | 14 +++++++++++---
1 file changed, 11 insertions(+), 3 deletions(-)
diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala
index 1296386..14c3407 100644
--- a/core/src/main/scala/org/apache/spark/SparkEnv.scala
+++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala
@@ -19,11 +19,13 @@ package org.apache.spark
import java.io.File
import java.net.Socket
+import java.util.concurrent.ConcurrentHashMap
import scala.collection.mutable
import scala.util.Properties
import com.google.common.collect.MapMaker
+import org.apache.hadoop.security.UserGroupInformation
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.api.python.PythonWorkerFactory
@@ -135,20 +137,26 @@ class SparkEnv (
}
object SparkEnv extends Logging {
- @volatile private var env: SparkEnv = _
+ private val env = new ConcurrentHashMap[String, SparkEnv]()
private[spark] val driverSystemName = "sparkDriver"
private[spark] val executorSystemName = "sparkExecutor"
+ private[this] def user = UserGroupInformation.getCurrentUser.getShortUserName
+
def set(e: SparkEnv) {
- env = e
+ if (e == null) {
+ env.remove(user)
+ } else {
+ env.put(user, e)
+ }
}
/**
* Returns the SparkEnv.
*/
def get: SparkEnv = {
- env
+ env.get(user)
}
/**
--
2.6.2

View File

@ -1,61 +0,0 @@
From 0813b0fede8821708df7f03f585e856bbd8d2fb9 Mon Sep 17 00:00:00 2001
From: Kent Yao <yaooqinn@hotmail.com>
Date: Fri, 19 Jan 2018 16:50:35 +0800
Subject: [PATCH] support multi sc for diff users towards spark 2.2
---
core/src/main/scala/org/apache/spark/SparkEnv.scala | 14 +++++++++++---
1 file changed, 11 insertions(+), 3 deletions(-)
diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala
index f4a59f0..a32e2fb 100644
--- a/core/src/main/scala/org/apache/spark/SparkEnv.scala
+++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala
@@ -19,12 +19,14 @@ package org.apache.spark
import java.io.File
import java.net.Socket
+import java.util.concurrent.ConcurrentHashMap
import java.util.Locale
import scala.collection.mutable
import scala.util.Properties
import com.google.common.collect.MapMaker
+import org.apache.hadoop.security.UserGroupInformation
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.api.python.PythonWorkerFactory
@@ -136,20 +138,26 @@ class SparkEnv (
}
object SparkEnv extends Logging {
- @volatile private var env: SparkEnv = _
+ private val env = new ConcurrentHashMap[String, SparkEnv]()
private[spark] val driverSystemName = "sparkDriver"
private[spark] val executorSystemName = "sparkExecutor"
+ private[this] def user = UserGroupInformation.getCurrentUser.getShortUserName
+
def set(e: SparkEnv) {
- env = e
+ if (e == null) {
+ env.remove(user)
+ } else {
+ env.put(user, e)
+ }
}
/**
* Returns the SparkEnv.
*/
def get: SparkEnv = {
- env
+ env.get(user)
}
/**
--
2.6.2

View File

@ -17,13 +17,18 @@
package org.apache.spark
import java.net.URL
import java.security.PrivilegedExceptionAction
import scala.reflect.internal.util.ScalaClassLoader.URLClassLoader
import org.apache.hadoop.security.UserGroupInformation
import org.apache.spark.util.SignalUtils
import yaooqinn.kyuubi.{KyuubiServerException, SPARK_COMPILE_VERSION}
import yaooqinn.kyuubi.{KyuubiServerException, Logging, SPARK_COMPILE_VERSION}
import yaooqinn.kyuubi.utils.ReflectUtils
class KyuubiSparkUtilSuite extends SparkFunSuite {
class KyuubiSparkUtilSuite extends SparkFunSuite with Logging {
test("get current user name") {
val user = KyuubiSparkUtil.getCurrentUserName()
@ -101,8 +106,11 @@ class KyuubiSparkUtilSuite extends SparkFunSuite {
test("testCreateTempDir") {
val tmpDir = KyuubiSparkUtil.createTempDir(namePrefix = "test_kyuubi")
val tmpDir2 = KyuubiSparkUtil.createTempDir()
assert(tmpDir.exists())
assert(tmpDir.isDirectory)
assert(tmpDir2.exists())
assert(tmpDir2.isDirectory)
}
test("testExceptionString") {
@ -154,4 +162,33 @@ class KyuubiSparkUtilSuite extends SparkFunSuite {
assert(KyuubiSparkUtil.timeStringAsMs("50min") === 50 * 60 * 1000L)
assert(KyuubiSparkUtil.timeStringAsMs("100ms") === 100L)
}
test("testGetContextClassLoader") {
val origin = Thread.currentThread().getContextClassLoader
try {
assert(KyuubiSparkUtil.getContextOrSparkClassLoader() === origin)
val classloader = new URLClassLoader(Seq.empty[URL], origin)
Thread.currentThread().setContextClassLoader(classloader)
assert(KyuubiSparkUtil.getContextOrSparkClassLoader() === classloader)
} finally {
Thread.currentThread().setContextClassLoader(origin)
}
}
test("testInitDaemon") {
KyuubiSparkUtil.initDaemon(logger)
assert(ReflectUtils.getFieldValue(SignalUtils, "loggerRegistered") === true)
}
test("testAddShutdownHook") {
val x = 1
var y: Int = 0
def f(): Int = {
y = x * 2
y
}
KyuubiSparkUtil.addShutdownHook(f)
assert(y === 0)
}
}