aboutsummaryrefslogtreecommitdiff
path: root/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProviderSuite.scala
blob: 0eb25127238c65d8b87bf0f439d92d7d8a52c2ee (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
/*
 * 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.deploy.yarn.security

import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.scalatest.{Matchers, PrivateMethodTester}

import org.apache.spark.{SparkConf, SparkException, SparkFunSuite}

class HadoopFSCredentialProviderSuite
    extends SparkFunSuite
    with PrivateMethodTester
    with Matchers {
  private val _getTokenRenewer = PrivateMethod[String]('getTokenRenewer)

  private def getTokenRenewer(
      fsCredentialProvider: HadoopFSCredentialProvider, conf: Configuration): String = {
    fsCredentialProvider invokePrivate _getTokenRenewer(conf)
  }

  private var hadoopFsCredentialProvider: HadoopFSCredentialProvider = null

  override def beforeAll() {
    super.beforeAll()

    if (hadoopFsCredentialProvider == null) {
      hadoopFsCredentialProvider = new HadoopFSCredentialProvider()
    }
  }

  override def afterAll() {
    if (hadoopFsCredentialProvider != null) {
      hadoopFsCredentialProvider = null
    }

    super.afterAll()
  }

  test("check token renewer") {
    val hadoopConf = new Configuration()
    hadoopConf.set("yarn.resourcemanager.address", "myrm:8033")
    hadoopConf.set("yarn.resourcemanager.principal", "yarn/myrm:8032@SPARKTEST.COM")
    val renewer = getTokenRenewer(hadoopFsCredentialProvider, hadoopConf)
    renewer should be ("yarn/myrm:8032@SPARKTEST.COM")
  }

  test("check token renewer default") {
    val hadoopConf = new Configuration()
    val caught =
      intercept[SparkException] {
        getTokenRenewer(hadoopFsCredentialProvider, hadoopConf)
      }
    assert(caught.getMessage === "Can't get Master Kerberos principal for use as renewer")
  }
}