[KYUUBI #1477] Use KyuubiHadoopUtils.newHadoopConf instead of new Con…
…figuration() <!-- 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/apache/incubator-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. --> ### _How was this patch tested?_ - [x] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [x] Add screenshots for manual tests if appropriate - EventLoggingServiceSuite  - HadoopFsDelegationTokenProviderSuite  - KyuubiHadoopUtilsSuite  - [x] [Run test](https://kyuubi.readthedocs.io/en/latest/develop_tools/testing.html#running-tests) locally before make a pull request Closes #1661 from xifeng/kyuubi-1477. Closes #1477 ad068c6e [xifeng yang] [KYUUBI #1477] Use KyuubiHadoopUtils.newHadoopConf instead of new Configuration() Authored-by: xifeng yang <xifeng.yang@hotmail.com> Signed-off-by: Kent Yao <yao@apache.org>
This commit is contained in:
parent
2f1f2589cc
commit
628719fd60
@ -41,9 +41,12 @@ object KyuubiHadoopUtils {
|
||||
classOf[Credentials].getDeclaredField("tokenMap")
|
||||
tokenMapField.setAccessible(true)
|
||||
|
||||
def newHadoopConf(conf: KyuubiConf): Configuration = {
|
||||
val hadoopConf = new Configuration()
|
||||
conf.getAll.foreach { case (k, v) => hadoopConf.set(k, v) }
|
||||
def newHadoopConf(
|
||||
conf: KyuubiConf,
|
||||
loadDefaults: Boolean = true): Configuration = {
|
||||
val hadoopConf = new Configuration(loadDefaults)
|
||||
conf.getAll
|
||||
.foreach { case (k, v) => hadoopConf.set(k, v) }
|
||||
hadoopConf
|
||||
}
|
||||
|
||||
|
||||
@ -17,6 +17,8 @@
|
||||
|
||||
package org.apache.kyuubi.util
|
||||
|
||||
import java.util.stream.StreamSupport
|
||||
|
||||
import scala.util.Random
|
||||
|
||||
import org.apache.hadoop.io.Text
|
||||
@ -59,4 +61,18 @@ class KyuubiHadoopUtilsSuite extends KyuubiFunSuite {
|
||||
KyuubiHadoopUtils.encodeCredentials(credentials))
|
||||
assert(decoded.getToken(token.getKind) == credentials.getToken(token.getKind))
|
||||
}
|
||||
|
||||
test("new hadoop conf with kyuubi conf with loadDefaults") {
|
||||
val abc = "kyuubi.abc"
|
||||
val kyuubiConf = new KyuubiConf()
|
||||
.set(abc, "xyz")
|
||||
|
||||
var hadoopConf = KyuubiHadoopUtils.newHadoopConf(kyuubiConf)
|
||||
assert(StreamSupport.stream(hadoopConf.spliterator(), false)
|
||||
.anyMatch(entry => entry.getKey.startsWith("hadoop") || entry.getKey.startsWith("fs")))
|
||||
|
||||
hadoopConf = KyuubiHadoopUtils.newHadoopConf(kyuubiConf, loadDefaults = false)
|
||||
assert(StreamSupport.stream(hadoopConf.spliterator(), false)
|
||||
.noneMatch(entry => entry.getKey.startsWith("hadoop") || entry.getKey.startsWith("fs")))
|
||||
}
|
||||
}
|
||||
|
||||
@ -31,6 +31,7 @@ import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod
|
||||
import org.apache.kyuubi.Logging
|
||||
import org.apache.kyuubi.config.KyuubiConf
|
||||
import org.apache.kyuubi.credentials.HadoopFsDelegationTokenProvider.{disableFsCache, doAsProxyUser}
|
||||
import org.apache.kyuubi.util.KyuubiHadoopUtils
|
||||
|
||||
class HadoopFsDelegationTokenProvider extends HadoopDelegationTokenProvider with Logging {
|
||||
|
||||
@ -80,7 +81,10 @@ object HadoopFsDelegationTokenProvider {
|
||||
|
||||
def disableFsCache(kyuubiConf: KyuubiConf, hadoopConf: Configuration): Configuration = {
|
||||
// Avoid unnecessary disk io by not loading default resources
|
||||
val newConf = new Configuration(false)
|
||||
val newConf = KyuubiHadoopUtils.newHadoopConf(
|
||||
kyuubiConf,
|
||||
loadDefaults = false)
|
||||
|
||||
hadoopConf.iterator().asScala.foreach(e => newConf.set(e.getKey, e.getValue))
|
||||
|
||||
hadoopFSsToAccess(kyuubiConf, hadoopConf)
|
||||
|
||||
@ -19,8 +19,6 @@ package org.apache.kyuubi.server
|
||||
|
||||
import java.net.InetAddress
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
|
||||
import org.apache.kyuubi.config.KyuubiConf
|
||||
import org.apache.kyuubi.config.KyuubiConf.SERVER_EVENT_JSON_LOG_PATH
|
||||
import org.apache.kyuubi.config.KyuubiConf.SERVER_EVENT_LOGGERS
|
||||
@ -33,6 +31,7 @@ import org.apache.kyuubi.util.KyuubiHadoopUtils
|
||||
class EventLoggingService extends AbstractEventLoggingService[KyuubiServerEvent] {
|
||||
|
||||
override def initialize(conf: KyuubiConf): Unit = {
|
||||
val hadoopConf = KyuubiHadoopUtils.newHadoopConf(conf)
|
||||
conf.get(SERVER_EVENT_LOGGERS)
|
||||
.map(EventLoggerType.withName)
|
||||
.foreach {
|
||||
@ -41,9 +40,9 @@ class EventLoggingService extends AbstractEventLoggingService[KyuubiServerEvent]
|
||||
val jsonEventLogger = new JsonEventLogger[KyuubiServerEvent](
|
||||
s"server-$hostName",
|
||||
SERVER_EVENT_JSON_LOG_PATH,
|
||||
new Configuration())
|
||||
hadoopConf)
|
||||
// TODO: #1180 kyuubiServerEvent need create logRoot automatically
|
||||
jsonEventLogger.createEventLogRootDir(conf, KyuubiHadoopUtils.newHadoopConf(conf))
|
||||
jsonEventLogger.createEventLogRootDir(conf, hadoopConf)
|
||||
addService(jsonEventLogger)
|
||||
addEventLogger(jsonEventLogger)
|
||||
case logger =>
|
||||
|
||||
Loading…
Reference in New Issue
Block a user