From cdb46e90dbec5d3677860141a0442e9ce5e4694c Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 4 Nov 2020 22:40:02 +0800 Subject: [PATCH] Add Configurations Guide --- conf/kyuubi-defaults.conf | 15 ++ conf/kyuubi-env.sh | 1 + docs/quick_start/quick_start.md | 6 +- docs/server/configurations.md | 148 ----------- docs/server/index.rst | 4 +- docs/server/settings.md | 232 ++++++++++++++++++ .../org/apache/kyuubi/config/KyuubiConf.scala | 57 +++-- .../src/test/resources/kyuubi-defaults.conf | 1 + .../kyuubi/config/KyuubiConfSuite.scala | 1 + .../service/authentication/SaslQOPSuite.scala | 3 +- .../kyuubi/ha/HighAvailabilityConf.scala | 3 +- kyuubi-main/settings.md | 0 .../kyuubi/session/KyuubiSessionImpl.scala | 11 +- .../apache/kyuubi/session/SessionConf.scala | 31 --- .../config/AllKyuubiConfiguration.scala | 199 +++++++++++++++ .../org/apache/spark/KyuubiConfSuite.scala | 52 ---- .../yaooqinn/kyuubi/KyuubiConfSuite.scala | 39 --- 17 files changed, 497 insertions(+), 306 deletions(-) delete mode 100644 docs/server/configurations.md create mode 100644 docs/server/settings.md create mode 100644 kyuubi-main/settings.md delete mode 100644 kyuubi-main/src/main/scala/org/apache/kyuubi/session/SessionConf.scala create mode 100644 kyuubi-main/src/test/scala/org/apache/kyuubi/config/AllKyuubiConfiguration.scala delete mode 100644 kyuubi-server/src/test/scala/org/apache/spark/KyuubiConfSuite.scala delete mode 100644 kyuubi-server/src/test/scala/yaooqinn/kyuubi/KyuubiConfSuite.scala diff --git a/conf/kyuubi-defaults.conf b/conf/kyuubi-defaults.conf index cce3acad3..4c3f98106 100644 --- a/conf/kyuubi-defaults.conf +++ b/conf/kyuubi-defaults.conf @@ -14,3 +14,18 @@ # See the License for the specific language governing permissions and # limitations under the License. # + +## Kyuubi Configurations +# +# kyuubi.authentication NONE +# kyuubi.frontend.bind.port 10009 + +## Spark Configurations +# +# spark.master local +# spark.ui.enabled false + +## Hadoop Configurations +# +# kyuubi.hadoop.authentication KERBEROS +# \ No newline at end of file diff --git a/conf/kyuubi-env.sh b/conf/kyuubi-env.sh index 5d0c2ed38..d14f04e4d 100755 --- a/conf/kyuubi-env.sh +++ b/conf/kyuubi-env.sh @@ -38,3 +38,4 @@ # - SPARK_HOME Spark distribution which you would like to use in Kyuubi. # - SPARK_CONF_DIR Optional directory where the Spark configuration lives. # (Default: $SPARK_HOME/conf) +# diff --git a/docs/quick_start/quick_start.md b/docs/quick_start/quick_start.md index 9924766a5..e5f7785ff 100644 --- a/docs/quick_start/quick_start.md +++ b/docs/quick_start/quick_start.md @@ -109,7 +109,7 @@ bin/kyuubi.sh start During the server starts, it will print all essential environment variables on the screen, you may check whether they are expected. -``` +```logtalk Starting Kyuubi Server from /Users/kentyao/kyuubi/kyuubi-1.0.0-SNAPSHOT-bin-spark-3.0.1 Using kyuubi.sh environment file /Users/kentyao/kyuubi/kyuubi-1.0.0-SNAPSHOT-bin-spark-3.0.1/conf/kyuubi-env.sh to initialize... JAVA_HOME: /Library/Java/JavaVirtualMachines/jdk1.8.0_251.jdk/Contents/Home @@ -154,7 +154,7 @@ and a builtin beeline tool can be found within the pre-built Spark package in th The command below will tell Kyuubi server to create a session with itself. -```sql +```logtalk bin/beeline -u 'jdbc:hive2://localhost:10009/' Connecting to jdbc:hive2://localhost:10009/ Connected to: Spark SQL (version 1.0.0-SNAPSHOT) @@ -190,7 +190,7 @@ Then, you can see 3 processes running in your local environment, including one ` If the beeline session is successfully connected, then you can run any query supported by Spark SQL now. For example, -```sql +```logtalk 0: jdbc:hive2://localhost:10009/> select timestamp '2018-11-17'; 2020-11-02 20:51:49.019 INFO operation.ExecuteStatement: Spark application name: kyuubi_kentyao_spark_20:44:57.240 diff --git a/docs/server/configurations.md b/docs/server/configurations.md deleted file mode 100644 index 11ac06284..000000000 --- a/docs/server/configurations.md +++ /dev/null @@ -1,148 +0,0 @@ -# Kyuubi Configuration Guide - -Kyuubi provides several kinds of properties to configure the system: - -**Kyuubi properties:** control most Kyuubi server's own behaviors. Most of them determined on server starting. They can be treat like normal Spark properties by setting them in `spark-defaults.conf` file or via `--conf` parameter in server starting scripts. - -**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. - -**Hive properties:** are used for SparkSession to talk to the Hive MetaStore Server could be configured in a `hive-site.xml` and placed it in `$SPARK_HOME/conf` directory, or treating them as Spark properties with `spark.hadoop.` prefix. - -**Hadoop properties:** specifying `HADOOP_CONF_DIR` or `YARN_CONF_DIR` to the directory contains hadoop configuration files. - -**Logging** can be configured through `$SPARK_HOME/conf/log4j.properties`. - -## Kyuubi Configurations - -Kyuubi properties control most Kyuubi server's own behaviors. Most of them determined on server starting. They can be treat like normal Spark properties by setting them in `spark-defaults.conf` file or via `--conf` parameter in server starting scripts. - -For instance, start Kyuubi with HA (load balance) enabled. -```bash -$ bin/start-kyuubi.sh \ - --master yarn \ - --deploy-mode client \ - --driver-memory 10g \ - --conf spark.kyuubi.ha.enabled=true \ - --conf spark.kyuubi.ha.zk.quorum=zk1.server.url,zk2.server.url -``` - -#### High Availability - -Name|Default|Description ----|---|--- -spark.kyuubi.
ha.enabled|false|Whether KyuubiServer supports dynamic service discovery for its clients. To support this, each instance of KyuubiServer currently uses ZooKeeper to register itself, when it is brought up. JDBC/ODBC clients should use the ZooKeeper ensemble: spark.kyuubi.ha.zk.quorum in their connection string. -spark.kyuubi.
ha.mode|load-balance|High availability mode, one is load-balance which is used by default, another is failover as master-slave mode. -spark.kyuubi.
ha.zk.quorum|none|Comma separated list of ZooKeeper servers to talk to, when KyuubiServer supports service discovery via Zookeeper. -spark.kyuubi.
ha.zk.namespace|kyuubiserver|The parent node in ZooKeeper used by KyuubiServer when supporting dynamic service discovery. -spark.kyuubi.
ha.zk.client.port|2181|The port of ZooKeeper servers to talk to. If the list of Zookeeper servers specified in spark.kyuubi.zookeeper.quorum does not contain port numbers, this value is used. -spark.kyuubi.
ha.zk.session.timeout|1,200,000|ZooKeeper client's session timeout (in milliseconds). The client is disconnected, and as a result, all locks released, if a heartbeat is not sent in the timeout. -spark.kyuubi.
ha.zk.connection.basesleeptime|1,000|Initial amount of time (in milliseconds) to wait between retries when connecting to the ZooKeeper server when using ExponentialBackoffRetry policy. -spark.kyuubi.
ha.zk.connection.max.retries|3|Max retry times for connecting to the zk server - -#### Operation Log - -Name|Default|Description ----|---|--- -spark.kyuubi.
logging.operation.enabled|true|When true, Kyuubi Server will save operation logs and make them available for clients -spark.kyuubi.
logging.operation.log.dir|KYUUBI_LOG_DIR/
operation_logs|Top level directory where operation logs are stored if logging functionality is enabled - -#### Frontend Service options - -Name|Default|Description ----|---|--- -spark.kyuubi.
frontend.bind.host | localhost | Bind host on which to run the Kyuubi Frontend service. -spark.kyuubi.
frontend.bind.port| 10009 | Port number of Kyuubi Frontend service. set 0 will get a random available one -spark.kyuubi.
frontend.min.worker.threads| 50 | Minimum number of Thrift worker threads. -spark.kyuubi.
frontend.max.worker.threads| 500 | Maximum number of Thrift worker threads -spark.kyuubi.
frontend.worker.keepalive.time | 60s| Keepalive time (in seconds) for an idle worker thread. When the number of workers exceeds min workers, excessive threads are killed after this time interval. -spark.kyuubi.
authentication | NONE | Client authentication types. NONE: no authentication check; NOSASL: no authentication check KERBEROS: Kerberos/GSSAPI authentication. -spark.kyuubi.
frontend.allow.user.substitution | true | Allow alternate user to be specified as part of Kyuubi open connection request. -spark.kyuubi.
frontend.enable.doAs | true | Set true to have Kyuubi execute SQL operations as the user making the calls to it. -spark.kyuubi.
frontend.max.message.size | 104857600 | Maximum message size in bytes a Kyuubi server will accept. - -#### Background Execution Thread Pool - -Name|Default|Description ----|---|--- -spark.kyuubi.
async.exec.threads|100|Number of threads in the async thread pool for KyuubiServer. -spark.kyuubi.
async.exec.wait.queue.size|100|Size of the wait queue for async thread pool in KyuubiServer. After hitting this limit, the async thread pool will reject new requests. -spark.kyuubi.
async.exec.keep.alive.time|10,000|Time (in milliseconds) that an idle KyuubiServer async thread (from the thread pool) will wait for a new task to arrive before terminating. -spark.kyuubi.
async.exec.shutdown.timeout|10,000|How long KyuubiServer shutdown will wait for async threads to terminate. - -#### Kyuubi Session - -Name|Default|Description ----|---|--- -spark.kyuubi.
frontend.session.check.interval|6h|The check interval for frontend session/operation timeout, which can be disabled by setting to zero or negative value. -spark.kyuubi.
frontend.session.timeout|8h|The check interval for session/operation timeout, which can be disabled by setting to zero or negative value. -spark.kyuubi.
frontend.session.check.operation| true |Session will be considered to be idle only if there is no activity, and there is no pending operation. This setting takes effect only if session idle timeout `spark.kyuubi.frontend.session.timeout` and checking `spark.kyuubi.frontend.session.check.interval` are enabled. - -#### Spark Session - -Name|Default|Description ----|---|--- -spark.kyuubi.
backend.session.init.timeout|60s|How long we suggest the server to give up instantiating SparkContext. -spark.kyuubi.
backend.session.check.interval|5min|The check interval for backend session a.k.a SparkSession timeout. -spark.kyuubi.
backend.session.idle.timeout|30min|How long the SparkSession instance will be cached after user logout. Using cached SparkSession can significantly cut the startup time for SparkContext, which makes sense for queries that are short lived. The timeout is calculated from when all sessions of the user are disconnected -spark.kyuubi.
backend.session.max.cache.time|5d|Max cache time for a SparkSession instance when its original copy has been created. When `spark.kyuubi.backend.session.idle.timeout` never is reached for user may continuously run queries, we need this configuration to stop the cached SparkSession which may end up with token expiry issue in kerberized clusters. When in the interval of [t, t * 1.25], we will try to stop the SparkSession gracefully util no connections. But once it fails stop in that region, we will force to stop it -spark.kyuubi.
backend.session.local.dir|KYUUBI_HOME/
local|Default value to set `spark.local.dir`. For YARN mode, this only affect the Kyuubi server side settings according to the rule of Spark treating `spark.local.dir`. - -#### Operation - -Name|Default|Description ----|---|--- -spark.kyuubi.
operation.idle.timeout|6h|Operation will be closed when it's not accessed for this duration of time. -spark.kyuubi.
operation.incremental.collect|false|Whether to use incremental result collection from Spark executor side to Kyuubi server side. -spark.kyuubi.
operation.result.limit|-1|In non-incremental result collection mode, set this to a positive value to limit the size of result collected to driver side. - -#### Metrics - -Please refer to to online documation -- [Kyuubi Server Metrics](https://yaooqinn.github.io/kyuubi/docs/metrics.html) - ---- - -## Spark Configurations - -All properties of Spark can be set as server level ones. Some of them only work for Kyuubi server itself and become immutable, 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` - -#### 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 uses netty as RPC between driver and executor, Kyuubi Server may need much bigger directory memory size. - -```properties -spark.driver.extraJavaOptions -XX:+PrintFlagsFinal -XX:+UnlockDiagnosticVMOptions -XX:ParGCCardsPerStrideChunk=4096 -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:+CMSConcurrentMTEnabled -XX:CMSInitiatingOccupancyFraction=70 -XX:+UseCMSInitiatingOccupancyOnly -XX:+CMSClassUnloadingEnabled -XX:+CMSParallelRemarkEnabled -XX:+UseCondCardMark -XX:PermSize=1024m -XX:MaxPermSize=1024m -XX:MaxDirectMemorySize=8192m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=./logs -XX:OnOutOfMemoryError="kill -9 %p" -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:+PrintTenuringDistribution -Xloggc:./logs/kyuubi-server-gc-%t.log -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=50 -XX:GCLogFileSize=5M -XX:NewRatio=3 -Dio.netty.noPreferDirect=true -Dio.netty.recycler.maxCapacity=0 -``` - -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 properties 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. - -## Hive Configurations - -#### Hive client options - -These configurations are used for SparkSession to talk to Hive MetaStore Server could be configured in a `hive-site.xml` and placed it in `$SPARK_HOME/conf` directory, or treating them as Spark properties with `spark.hadoop.` prefix. - -## Hadoop Configurations - -Please refer to the [Apache Hadoop](http://hadoop.apache.org)'s online documentation for an overview on how to configure Hadoop. - -## Additional Documentations - -[Building Kyuubi](https://yaooqinn.github.io/kyuubi/docs/building.html) -[Kyuubi Deployment Guide](https://yaooqinn.github.io/kyuubi/docs/deploy.html) -[Kyuubi Containerization Guide](https://yaooqinn.github.io/kyuubi/docs/containerization.html) -[High Availability Guide](https://yaooqinn.github.io/kyuubi/docs/high_availability_guide.html) -[Authentication/Security Guide](https://yaooqinn.github.io/kyuubi/docs/authentication.html) -[Kyuubi ACL Management Guide](https://yaooqinn.github.io/kyuubi/docs/authorization.html) -[Kyuubi Architecture](https://yaooqinn.github.io/kyuubi/docs/architecture.html) -[Home Page](https://yaooqinn.github.io/kyuubi/) diff --git a/docs/server/index.rst b/docs/server/index.rst index b825a1827..396571a3d 100644 --- a/docs/server/index.rst +++ b/docs/server/index.rst @@ -4,12 +4,12 @@ Server Documentation .. toctree:: :maxdepth: 2 - :numbered: 2 + :numbered: 3 + settings architecture authentication authorization - configurations containerization deploy high_availability_guide diff --git a/docs/server/settings.md b/docs/server/settings.md new file mode 100644 index 000000000..29f1bf38d --- /dev/null +++ b/docs/server/settings.md @@ -0,0 +1,232 @@ + + + +
+ +![](../imgs/kyuubi_logo_simple.png) + +
+ + +# Configurations Guide + +Kyuubi provides several ways to configure the system. + +## Environments + +You can configure the environment variables in `$KYUUBI_HOME/conf/kyuubi-env.sh`, e.g, `JAVA_HOME`, then this java runtime will be used both for Kyuubi server instance and the applications it launches. You can also change the variable in the subprocess's env configuration file, e.g.`$SPARK_HOME/conf/spark-env.sh` to use more specific ENV for SQL engine applications. + +```bash +#!/usr/bin/env bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# +# - JAVA_HOME Java runtime to use. By default use "java" from PATH. +# +# +# - KYUUBI_CONF_DIR Directory containing the Kyuubi configurations to use. +# (Default: $KYUUBI_HOME/conf) +# - KYUUBI_LOG_DIR Directory for Kyuubi server-side logs. +# (Default: $KYUUBI_HOME/logs) +# - KYUUBI_PID_DIR Directory stores the Kyuubi instance pid file. +# (Default: $KYUUBI_HOME/pid) +# - KYUUBI_MAX_LOG_FILES Maximum number of Kyuubi server logs can rotate to. +# (Default: 5) +# - KYUUBI_JAVA_OPTS JVM options for the Kyuubi server itself in the form "-Dx=y". +# (Default: none). +# - KYUUBI_NICENESS The scheduling priority for Kyuubi server. +# (Default: 0) +# - KYUUBI_WORK_DIR_ROOT Root directory for launching sql engine applications. +# (Default: $KYUUBI_HOME/work) +# - HADOOP_CONF_DIR Directory containing the Hadoop / YARN configuration to use. +# +# - SPARK_HOME Spark distribution which you would like to use in Kyuubi. +# - SPARK_CONF_DIR Optional directory where the Spark configuration lives. +# (Default: $SPARK_HOME/conf) +# +``` +## Kyuubi Configurations + +You can configure the Kyuubi properties in `$KYUUBI_HOME/conf/kyuubi-defaults.conf`. For example: + +```bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +## Kyuubi Configurations +# +# kyuubi.authentication NONE +# kyuubi.frontend.bind.port 10009 + +## Spark Configurations +# +# spark.master local +# spark.ui.enabled false + +## Hadoop Configurations +# +# kyuubi.hadoop.authentication KERBEROS +# +``` +### Authentication + +Key | Default | Meaning | Since +--- | --- | --- | --- +kyuubi\.authentication|
NONE
|
Client authentication types.
|
1.0.0
+kyuubi\.authentication
\.keytab|
<undefined>
|
Location of Kyuubi server's keytab.
|
1.0.0
+kyuubi\.authentication
\.ldap\.base\.dn|
<undefined>
|
LDAP base DN.
|
1.0.0
+kyuubi\.authentication
\.ldap\.domain|
<undefined>
|
LDAP base DN.
|
1.0.0
+kyuubi\.authentication
\.ldap\.url|
<undefined>
|
SPACE character separated LDAP connection URL(s).
|
1.0.0
+kyuubi\.authentication
\.principal|
<undefined>
|
Name of the Kerberos principal.
|
1.0.0
+kyuubi\.authentication
\.sasl\.qop|
auth
|
Sasl QOP enable higher levels of protection for Kyuubi communication with clients.
|
1.0.0
+ +### Delegation + +Key | Default | Meaning | Since +--- | --- | --- | --- +kyuubi\.delegation\.key
\.update\.interval|
PT24H
|
unused yet
|
1.0.0
+kyuubi\.delegation
\.token\.gc\.interval|
PT1H
|
unused yet
|
1.0.0
+kyuubi\.delegation
\.token\.max\.lifetime|
PT168H
|
unused yet
|
1.0.0
+kyuubi\.delegation
\.token\.renew\.interval|
PT168H
|
unused yet
|
1.0.0
+ +### Frontend + +Key | Default | Meaning | Since +--- | --- | --- | --- +kyuubi\.frontend
\.backoff\.slot\.length|
PT0.1S
|
Time to back off during login to the frontend service.
|
1.0.0
+kyuubi\.frontend\.bind
\.host|
<undefined>
|
Hostname or IP of the machine on which to run the frontend service.
|
1.0.0
+kyuubi\.frontend\.bind
\.port|
10009
|
Port of the machine on which to run the frontend service.
|
1.0.0
+kyuubi\.frontend\.login
\.timeout|
PT20S
|
Timeout for Thrift clients during login to the frontend service.
|
1.0.0
+kyuubi\.frontend\.max
\.message\.size|
104857600
|
Maximum message size in bytes a Kyuubi server will accept.
|
1.0.0
+kyuubi\.frontend\.max
\.worker\.threads|
99
|
Maximum number of threads in the of frontend worker thread pool for the frontend service
|
1.0.0
+kyuubi\.frontend\.min
\.worker\.threads|
9
|
Minimum number of threads in the of frontend worker thread pool for the frontend service
|
1.0.0
+kyuubi\.frontend
\.worker\.keepalive\.time|
PT1M
|
Keep-alive time (in milliseconds) for an idle worker thread
|
1.0.0
+ +### Ha + +Key | Default | Meaning | Since +--- | --- | --- | --- +kyuubi\.ha\.zookeeper
\.connection\.base\.retry
\.wait|
1000
|
Initial amount of time to wait between retries to the zookeeper ensemble
|
1.0.0
+kyuubi\.ha\.zookeeper
\.connection\.max
\.retries|
3
|
Max retry times for connecting to the zookeeper ensemble
|
1.0.0
+kyuubi\.ha\.zookeeper
\.connection\.max\.retry
\.wait|
30000
|
Max amount of time to wait between retries for BONDED_EXPONENTIAL_BACKOFF policy can reach, or max time until elapsed for UNTIL_ELAPSED policy to connect the zookeeper ensemble
|
1.0.0
+kyuubi\.ha\.zookeeper
\.connection\.retry
\.policy|
EXPONENTIAL_BACKOFF
|
The retry policy for connecting to the zookeeper ensemble, all candidates are:
|
1.0.0
+kyuubi\.ha\.zookeeper
\.connection\.timeout|
15000
|
The timeout(ms) of creating the connection to the zookeeper ensemble
|
1.0.0
+kyuubi\.ha\.zookeeper
\.namespace|
kyuubi
|
The root directory for the service to deploy its instance uri. Additionally, it will creates a -[username] suffixed root directory for each application
|
1.0.0
+kyuubi\.ha\.zookeeper
\.quorum|
|
The connection string for the zookeeper ensemble
|
1.0.0
+kyuubi\.ha\.zookeeper
\.session\.timeout|
60000
|
The timeout(ms) of a connected session to be idled
|
1.0.0
+ +### Kinit + +Key | Default | Meaning | Since +--- | --- | --- | --- +kyuubi\.kinit\.interval|
PT1H
|
How often will Kyuubi server run `kinit -kt [keytab] [principal]` to renew the local Kerberos credentials cache
|
1.0.0
+kyuubi\.kinit\.max
\.attempts|
10
|
How many times will `kinit` process retry
|
1.0.0
+ +### Operation + +Key | Default | Meaning | Since +--- | --- | --- | --- +kyuubi\.operation\.idle
\.timeout|
PT3H
|
Operation will be closed when it's not accessed for this duration of time
|
1.0.0
+ +### Session + +Key | Default | Meaning | Since +--- | --- | --- | --- +kyuubi\.session\.engine
\.initialize\.timeout|
PT1M
|
Timeout for starting the background engine, e.g. SparkSQLEngine.
|
1.0.0
+kyuubi\.session\.engine
\.login\.timeout|
PT15S
|
The timeout(ms) of creating the connection to remote sql query engine
|
1.0.0
+kyuubi\.session\.engine
\.spark\.main\.resource|
<undefined>
|
The package used to create Spark SQL engine remote application. If it is undefined, Kyuubi will use the default
|
1.0.0
+ +### Zookeeper + +Key | Default | Meaning | Since +--- | --- | --- | --- +kyuubi\.zookeeper
\.embedded\.directory|
embedded_zookeeper
|
The temporary directory for the embedded zookeeper server
|
1.0.0
+kyuubi\.zookeeper
\.embedded\.port|
2181
|
The port of the embedded zookeeper server
|
1.0.0
+ +## Spark Configurations + +### Via spark-defaults.conf + +Setting them in `$SPARK_HOME/conf/spark-defaults.conf` supplies with default values for SQL engine application. Available properties can be found at Spark official online documentation for [Spark Configurations](http://spark.apache.org/docs/latest/configuration.html) + +### Via kyuubi-defaults.conf + +Setting them in `$KYUUBI_HOME/conf/kyuubi-defaults.conf` supplies with default values for SQL engine application too. These properties will override all settings in `$SPARK_HOME/conf/spark-defaults.conf` + +### Via JDBC Connection URL + +Setting them in the JDBC Connection URL supplies session-specific for each SQL engine. For example: ```jdbc:hive2://localhost:10009/default;#spark.sql.shuffle.partitions=2;spark.executor.memory=5g``` + +- **Runtime SQL Configuration** + - For [Runtime SQL Configurations](http://spark.apache.org/docs/latest/configuration.html#runtime-sql-configuration), they will take affect every time +- **Static SQL and Spark Core Configuration** + - For [Static SQL Configurations](http://spark.apache.org/docs/latest/configuration.html#static-sql-configuration) and other spark core configs, e.g. `spark.executor.memory`, they will take affect if there is no existing SQL engine application. Otherwise, they will just be ignored or sometimes fail the connection. +### Via SET Syntax + +Please refer to the Spark official online documentation for [SET Command](http://spark.apache.org/docs/latest/sql-ref-syntax-aux-conf-mgmt-set.html) +## Logging + +Kyuubi uses [log4j](https://logging.apache.org/log4j/2.x/) for logging. You can configure it using `$KYUUBI_HOME/conf/log4j.properties`. + +```bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the console +log4j.rootCategory=INFO, console +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss.SSS} %p %c{2}: %m%n +``` +## Other Configurations + +### Hadoop Configurations + +Specifying `HADOOP_CONF_DIR` to the directory contains hadoop configuration files or treating them as Spark properties with a `spark.hadoop.` prefix. Please refer to the Spark official online documentation for [Inheriting Hadoop Cluster Configuration](http://spark.apache.org/docs/latest/configuration.html#inheriting-hadoop-cluster-configuration). Also, please refer to the [Apache Hadoop](http://hadoop.apache.org)'s online documentation for an overview on how to configure Hadoop. + +### Hive Configurations + +These configurations are used for SQL engine application to talk to Hive MetaStore and could be configured in a `hive-site.xml`. Placed it in `$SPARK_HOME/conf` directory, or treating them as Spark properties with a `spark.hadoop.` prefix. diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala index acb7b0191..51a1faabc 100644 --- a/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala +++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala @@ -35,7 +35,7 @@ case class KyuubiConf(loadSysDefault: Boolean = true) extends Logging { } private def loadFromMap(props: Map[String, String] = Utils.getSystemProperties): KyuubiConf = { - for ((key, value) <- props if key.startsWith("kyuubi.")) { + for ((key, value) <- props if key.startsWith("kyuubi.") || key.startsWith("spark.")) { set(key, value) } this @@ -139,32 +139,32 @@ object KyuubiConf { new ConfigBuilder("kyuubi." + key).onCreate(register) } - val EMBEDDED_ZK_PORT: ConfigEntry[Int] = buildConf("embedded.zookeeper.port") + val EMBEDDED_ZK_PORT: ConfigEntry[Int] = buildConf("zookeeper.embedded.port") .doc("The port of the embedded zookeeper server") .version("1.0.0") .intConf .createWithDefault(2181) - val EMBEDDED_ZK_TEMP_DIR: ConfigEntry[String] = buildConf("embedded.zookeeper.directory") + val EMBEDDED_ZK_TEMP_DIR: ConfigEntry[String] = buildConf("zookeeper.embedded.directory") .doc("The temporary directory for the embedded zookeeper server") .version("1.0.0") .stringConf .createWithDefault("embedded_zookeeper") - val SERVER_PRINCIPAL: OptionalConfigEntry[String] = buildConf("server.principal") + val SERVER_PRINCIPAL: OptionalConfigEntry[String] = buildConf("authentication.principal") .doc("Name of the Kerberos principal.") .version("1.0.0") .stringConf .createOptional - val SERVER_KEYTAB: OptionalConfigEntry[String] = buildConf("server.keytab") + val SERVER_KEYTAB: OptionalConfigEntry[String] = buildConf("authentication.keytab") .doc("Location of Kyuubi server's keytab.") .version("1.0.0") .stringConf .createOptional val KINIT_INTERVAL: ConfigEntry[Long] = buildConf("kinit.interval") - .doc("How often will Kyuubi server run `kinit -kt [keytab] [princical]` to renew the" + + .doc("How often will Kyuubi server run `kinit -kt [keytab] [principal]` to renew the" + " local Kerberos credentials cache") .version("1.0.0") .timeConf @@ -224,26 +224,29 @@ object KyuubiConf { val FRONTEND_MAX_MESSAGE_SIZE: ConfigEntry[Int] = buildConf("frontend.max.message.size") .doc("Maximum message size in bytes a Kyuubi server will accept.") + .version("1.0.0") .intConf .createWithDefault(104857600) val FRONTEND_LOGIN_TIMEOUT: ConfigEntry[Long] = buildConf("frontend.login.timeout") .doc("Timeout for Thrift clients during login to the frontend service.") + .version("1.0.0") .timeConf .createWithDefault(Duration.ofSeconds(20).toMillis) val FRONTEND_LOGIN_BACKOFF_SLOT_LENGTH: ConfigEntry[Long] = buildConf("frontend.backoff.slot.length") .doc("Time to back off during login to the frontend service.") + .version("1.0.0") .timeConf .createWithDefault(Duration.ofMillis(100).toMillis) val AUTHENTICATION_METHOD: ConfigEntry[String] = buildConf("authentication") - .doc("Client authentication types." + - " NONE: no authentication check." + - " KERBEROS: Kerberos/GSSAPI authentication." + - " LDAP: Lightweight Directory Access Protocol authentication.") + .doc("Client authentication types.") .version("1.0.0") .stringConf .transform(_.toUpperCase(Locale.ROOT)) @@ -272,38 +275,38 @@ object KyuubiConf { val DELEGATION_KEY_UPDATE_INTERVAL: ConfigEntry[Long] = buildConf("delegation.key.update.interval") - .doc("") + .doc("unused yet") .version("1.0.0") .timeConf .createWithDefault(Duration.ofDays(1).toMillis) val DELEGATION_TOKEN_MAX_LIFETIME: ConfigEntry[Long] = buildConf("delegation.token.max.lifetime") - .doc("") + .doc("unused yet") .version("1.0.0") .timeConf .createWithDefault(Duration.ofDays(7).toMillis) val DELEGATION_TOKEN_GC_INTERVAL: ConfigEntry[Long] = buildConf("delegation.token.gc.interval") - .doc("") + .doc("unused yet") .version("1.0.0") .timeConf .createWithDefault(Duration.ofHours(1).toMillis) val DELEGATION_TOKEN_RENEW_INTERVAL: ConfigEntry[Long] = buildConf("delegation.token.renew.interval") - .doc("") + .doc("unused yet") .version("1.0.0") .timeConf .createWithDefault(Duration.ofDays(7).toMillis) - val SASL_QOP: ConfigEntry[String] = buildConf("sasl.qop") - .doc("Sasl QOP enable higher levels of protection for Kyuubi communication with clients." + - " auth - authentication only (default)" + - " auth-int - authentication plus integrity protection" + - " auth-conf - authentication plus integrity and confidentiality protectionThis is" + - " applicable only if Kyuubi is configured to use Kerberos authentication.") + val SASL_QOP: ConfigEntry[String] = buildConf("authentication.sasl.qop") + .doc("Sasl QOP enable higher levels of protection for Kyuubi communication with clients.") .version("1.0.0") .stringConf .checkValues(SaslQOP.values.map(_.toString)) @@ -315,14 +318,22 @@ object KyuubiConf { ///////////////////////////////////////////////////////////////////////////////////////////////// val ENGINE_SPARK_MAIN_RESOURCE: OptionalConfigEntry[String] = - buildConf("engine.spark.main.resource") - .doc("The connection string for the zookeeper ensemble") + buildConf("session.engine.spark.main.resource") + .doc("The package used to create Spark SQL engine remote application. If it is undefined," + + " Kyuubi will use the default") .version("1.0.0") .stringConf .createOptional - val ENGINE_INIT_TIMEOUT: ConfigEntry[Long] = buildConf("engine.initialize.timeout") + val ENGINE_LOGIN_TIMEOUT: ConfigEntry[Long] = buildConf("session.engine.login.timeout") + .doc("The timeout(ms) of creating the connection to remote sql query engine") + .version("1.0.0") + .timeConf + .createWithDefault(Duration.ofSeconds(15).toMillis) + + val ENGINE_INIT_TIMEOUT: ConfigEntry[Long] = buildConf("session.engine.initialize.timeout") .doc("Timeout for starting the background engine, e.g. SparkSQLEngine.") + .version("1.0.0") .timeConf .createWithDefault(Duration.ofSeconds(60).toMillis) } diff --git a/kyuubi-common/src/test/resources/kyuubi-defaults.conf b/kyuubi-common/src/test/resources/kyuubi-defaults.conf index cccfa08fc..4c0005eb1 100644 --- a/kyuubi-common/src/test/resources/kyuubi-defaults.conf +++ b/kyuubi-common/src/test/resources/kyuubi-defaults.conf @@ -1,2 +1,3 @@ kyuubi.yes yes +spark.kyuubi.yes no # kyuubi.no no \ No newline at end of file diff --git a/kyuubi-common/src/test/scala/org/apache/kyuubi/config/KyuubiConfSuite.scala b/kyuubi-common/src/test/scala/org/apache/kyuubi/config/KyuubiConfSuite.scala index 5d405cdd3..2849318c0 100644 --- a/kyuubi-common/src/test/scala/org/apache/kyuubi/config/KyuubiConfSuite.scala +++ b/kyuubi-common/src/test/scala/org/apache/kyuubi/config/KyuubiConfSuite.scala @@ -42,6 +42,7 @@ class KyuubiConfSuite extends KyuubiFunSuite { test("load default config file") { val conf = KyuubiConf().loadFileDefaults() assert(conf.getOption("kyuubi.yes").get === "yes") + assert(conf.getOption("spark.kyuubi.yes").get === "no") } diff --git a/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/SaslQOPSuite.scala b/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/SaslQOPSuite.scala index 258e136f1..0de372bda 100644 --- a/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/SaslQOPSuite.scala +++ b/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/SaslQOPSuite.scala @@ -33,7 +33,8 @@ class SaslQOPSuite extends KyuubiFunSuite { conf.set(SASL_QOP, "abc") val e = intercept[IllegalArgumentException](conf.get(SASL_QOP)) assert(e.getMessage === - "The value of kyuubi.sasl.qop should be one of auth, auth-conf, auth-int, but was abc") + "The value of kyuubi.authentication.sasl.qop should be one of" + + " auth, auth-conf, auth-int, but was abc") } } diff --git a/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/HighAvailabilityConf.scala b/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/HighAvailabilityConf.scala index 6fc96d030..4e7faf84a 100644 --- a/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/HighAvailabilityConf.scala +++ b/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/HighAvailabilityConf.scala @@ -75,7 +75,8 @@ object HighAvailabilityConf { val HA_ZK_CONN_RETRY_POLICY: ConfigEntry[String] = buildConf("ha.zookeeper.connection.retry.policy") .doc("The retry policy for connecting to the zookeeper ensemble, all candidates are:" + - s" ${RetryPolicies.values.mkString("[", ", ", "]")}") + s" ${RetryPolicies.values.mkString("")}") + .version("1.0.0") .stringConf .checkValues(RetryPolicies.values.map(_.toString)) .createWithDefault(RetryPolicies.EXPONENTIAL_BACKOFF.toString) diff --git a/kyuubi-main/settings.md b/kyuubi-main/settings.md new file mode 100644 index 000000000..e69de29bb diff --git a/kyuubi-main/src/main/scala/org/apache/kyuubi/session/KyuubiSessionImpl.scala b/kyuubi-main/src/main/scala/org/apache/kyuubi/session/KyuubiSessionImpl.scala index 4c2d4b981..423043f69 100644 --- a/kyuubi-main/src/main/scala/org/apache/kyuubi/session/KyuubiSessionImpl.scala +++ b/kyuubi-main/src/main/scala/org/apache/kyuubi/session/KyuubiSessionImpl.scala @@ -57,7 +57,7 @@ class KyuubiSessionImpl( configureSession() - private val timeout = sessionConf.get(ENGINE_INIT_TIMEOUT) / 1000 + private val timeout: Long = sessionConf.get(ENGINE_INIT_TIMEOUT) private val zkNamespace = s"$zkNamespacePrefix-$user" private val zkPath = ZKPaths.makePath(null, zkNamespace) private lazy val zkClient = ServiceDiscovery.startZookeeperClient(sessionConf) @@ -91,16 +91,15 @@ class KyuubiSessionImpl( val builder = new SparkProcessBuilder(user, sessionConf.toSparkPrefixedConf) val process = builder.start var sh = getServerHost - var count = 0 + val started = System.currentTimeMillis() while (sh.isEmpty) { if (process.waitFor(1, TimeUnit.SECONDS)) { throw builder.getError } - if (count >= timeout) { + if (started + timeout <= System.currentTimeMillis()) { process.destroyForcibly() - throw KyuubiSQLException("Timed out to launched Spark") + throw KyuubiSQLException(s"Timed out($timeout ms) to launched Spark") } - count += 1 sh = getServerHost } val Some((host, port)) = getServerHost @@ -110,7 +109,7 @@ class KyuubiSessionImpl( private def openSession(host: String, port: Int): Unit = { val passwd = Option(password).getOrElse("anonymous") - val loginTimeout = sessionConf.get(SessionConf.ENGINE_LOGIN_TIMEOUT) + val loginTimeout = sessionConf.get(ENGINE_LOGIN_TIMEOUT).toInt transport = PlainSASLHelper.getPlainTransport( user, passwd, new TSocket(host, port, loginTimeout)) if (!transport.isOpen) transport.open() diff --git a/kyuubi-main/src/main/scala/org/apache/kyuubi/session/SessionConf.scala b/kyuubi-main/src/main/scala/org/apache/kyuubi/session/SessionConf.scala deleted file mode 100644 index e1774c3c7..000000000 --- a/kyuubi-main/src/main/scala/org/apache/kyuubi/session/SessionConf.scala +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kyuubi.session - -import org.apache.kyuubi.config.{ConfigBuilder, ConfigEntry, KyuubiConf} - -object SessionConf { - - private def buildConf(key: String): ConfigBuilder = KyuubiConf.buildConf(key) - - val ENGINE_LOGIN_TIMEOUT: ConfigEntry[Int] = buildConf("session.engine.login.timeout") - .doc("The timeout(ms) of creating the connection to remote sql query engine") - .version("1.0.0") - .intConf - .createWithDefault(15 * 1000) -} diff --git a/kyuubi-main/src/test/scala/org/apache/kyuubi/config/AllKyuubiConfiguration.scala b/kyuubi-main/src/test/scala/org/apache/kyuubi/config/AllKyuubiConfiguration.scala new file mode 100644 index 000000000..c925784a2 --- /dev/null +++ b/kyuubi-main/src/test/scala/org/apache/kyuubi/config/AllKyuubiConfiguration.scala @@ -0,0 +1,199 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kyuubi.config + +import java.nio.charset.StandardCharsets +import java.nio.file.{Files, Path, Paths, StandardOpenOption} + +import scala.collection.JavaConverters._ + +import org.apache.kyuubi.KyuubiFunSuite +import org.apache.kyuubi.ha.HighAvailabilityConf + +class AllKyuubiConfiguration extends KyuubiFunSuite { + + private val markdown = Paths.get("..", "docs", "server", "settings.md").toAbsolutePath + + private val writer = Files.newBufferedWriter( + markdown, StandardCharsets.UTF_8, + StandardOpenOption.TRUNCATE_EXISTING, + StandardOpenOption.CREATE) + + def writeWithNewLine(content: String): Unit = { + writer.write(content) + writer.newLine() + writer.flush() + } + + def writeWith2Line(content: String): Unit = { + writeWithNewLine(content) + writer.newLine() + } + + def rewriteToConf(path: Path): Unit = { + val env = + Files.newBufferedReader(path, StandardCharsets.UTF_8) + + try { + writeWithNewLine("```bash") + var line = env.readLine() + while(line != null) { + writeWithNewLine(line) + line = env.readLine() + } + writeWithNewLine("```") + } finally { + env.close() + } + } + + override def afterAll(): Unit = { + writer.close() + super.afterAll() + } + + test("Check all kyuubi configs") { + + KyuubiConf + HighAvailabilityConf + + writeWith2Line("") + + writeWith2Line( + """ + |
+ | + |![](../imgs/kyuubi_logo_simple.png) + | + |
+ |""".stripMargin) + + writeWith2Line("# Configurations Guide") + + writeWith2Line("Kyuubi provides several ways to configure the system.") + + writeWith2Line("## Environments") + writeWith2Line("You can configure the environment variables in" + + " `$KYUUBI_HOME/conf/kyuubi-env.sh`, e.g, `JAVA_HOME`, then this java runtime will be used" + + " both for Kyuubi server instance and the applications it launches. You can also change" + + " the variable in the subprocess's env configuration file, e.g." + + "`$SPARK_HOME/conf/spark-env.sh` to use more specific ENV for SQL engine applications.") + + rewriteToConf(Paths.get("..", "conf", "kyuubi-env.sh")) + + writeWith2Line("## Kyuubi Configurations") + writeWith2Line("You can configure the Kyuubi properties in" + + " `$KYUUBI_HOME/conf/kyuubi-defaults.conf`. For example:") + + rewriteToConf(Paths.get("..", "conf", "kyuubi-defaults.conf")) + + KyuubiConf.kyuubiConfEntries.values().asScala + .toSeq + .groupBy(_.key.split("\\.")(1)) + .toSeq.sortBy(_._1).foreach { case (category, entries) => + + writeWith2Line(s"### ${category.capitalize}") + + writeWithNewLine("Key | Default | Meaning | Since") + writeWithNewLine("--- | --- | --- | ---") + + entries.sortBy(_.key).foreach { c => + val key = { + val sb = new StringBuilder() + var curLen = 0 + c.key.split("\\.").foreach { str => + if (curLen + str.length > 21) { + sb.append("
\\." + str) + curLen = str.length + 1 + } else { + sb.append("\\." + str) + curLen += (str.length + 1) + } + } + sb.toString().stripPrefix("\\.") + } + val dft = c.defaultValStr.replace("<", "<").replace(">", ">") + val seq = Seq( + key, + s"
$dft
", + s"
${c.doc}
", + s"
${c.version}
") + writeWithNewLine(seq.mkString("|")) + } + writer.newLine() + } + + writeWith2Line("## Spark Configurations") + writeWith2Line("### Via spark-defaults.conf") + writeWith2Line("Setting them in `$SPARK_HOME/conf/spark-defaults.conf`" + + " supplies with default values for SQL engine application. Available properties can be" + + " found at Spark official online documentation for" + + " [Spark Configurations](http://spark.apache.org/docs/latest/configuration.html)") + + writeWith2Line("### Via kyuubi-defaults.conf") + writeWith2Line("Setting them in `$KYUUBI_HOME/conf/kyuubi-defaults.conf`" + + " supplies with default values for SQL engine application too. These properties will" + + " override all settings in `$SPARK_HOME/conf/spark-defaults.conf`") + + writeWith2Line("### Via JDBC Connection URL") + writeWith2Line("Setting them in the JDBC Connection URL" + + " supplies session-specific for each SQL engine. For example: " + + "```" + + "jdbc:hive2://localhost:10009/default;#" + + "spark.sql.shuffle.partitions=2;spark.executor.memory=5g" + + "```") + writeWithNewLine("- **Runtime SQL Configuration**") + writeWithNewLine(" - For [Runtime SQL Configurations](" + + "http://spark.apache.org/docs/latest/configuration.html#runtime-sql-configuration), they" + + " will take affect every time") + writeWithNewLine("- **Static SQL and Spark Core Configuration**") + writeWithNewLine(" - For [Static SQL Configurations](" + + "http://spark.apache.org/docs/latest/configuration.html#static-sql-configuration) and" + + " other spark core configs, e.g. `spark.executor.memory`, they will take affect if there" + + " is no existing SQL engine application. Otherwise, they will just be ignored or sometimes" + + " fail the connection.") + writeWith2Line("### Via SET Syntax") + writeWithNewLine("Please refer to the Spark official online documentation for" + + " [SET Command](http://spark.apache.org/docs/latest/sql-ref-syntax-aux-conf-mgmt-set.html)") + + + writeWith2Line("## Logging") + writeWith2Line("Kyuubi uses [log4j](https://logging.apache.org/log4j/2.x/) for logging." + + " You can configure it using `$KYUUBI_HOME/conf/log4j.properties`.") + + rewriteToConf(Paths.get("..", "conf", "log4j.properties")) + + writeWith2Line("## Other Configurations") + + writeWith2Line("### Hadoop Configurations") + writeWith2Line("Specifying `HADOOP_CONF_DIR` to the directory contains hadoop configuration" + + " files or treating them as Spark properties with a `spark.hadoop.` prefix." + + " Please refer to the Spark official online documentation for" + + " [Inheriting Hadoop Cluster Configuration](http://spark.apache.org/docs/latest/" + + "configuration.html#inheriting-hadoop-cluster-configuration)." + + " Also, please refer to the [Apache Hadoop](http://hadoop.apache.org)'s" + + " online documentation for an overview on how to configure Hadoop.") + writeWith2Line("### Hive Configurations") + writeWithNewLine("These configurations are used for SQL engine application to talk to" + + " Hive MetaStore and could be configured in a `hive-site.xml`." + + " Placed it in `$SPARK_HOME/conf` directory, or treating them as Spark properties with" + + " a `spark.hadoop.` prefix.") + + } +} diff --git a/kyuubi-server/src/test/scala/org/apache/spark/KyuubiConfSuite.scala b/kyuubi-server/src/test/scala/org/apache/spark/KyuubiConfSuite.scala deleted file mode 100644 index da0a1bdfa..000000000 --- a/kyuubi-server/src/test/scala/org/apache/spark/KyuubiConfSuite.scala +++ /dev/null @@ -1,52 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark - -class KyuubiConfSuite extends SparkFunSuite { - - test("set kyuubi defaults to spark conf") { - val conf = new SparkConf() - for (kv <- KyuubiConf.getAllDefaults) { - conf.setIfMissing(kv._1, kv._2) - } - assert(conf.get(KyuubiConf.AUTHENTICATION_METHOD) === "NONE") - } - - test("not set explicitly declared kyuubi confs to spark conf") { - val conf = new SparkConf().set(KyuubiConf.AUTHENTICATION_METHOD, "KERBEROS") - for (kv <- KyuubiConf.getAllDefaults) { - conf.setIfMissing(kv._1, kv._2) - } - assert(conf.get(KyuubiConf.AUTHENTICATION_METHOD) === "KERBEROS") - } - - test("load sys props") { - System.setProperty("spark.kyuubi.authentication", "KERBEROS") - val conf = new SparkConf() - for (kv <- KyuubiConf.getAllDefaults) { - conf.setIfMissing(kv._1, kv._2) - } - assert(conf.get(KyuubiConf.AUTHENTICATION_METHOD) === "KERBEROS") - } - - test("register") { - val e = intercept[IllegalArgumentException]( - KyuubiConf.register(KyuubiConf.AUTHENTICATION_METHOD)) - assert(e.getMessage.contains("spark.kyuubi.authentication has been registered")) - } -} diff --git a/kyuubi-server/src/test/scala/yaooqinn/kyuubi/KyuubiConfSuite.scala b/kyuubi-server/src/test/scala/yaooqinn/kyuubi/KyuubiConfSuite.scala deleted file mode 100644 index 7e6e7e2a7..000000000 --- a/kyuubi-server/src/test/scala/yaooqinn/kyuubi/KyuubiConfSuite.scala +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package yaooqinn.kyuubi - -import org.apache.spark.{KyuubiConf, SparkConf, SparkFunSuite} -import org.apache.spark.KyuubiConf._ - -class KyuubiConfSuite extends SparkFunSuite { - private val conf: SparkConf = new SparkConf() - - KyuubiConf.getAllDefaults.foreach { case (k, v) => conf.set(k, v) } - test("implicits") { - - assert(conf.getOption(AUTHORIZATION_ENABLE).nonEmpty) - assert(!conf.get(AUTHORIZATION_ENABLE).toBoolean) - - assert(conf.getOption(YARN_CONTAINER_TIMEOUT).nonEmpty) - assert(conf.get(YARN_CONTAINER_TIMEOUT) === "60000ms") - - assert(conf.getOption(AUTHENTICATION_METHOD).nonEmpty) - assert(conf.get(AUTHENTICATION_METHOD) === "NONE") - } - -}