From 7abb5ee4743b0fe1507e6a7c9bb73da081652599 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Sat, 19 Aug 2017 15:13:37 -0700 Subject: [PATCH 01/43] Add skeleton --- resource-managers/kubernetes/core/pom.xml | 5 + .../kubernetes/HadoopTokenRenewServer.scala | 57 ++++++++ .../security/kubernetes/SecretFinder.scala | 80 +++++++++++ .../security/kubernetes/SecretWatcher.scala | 23 ++++ .../kubernetes/TokenRenewService.scala | 125 ++++++++++++++++++ .../spark/security/kubernetes/constants.scala | 30 +++++ .../hadoop-token-renew-server/Dockerfile | 25 ++++ 7 files changed, 345 insertions(+) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/HadoopTokenRenewServer.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretWatcher.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRenewService.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/constants.scala create mode 100644 resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/hadoop-token-renew-server/Dockerfile diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 01c94670b807a..7563809c80484 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -76,6 +76,11 @@ jackson-dataformat-yaml ${fasterxml.jackson.version} + + com.typesafe.akka + akka-actor_${scala.binary.version} + 2.5.4 + org.glassfish.jersey.containers jersey-container-servlet diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/HadoopTokenRenewServer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/HadoopTokenRenewServer.scala new file mode 100644 index 0000000000000..954b3d32051c1 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/HadoopTokenRenewServer.scala @@ -0,0 +1,57 @@ +/* + * 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.security.kubernetes + +import akka.actor.ActorSystem + +import scala.concurrent.duration.Duration +import scala.concurrent.Await + +private class Server { + + private val actorSystem = ActorSystem("HadoopTokenRenewServer") + private var secretFinder : SecretFinder = _ + + def start(): Unit = { + val renewService = TokenRenewService(actorSystem) + secretFinder = SecretFinder(renewService) + } + + def join() : Unit = { + // scalastyle:off awaitready + Await.ready(actorSystem.whenTerminated, Duration.Inf) + // scalastyle:on awaitready + } + + def stop(): Unit = { + actorSystem.terminate() + secretFinder.stop() + } +} + +object HadoopTokenRenewServer { + + def main(args: Array[String]): Unit = { + val server = new Server + server.start() + try { + server.join() + } finally { + server.stop() + } + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala new file mode 100644 index 0000000000000..01ef2efc7a24d --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala @@ -0,0 +1,80 @@ +/* + * 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.security.kubernetes + +import java.util.{Timer, TimerTask} + +import scala.collection.JavaConverters._ + +import akka.actor.ActorRef +import io.fabric8.kubernetes.api.model.Secret +import io.fabric8.kubernetes.client.{DefaultKubernetesClient, KubernetesClient, KubernetesClientException, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action + +import org.apache.spark.security.kubernetes.constants._ + +private class SecretFinder(renewService: ActorRef, + timer: Timer, + kubernetesClient: KubernetesClient) { + + timer.schedule(new SecretScanner(renewService, kubernetesClient), 1000L, 1000L) + kubernetesClient + .secrets() + .withLabel(HADOOP_DELEGATION_TOKEN_LABEL_IN_SECRET) + .watch(new SecretWatcher(renewService)) + + def stop(): Unit = { + timer.cancel() + kubernetesClient.close() + } +} + +private class SecretScanner(renewService: ActorRef, + kubernetesClient: KubernetesClient) extends TimerTask { + + override def run(): Unit = { + val secrets = kubernetesClient + .secrets + .withLabel(HADOOP_DELEGATION_TOKEN_LABEL_IN_SECRET) + renewService ! AllSecrets(secrets.list.getItems.asScala.toList) + } +} + +private class SecretWatcher(renewService: ActorRef) extends Watcher[Secret] { + + override def eventReceived(action: Action, secret: Secret): Unit = { + action match { + case Action.ADDED => + renewService ! AddedSecret(secret) + case Action.DELETED => + renewService ! DeletedSecret(secret) + } + } + + override def onClose(cause: KubernetesClientException): Unit = { + // FIXME. TBD. + } +} + +private object SecretFinder { + + def apply(renewService: ActorRef) : SecretFinder = { + new SecretFinder(renewService, + new Timer(SECRET_FIND_THREAD_NAME, IS_DAEMON_THREAD), + new DefaultKubernetesClient) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretWatcher.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretWatcher.scala new file mode 100644 index 0000000000000..40e4f340af547 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretWatcher.scala @@ -0,0 +1,23 @@ +/* + * 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.security.kubernetes + +import akka.actor.ActorRef +import io.fabric8.kubernetes.api.model.Secret +import io.fabric8.kubernetes.client.Watcher.Action +import io.fabric8.kubernetes.client.{KubernetesClientException, Watcher} + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRenewService.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRenewService.scala new file mode 100644 index 0000000000000..dd0a4be876ffb --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRenewService.scala @@ -0,0 +1,125 @@ +/* + * 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.security.kubernetes + +import java.io.{ByteArrayInputStream, DataInputStream} +import java.util.concurrent.{Executors, ThreadFactory, TimeUnit} +import java.util.concurrent.atomic.AtomicBoolean + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import akka.actor.{Actor, ActorRef, ActorSystem, Props} +import io.fabric8.kubernetes.api.model.Secret +import org.apache.commons.codec.binary.Base64 +import org.apache.hadoop.security.Credentials +import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier +import org.apache.spark.security.kubernetes.constants._ + + +private class TokenRenewService extends Actor { + + private val scheduler = newScheduler() + private val taskBySecret = mutable.HashMap[String, RenewTask]() // Keyed by secret UID. + + def receive: PartialFunction[Any, Unit] = { + case AddedSecret(secret) => addTask(secret) + case DeletedSecret(secret) => removeTask(secret) + case AllScannedSecrets(secrets) => updateTaskSet(secrets) + case _ => + } + + private def newScheduler() = Executors.newScheduledThreadPool(TOKEN_RENEW_NUM_THREADS, + new ThreadFactory { + override def newThread(r: Runnable): Thread = { + val thread = new Thread(r, TOKEN_RENEW_THREAD_NAME) + thread.setDaemon(true) + thread + } + }) + + private def addTask(secret: Secret) = { + taskBySecret.getOrElseUpdate(uid(secret), { + val task = new RenewTask(secret) + scheduler.schedule(task, TOKEN_RENEW_TASK_INITIAL_DELAY_SEC, TimeUnit.SECONDS) + task + }) + } + + private def removeTask(secret: Secret) = { + val task = taskBySecret.remove(uid(secret)) + task.foreach(_.cancel()) + } + + private def updateTaskSet(currentSecrets: List[Secret]) = { + val secretByUid = currentSecrets.map(secret => (uid(secret), secret)).toMap + val currentUids = secretByUid.keySet + val priorUids = taskBySecret.keySet + val uidsToAdd = currentUids -- priorUids + uidsToAdd.foreach(uid => addTask(secretByUid(uid))) + val uidsToRemove = priorUids -- currentUids + uidsToRemove.foreach(uid => removeTask(secretByUid(uid))) + } + + private def uid(secret: Secret) = secret.getMetadata.getUid +} + +private class RenewTask(secret: Secret) extends Runnable { + + private val isCanceled = new AtomicBoolean + + override def run() : Unit = { + + } + + def cancel(): Unit = isCanceled.set(true) + + private def renewTokensInitially(secret: Secret) : Unit = { + val credentials = readCredentials(secret) + val tokens = credentials + .flatMap(_.getAllTokens.asScala) + .filter(_.decodeIdentifier.isInstanceOf[AbstractDelegationTokenIdentifier]) + tokens.foreach { + token => + val id = token.decodeIdentifier.asInstanceOf[AbstractDelegationTokenIdentifier] + id.getIssueDate + id.getMaxDate + } + } + + private def readCredentials(secret: Secret) = { + secret.getData.asScala + .filterKeys(_.startsWith(HADOOP_TOKEN_KEY_IN_SECRET_DATA)) + .toSeq.sortBy(_._1) + .lastOption + .map { + item => + val creds = new Credentials + creds.readTokenStorageStream( + new DataInputStream(new ByteArrayInputStream(Base64.decodeBase64(item._2)))) + creds + } + } +} + +private case class AllScannedSecrets(secrets: List[Secret]) +private case class AddedSecret(secret: Secret) +private case class DeletedSecret(secret: Secret) + +private object TokenRenewService { + + def apply(system: ActorSystem) : ActorRef = system.actorOf(Props[TokenRenewService]) +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/constants.scala new file mode 100644 index 0000000000000..6d60ded158af6 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/constants.scala @@ -0,0 +1,30 @@ +/* + * 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.security.kubernetes + +package object constants { + + val HADOOP_DELEGATION_TOKEN_LABEL_IN_SECRET = "hadoop.delegation.token" + val HADOOP_TOKEN_KEY_IN_SECRET_DATA = "hadoop-token" + + val TOKEN_RENEW_NUM_THREADS = 10 + val TOKEN_RENEW_THREAD_NAME = "token-renewer" + val TOKEN_RENEW_TASK_INITIAL_DELAY_SEC = 0 + + val SECRET_FIND_THREAD_NAME = "secret-finder" + val IS_DAEMON_THREAD = true +} diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/hadoop-token-renew-server/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/hadoop-token-renew-server/Dockerfile new file mode 100644 index 0000000000000..fd76818738fc7 --- /dev/null +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/hadoop-token-renew-server/Dockerfile @@ -0,0 +1,25 @@ +# +# 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. +# + +FROM spark-base + + +# If this docker file is being used in the context of building your images from a Spark distribution, the docker build +# command should be invoked from the top level directory of the Spark distribution. E.g.: +# docker build -t spark-resource-staging-server:latest -f dockerfiles/resource-staging-server/Dockerfile . + +ENTRYPOINT [ "/sbin/tini", "--", "bin/spark-class", "org.apache.spark.security.kubernetes.HadoopTokenRenewServer" ] From 88c0c033f4c28ba73d1a7431f744143fdda4dd80 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Mon, 21 Aug 2017 13:36:53 -0700 Subject: [PATCH 02/43] Renew part --- .../security/kubernetes/SecretFinder.scala | 6 +- .../security/kubernetes/SecretWatcher.scala | 23 --- ...wServer.scala => TokenRefreshServer.scala} | 12 +- .../kubernetes/TokenRefreshService.scala | 161 ++++++++++++++++++ .../kubernetes/TokenRenewService.scala | 125 -------------- .../spark/security/kubernetes/constants.scala | 3 +- 6 files changed, 172 insertions(+), 158 deletions(-) delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretWatcher.scala rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/{HadoopTokenRenewServer.scala => TokenRefreshServer.scala} (90%) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRenewService.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala index 01ef2efc7a24d..2c257420c51f5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala @@ -50,7 +50,7 @@ private class SecretScanner(renewService: ActorRef, val secrets = kubernetesClient .secrets .withLabel(HADOOP_DELEGATION_TOKEN_LABEL_IN_SECRET) - renewService ! AllSecrets(secrets.list.getItems.asScala.toList) + renewService ! UpdateRefreshList(secrets.list.getItems.asScala.toList) } } @@ -59,9 +59,9 @@ private class SecretWatcher(renewService: ActorRef) extends Watcher[Secret] { override def eventReceived(action: Action, secret: Secret): Unit = { action match { case Action.ADDED => - renewService ! AddedSecret(secret) + renewService ! StartRefresh(secret) case Action.DELETED => - renewService ! DeletedSecret(secret) + renewService ! StopRefresh(secret) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretWatcher.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretWatcher.scala deleted file mode 100644 index 40e4f340af547..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretWatcher.scala +++ /dev/null @@ -1,23 +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.security.kubernetes - -import akka.actor.ActorRef -import io.fabric8.kubernetes.api.model.Secret -import io.fabric8.kubernetes.client.Watcher.Action -import io.fabric8.kubernetes.client.{KubernetesClientException, Watcher} - diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/HadoopTokenRenewServer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala similarity index 90% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/HadoopTokenRenewServer.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala index 954b3d32051c1..941f8630babf2 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/HadoopTokenRenewServer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala @@ -16,18 +16,18 @@ */ package org.apache.spark.security.kubernetes -import akka.actor.ActorSystem - -import scala.concurrent.duration.Duration import scala.concurrent.Await +import scala.concurrent.duration.Duration + +import akka.actor.ActorSystem private class Server { - private val actorSystem = ActorSystem("HadoopTokenRenewServer") + private val actorSystem = ActorSystem("TokenRefreshServer") private var secretFinder : SecretFinder = _ def start(): Unit = { - val renewService = TokenRenewService(actorSystem) + val renewService = TokenRefreshService(actorSystem) secretFinder = SecretFinder(renewService) } @@ -43,7 +43,7 @@ private class Server { } } -object HadoopTokenRenewServer { +object TokenRefreshServer { def main(args: Array[String]): Unit = { val server = new Server diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala new file mode 100644 index 0000000000000..295a41c296ed4 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -0,0 +1,161 @@ +/* + * 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.security.kubernetes + +import java.io.{ByteArrayInputStream, DataInputStream} +import java.util.concurrent.{Executors, ScheduledFuture, ThreadFactory, TimeUnit} + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import akka.actor.{Actor, ActorRef, ActorSystem, Props} +import io.fabric8.kubernetes.api.model.Secret +import org.apache.commons.codec.binary.Base64 +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.security.Credentials +import org.apache.hadoop.security.token.Token + +import org.apache.spark.security.kubernetes.constants._ + +private class TokenRefreshService extends Actor { + + private val scheduler = newScheduler() + // Keyed by secret UID. + private val taskHandleBySecret = mutable.HashMap[String, ScheduledFuture[_]]() + private val hadoopConf = new Configuration + private val clock = new Clock + + def receive: PartialFunction[Any, Unit] = { + case StartRefresh(secret) => addStarterTask(secret) + case StopRefresh(secret) => removeRefreshTask(secret) + case UpdateRefreshList(secrets) => updateRefreshTaskSet(secrets) + case renew @ Renew(nextExpireTime, expireTimeByToken, secret) => scheduleRenewTask(renew) + case _ => + } + + private def newScheduler() = Executors.newScheduledThreadPool(TOKEN_RENEW_NUM_THREADS, + new ThreadFactory { + override def newThread(r: Runnable): Thread = { + val thread = new Thread(r, TOKEN_RENEW_THREAD_NAME) + thread.setDaemon(true) + thread + } + }) + + private def addStarterTask(secret: Secret) = { + taskHandleBySecret.getOrElseUpdate(uid(secret), { + val task = new StarterTask(secret, hadoopConf, self) + scheduler.schedule(task, TOKEN_RENEW_TASK_INITIAL_DELAY_MILLIS, TimeUnit.MILLISECONDS) + }) + } + + private def removeRefreshTask(secret: Secret) = { + val task = taskHandleBySecret.remove(uid(secret)) + task.foreach(_.cancel(true)) // Interrupt if running. + } + + private def updateRefreshTaskSet(currentSecrets: List[Secret]) = { + val secretByUid = currentSecrets.map(secret => (uid(secret), secret)).toMap + val currentUids = secretByUid.keySet + val priorUids = taskHandleBySecret.keySet + val uidsToAdd = currentUids -- priorUids + uidsToAdd.foreach(uid => addStarterTask(secretByUid(uid))) + val uidsToRemove = priorUids -- currentUids + uidsToRemove.foreach(uid => removeRefreshTask(secretByUid(uid))) + } + + private def scheduleRenewTask(renew: Renew) = { + val uid = uid(renew.secret) + if (taskHandleBySecret.get(uid).nonEmpty) { + val renewTime = math.min(0L, + renew.expireTime - TOKEN_RENEW_SCHEDULE_AHEAD_MILLIS - clock.nowInMillis()) + val task = new RenewTask(renew, hadoopConf, self) + taskHandleBySecret.put(uid, scheduler.schedule(task, renewTime, TimeUnit.MILLISECONDS)) + } + } + + private def uid(secret: Secret) = secret.getMetadata.getUid +} + +private class StarterTask(secret: Secret, hadoopConf: Configuration, refreshService: ActorRef) + extends Runnable { + + override def run() : Unit = { + val tokens = readHadoopTokens() + val expireTimeByToken = renewTokens(tokens) + val nextExpireTime = expireTimeByToken.values.min + refreshService ! Renew(nextExpireTime, expireTimeByToken, secret) + } + + private def readHadoopTokens() = { + val hadoopSecretData = secret.getData.asScala.filterKeys( + _.startsWith(HADOOP_TOKEN_KEY_IN_SECRET_DATA)) + val latestData = if (hadoopSecretData.nonEmpty) Some(hadoopSecretData.max._2) else None + val credentials = latestData.map { + data => + val creds = new Credentials + creds.readTokenStorageStream(new DataInputStream(new ByteArrayInputStream( + Base64.decodeBase64(data)))) + creds + } + val tokens = credentials.map { + creds => + creds.getAllTokens.asScala.toList + } + tokens.getOrElse(Nil) + } + + private def renewTokens(tokens: List[Token[_]]) = { + tokens.map(token => (token, token.renew(hadoopConf))).toMap + } +} + +private class RenewTask(renew: Renew, hadoopConf: Configuration, refreshService: ActorRef) + extends Runnable { + + override def run() : Unit = { + val deadline = renew.expireTime + TOKEN_RENEW_SCHEDULE_AHEAD_MILLIS + val newExpireTimeByToken = renew.expireTimeByToken.map { + item => + val token = item._1 + val expireTime = item._2 + val newExpireTime = if (expireTime <= deadline) { + token.renew(hadoopConf) + } else { + expireTime + } + (token, newExpireTime) + } + val nextExpireTime = newExpireTimeByToken.values.min + refreshService ! Renew(nextExpireTime, newExpireTimeByToken, renew.secret) + } +} + +private class Clock { + + def nowInMillis() : Long = System.currentTimeMillis() +} + +private case class UpdateRefreshList(secrets: List[Secret]) +private case class StartRefresh(secret: Secret) +private case class Renew(expireTime: Long, expireTimeByToken: Map[Token[_], Long], secret: Secret) +private case class StopRefresh(secret: Secret) + +private object TokenRefreshService { + + def apply(system: ActorSystem) : ActorRef = system.actorOf(Props[TokenRefreshService]) +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRenewService.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRenewService.scala deleted file mode 100644 index dd0a4be876ffb..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRenewService.scala +++ /dev/null @@ -1,125 +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.security.kubernetes - -import java.io.{ByteArrayInputStream, DataInputStream} -import java.util.concurrent.{Executors, ThreadFactory, TimeUnit} -import java.util.concurrent.atomic.AtomicBoolean - -import scala.collection.JavaConverters._ -import scala.collection.mutable -import akka.actor.{Actor, ActorRef, ActorSystem, Props} -import io.fabric8.kubernetes.api.model.Secret -import org.apache.commons.codec.binary.Base64 -import org.apache.hadoop.security.Credentials -import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier -import org.apache.spark.security.kubernetes.constants._ - - -private class TokenRenewService extends Actor { - - private val scheduler = newScheduler() - private val taskBySecret = mutable.HashMap[String, RenewTask]() // Keyed by secret UID. - - def receive: PartialFunction[Any, Unit] = { - case AddedSecret(secret) => addTask(secret) - case DeletedSecret(secret) => removeTask(secret) - case AllScannedSecrets(secrets) => updateTaskSet(secrets) - case _ => - } - - private def newScheduler() = Executors.newScheduledThreadPool(TOKEN_RENEW_NUM_THREADS, - new ThreadFactory { - override def newThread(r: Runnable): Thread = { - val thread = new Thread(r, TOKEN_RENEW_THREAD_NAME) - thread.setDaemon(true) - thread - } - }) - - private def addTask(secret: Secret) = { - taskBySecret.getOrElseUpdate(uid(secret), { - val task = new RenewTask(secret) - scheduler.schedule(task, TOKEN_RENEW_TASK_INITIAL_DELAY_SEC, TimeUnit.SECONDS) - task - }) - } - - private def removeTask(secret: Secret) = { - val task = taskBySecret.remove(uid(secret)) - task.foreach(_.cancel()) - } - - private def updateTaskSet(currentSecrets: List[Secret]) = { - val secretByUid = currentSecrets.map(secret => (uid(secret), secret)).toMap - val currentUids = secretByUid.keySet - val priorUids = taskBySecret.keySet - val uidsToAdd = currentUids -- priorUids - uidsToAdd.foreach(uid => addTask(secretByUid(uid))) - val uidsToRemove = priorUids -- currentUids - uidsToRemove.foreach(uid => removeTask(secretByUid(uid))) - } - - private def uid(secret: Secret) = secret.getMetadata.getUid -} - -private class RenewTask(secret: Secret) extends Runnable { - - private val isCanceled = new AtomicBoolean - - override def run() : Unit = { - - } - - def cancel(): Unit = isCanceled.set(true) - - private def renewTokensInitially(secret: Secret) : Unit = { - val credentials = readCredentials(secret) - val tokens = credentials - .flatMap(_.getAllTokens.asScala) - .filter(_.decodeIdentifier.isInstanceOf[AbstractDelegationTokenIdentifier]) - tokens.foreach { - token => - val id = token.decodeIdentifier.asInstanceOf[AbstractDelegationTokenIdentifier] - id.getIssueDate - id.getMaxDate - } - } - - private def readCredentials(secret: Secret) = { - secret.getData.asScala - .filterKeys(_.startsWith(HADOOP_TOKEN_KEY_IN_SECRET_DATA)) - .toSeq.sortBy(_._1) - .lastOption - .map { - item => - val creds = new Credentials - creds.readTokenStorageStream( - new DataInputStream(new ByteArrayInputStream(Base64.decodeBase64(item._2)))) - creds - } - } -} - -private case class AllScannedSecrets(secrets: List[Secret]) -private case class AddedSecret(secret: Secret) -private case class DeletedSecret(secret: Secret) - -private object TokenRenewService { - - def apply(system: ActorSystem) : ActorRef = system.actorOf(Props[TokenRenewService]) -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/constants.scala index 6d60ded158af6..e087d139ccd25 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/constants.scala @@ -23,7 +23,8 @@ package object constants { val TOKEN_RENEW_NUM_THREADS = 10 val TOKEN_RENEW_THREAD_NAME = "token-renewer" - val TOKEN_RENEW_TASK_INITIAL_DELAY_SEC = 0 + val TOKEN_RENEW_TASK_INITIAL_DELAY_MILLIS = 0L + val TOKEN_RENEW_SCHEDULE_AHEAD_MILLIS = 10000L val SECRET_FIND_THREAD_NAME = "secret-finder" val IS_DAEMON_THREAD = true From ca0b583861e6bd0cd9ef17cb37aa9b0a54498310 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Mon, 21 Aug 2017 14:57:14 -0700 Subject: [PATCH 03/43] Compile succeeds --- .../kubernetes/TokenRefreshService.scala | 25 +++++++++++-------- 1 file changed, 14 insertions(+), 11 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index 295a41c296ed4..c9ab91f747034 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -21,14 +21,12 @@ import java.util.concurrent.{Executors, ScheduledFuture, ThreadFactory, TimeUnit import scala.collection.JavaConverters._ import scala.collection.mutable - import akka.actor.{Actor, ActorRef, ActorSystem, Props} import io.fabric8.kubernetes.api.model.Secret import org.apache.commons.codec.binary.Base64 import org.apache.hadoop.conf.Configuration import org.apache.hadoop.security.Credentials -import org.apache.hadoop.security.token.Token - +import org.apache.hadoop.security.token.{Token, TokenIdentifier} import org.apache.spark.security.kubernetes.constants._ private class TokenRefreshService extends Actor { @@ -57,19 +55,19 @@ private class TokenRefreshService extends Actor { }) private def addStarterTask(secret: Secret) = { - taskHandleBySecret.getOrElseUpdate(uid(secret), { + taskHandleBySecret.getOrElseUpdate(getSecretUid(secret), { val task = new StarterTask(secret, hadoopConf, self) scheduler.schedule(task, TOKEN_RENEW_TASK_INITIAL_DELAY_MILLIS, TimeUnit.MILLISECONDS) }) } private def removeRefreshTask(secret: Secret) = { - val task = taskHandleBySecret.remove(uid(secret)) + val task = taskHandleBySecret.remove(getSecretUid(secret)) task.foreach(_.cancel(true)) // Interrupt if running. } private def updateRefreshTaskSet(currentSecrets: List[Secret]) = { - val secretByUid = currentSecrets.map(secret => (uid(secret), secret)).toMap + val secretByUid = currentSecrets.map(secret => (getSecretUid(secret), secret)).toMap val currentUids = secretByUid.keySet val priorUids = taskHandleBySecret.keySet val uidsToAdd = currentUids -- priorUids @@ -79,7 +77,7 @@ private class TokenRefreshService extends Actor { } private def scheduleRenewTask(renew: Renew) = { - val uid = uid(renew.secret) + val uid = getSecretUid(renew.secret) if (taskHandleBySecret.get(uid).nonEmpty) { val renewTime = math.min(0L, renew.expireTime - TOKEN_RENEW_SCHEDULE_AHEAD_MILLIS - clock.nowInMillis()) @@ -88,7 +86,7 @@ private class TokenRefreshService extends Actor { } } - private def uid(secret: Secret) = secret.getMetadata.getUid + private def getSecretUid(secret: Secret) = secret.getMetadata.getUid } private class StarterTask(secret: Secret, hadoopConf: Configuration, refreshService: ActorRef) @@ -119,7 +117,8 @@ private class StarterTask(secret: Secret, hadoopConf: Configuration, refreshServ tokens.getOrElse(Nil) } - private def renewTokens(tokens: List[Token[_]]) = { + private def renewTokens(tokens: List[Token[_ <: TokenIdentifier]]) + : Map[Token[_ <: TokenIdentifier], Long] = { tokens.map(token => (token, token.renew(hadoopConf))).toMap } } @@ -129,7 +128,8 @@ private class RenewTask(renew: Renew, hadoopConf: Configuration, refreshService: override def run() : Unit = { val deadline = renew.expireTime + TOKEN_RENEW_SCHEDULE_AHEAD_MILLIS - val newExpireTimeByToken = renew.expireTimeByToken.map { + val newExpireTimeByToken : Map[Token[_ <: TokenIdentifier], Long] = + renew.expireTimeByToken.map { item => val token = item._1 val expireTime = item._2 @@ -140,6 +140,7 @@ private class RenewTask(renew: Renew, hadoopConf: Configuration, refreshService: } (token, newExpireTime) } + .toMap val nextExpireTime = newExpireTimeByToken.values.min refreshService ! Renew(nextExpireTime, newExpireTimeByToken, renew.secret) } @@ -152,7 +153,9 @@ private class Clock { private case class UpdateRefreshList(secrets: List[Secret]) private case class StartRefresh(secret: Secret) -private case class Renew(expireTime: Long, expireTimeByToken: Map[Token[_], Long], secret: Secret) +private case class Renew(expireTime: Long, + expireTimeByToken: Map[Token[_ <: TokenIdentifier], Long], + secret: Secret) private case class StopRefresh(secret: Secret) private object TokenRefreshService { From 025e2ba717c44a85a428482184b43aa42b02ca20 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Mon, 21 Aug 2017 15:42:32 -0700 Subject: [PATCH 04/43] Login to kerberos --- .../security/kubernetes/SecretFinder.scala | 5 +++-- .../kubernetes/TokenRefreshService.scala | 21 ++++++++++++------- .../spark/security/kubernetes/constants.scala | 15 ++++++++----- 3 files changed, 27 insertions(+), 14 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala index 2c257420c51f5..2864d5e7921fd 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala @@ -31,7 +31,8 @@ private class SecretFinder(renewService: ActorRef, timer: Timer, kubernetesClient: KubernetesClient) { - timer.schedule(new SecretScanner(renewService, kubernetesClient), 1000L, 1000L) + timer.schedule(new SecretScanner(renewService, kubernetesClient), + SECRET_SCANNER_INITIAL_DELAY_MILLIS, SECRET_SCANNER_PERIOD_MILLIS) kubernetesClient .secrets() .withLabel(HADOOP_DELEGATION_TOKEN_LABEL_IN_SECRET) @@ -74,7 +75,7 @@ private object SecretFinder { def apply(renewService: ActorRef) : SecretFinder = { new SecretFinder(renewService, - new Timer(SECRET_FIND_THREAD_NAME, IS_DAEMON_THREAD), + new Timer(SECRET_SCANNER_THREAD_NAME, IS_DAEMON_THREAD), new DefaultKubernetesClient) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index c9ab91f747034..6b445e61d151c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -21,12 +21,14 @@ import java.util.concurrent.{Executors, ScheduledFuture, ThreadFactory, TimeUnit import scala.collection.JavaConverters._ import scala.collection.mutable + import akka.actor.{Actor, ActorRef, ActorSystem, Props} import io.fabric8.kubernetes.api.model.Secret import org.apache.commons.codec.binary.Base64 import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.security.Credentials +import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.security.token.{Token, TokenIdentifier} + import org.apache.spark.security.kubernetes.constants._ private class TokenRefreshService extends Actor { @@ -45,10 +47,10 @@ private class TokenRefreshService extends Actor { case _ => } - private def newScheduler() = Executors.newScheduledThreadPool(TOKEN_RENEW_NUM_THREADS, + private def newScheduler() = Executors.newScheduledThreadPool(REFERSH_TASKS_NUM_THREADS, new ThreadFactory { override def newThread(r: Runnable): Thread = { - val thread = new Thread(r, TOKEN_RENEW_THREAD_NAME) + val thread = new Thread(r, REFRESH_TASK_THREAD_NAME) thread.setDaemon(true) thread } @@ -57,7 +59,7 @@ private class TokenRefreshService extends Actor { private def addStarterTask(secret: Secret) = { taskHandleBySecret.getOrElseUpdate(getSecretUid(secret), { val task = new StarterTask(secret, hadoopConf, self) - scheduler.schedule(task, TOKEN_RENEW_TASK_INITIAL_DELAY_MILLIS, TimeUnit.MILLISECONDS) + scheduler.schedule(task, REFRESH_STARTER_TASK_INITIAL_DELAY_MILLIS, TimeUnit.MILLISECONDS) }) } @@ -80,7 +82,7 @@ private class TokenRefreshService extends Actor { val uid = getSecretUid(renew.secret) if (taskHandleBySecret.get(uid).nonEmpty) { val renewTime = math.min(0L, - renew.expireTime - TOKEN_RENEW_SCHEDULE_AHEAD_MILLIS - clock.nowInMillis()) + renew.expireTime - RENEW_TASK_SCHEDULE_AHEAD_MILLIS - clock.nowInMillis()) val task = new RenewTask(renew, hadoopConf, self) taskHandleBySecret.put(uid, scheduler.schedule(task, renewTime, TimeUnit.MILLISECONDS)) } @@ -127,7 +129,7 @@ private class RenewTask(renew: Renew, hadoopConf: Configuration, refreshService: extends Runnable { override def run() : Unit = { - val deadline = renew.expireTime + TOKEN_RENEW_SCHEDULE_AHEAD_MILLIS + val deadline = renew.expireTime + RENEW_TASK_SCHEDULE_AHEAD_MILLIS val newExpireTimeByToken : Map[Token[_ <: TokenIdentifier], Long] = renew.expireTimeByToken.map { item => @@ -160,5 +162,10 @@ private case class StopRefresh(secret: Secret) private object TokenRefreshService { - def apply(system: ActorSystem) : ActorRef = system.actorOf(Props[TokenRefreshService]) + def apply(system: ActorSystem) : ActorRef = { + UserGroupInformation.loginUserFromKeytab( + REFRESH_SERVER_KERBEROS_PRINCIPAL, + REFRESH_SERVER_KERBEROS_KEYTAB_PATH) + system.actorOf(Props[TokenRefreshService]) + } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/constants.scala index e087d139ccd25..5c01a4ba985e1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/constants.scala @@ -18,14 +18,19 @@ package org.apache.spark.security.kubernetes package object constants { + val REFRESH_SERVER_KERBEROS_PRINCIPAL = "kimoonkim" + val REFRESH_SERVER_KERBEROS_KEYTAB_PATH = "/etc/security/krb5.keytab" + val HADOOP_DELEGATION_TOKEN_LABEL_IN_SECRET = "hadoop.delegation.token" val HADOOP_TOKEN_KEY_IN_SECRET_DATA = "hadoop-token" - val TOKEN_RENEW_NUM_THREADS = 10 - val TOKEN_RENEW_THREAD_NAME = "token-renewer" - val TOKEN_RENEW_TASK_INITIAL_DELAY_MILLIS = 0L - val TOKEN_RENEW_SCHEDULE_AHEAD_MILLIS = 10000L + val REFERSH_TASKS_NUM_THREADS = 10 + val REFRESH_TASK_THREAD_NAME = "token-renewer" + val REFRESH_STARTER_TASK_INITIAL_DELAY_MILLIS = 0L + val RENEW_TASK_SCHEDULE_AHEAD_MILLIS = 10000L - val SECRET_FIND_THREAD_NAME = "secret-finder" + val SECRET_SCANNER_THREAD_NAME = "secret-scanner" + val SECRET_SCANNER_INITIAL_DELAY_MILLIS = 10 * 1000L + val SECRET_SCANNER_PERIOD_MILLIS = 60 * 60 * 1000L val IS_DAEMON_THREAD = true } From 0a7a15d2d4fcb020248cd149021c66ea6da53104 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Tue, 22 Aug 2017 10:15:43 -0700 Subject: [PATCH 05/43] Clean up constants --- .../org/apache/spark/security/kubernetes/SecretFinder.scala | 4 ++-- .../spark/security/kubernetes/TokenRefreshService.scala | 2 +- .../org/apache/spark/security/kubernetes/constants.scala | 5 +++-- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala index 2864d5e7921fd..f0545f404c0a9 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala @@ -35,7 +35,7 @@ private class SecretFinder(renewService: ActorRef, SECRET_SCANNER_INITIAL_DELAY_MILLIS, SECRET_SCANNER_PERIOD_MILLIS) kubernetesClient .secrets() - .withLabel(HADOOP_DELEGATION_TOKEN_LABEL_IN_SECRET) + .withLabel(SECRET_LABEL_KEY_REFRESH_HADOOP_TOKENS, SECRET_LABEL_VALUE_REFRESH_HADOOP_TOKENS) .watch(new SecretWatcher(renewService)) def stop(): Unit = { @@ -50,7 +50,7 @@ private class SecretScanner(renewService: ActorRef, override def run(): Unit = { val secrets = kubernetesClient .secrets - .withLabel(HADOOP_DELEGATION_TOKEN_LABEL_IN_SECRET) + .withLabel(SECRET_LABEL_KEY_REFRESH_HADOOP_TOKENS) renewService ! UpdateRefreshList(secrets.list.getItems.asScala.toList) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index 6b445e61d151c..8e6941ff7733d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -103,7 +103,7 @@ private class StarterTask(secret: Secret, hadoopConf: Configuration, refreshServ private def readHadoopTokens() = { val hadoopSecretData = secret.getData.asScala.filterKeys( - _.startsWith(HADOOP_TOKEN_KEY_IN_SECRET_DATA)) + _.startsWith(SECRET_DATA_KEY_PREFIX_HADOOP_TOKENS)) val latestData = if (hadoopSecretData.nonEmpty) Some(hadoopSecretData.max._2) else None val credentials = latestData.map { data => diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/constants.scala index 5c01a4ba985e1..8715656390318 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/constants.scala @@ -21,8 +21,9 @@ package object constants { val REFRESH_SERVER_KERBEROS_PRINCIPAL = "kimoonkim" val REFRESH_SERVER_KERBEROS_KEYTAB_PATH = "/etc/security/krb5.keytab" - val HADOOP_DELEGATION_TOKEN_LABEL_IN_SECRET = "hadoop.delegation.token" - val HADOOP_TOKEN_KEY_IN_SECRET_DATA = "hadoop-token" + val SECRET_LABEL_KEY_REFRESH_HADOOP_TOKENS = "refresh-hadoop-tokens" + val SECRET_LABEL_VALUE_REFRESH_HADOOP_TOKENS = "yes" + val SECRET_DATA_KEY_PREFIX_HADOOP_TOKENS = "hadoop-token-" val REFERSH_TASKS_NUM_THREADS = 10 val REFRESH_TASK_THREAD_NAME = "token-renewer" From b3534f107b0b9a67d296289b28a84a352546fcdd Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Tue, 22 Aug 2017 15:57:42 -0700 Subject: [PATCH 06/43] Refresh server works --- .../security/kubernetes/SecretFinder.scala | 32 +++-- .../kubernetes/TokenRefreshServer.scala | 11 +- .../kubernetes/TokenRefreshService.scala | 116 ++++++++++++++---- .../spark/security/kubernetes/constants.scala | 6 +- 4 files changed, 123 insertions(+), 42 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala index f0545f404c0a9..e17da5469ad3b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala @@ -16,15 +16,18 @@ */ package org.apache.spark.security.kubernetes +import java.lang import java.util.{Timer, TimerTask} import scala.collection.JavaConverters._ import akka.actor.ActorRef -import io.fabric8.kubernetes.api.model.Secret -import io.fabric8.kubernetes.client.{DefaultKubernetesClient, KubernetesClient, KubernetesClientException, Watcher} +import io.fabric8.kubernetes.api.model.{Secret, SecretList} +import io.fabric8.kubernetes.client._ import io.fabric8.kubernetes.client.Watcher.Action +import io.fabric8.kubernetes.client.dsl.FilterWatchListDeletable +import org.apache.spark.internal.Logging import org.apache.spark.security.kubernetes.constants._ private class SecretFinder(renewService: ActorRef, @@ -33,9 +36,7 @@ private class SecretFinder(renewService: ActorRef, timer.schedule(new SecretScanner(renewService, kubernetesClient), SECRET_SCANNER_INITIAL_DELAY_MILLIS, SECRET_SCANNER_PERIOD_MILLIS) - kubernetesClient - .secrets() - .withLabel(SECRET_LABEL_KEY_REFRESH_HADOOP_TOKENS, SECRET_LABEL_VALUE_REFRESH_HADOOP_TOKENS) + SecretFinder.selectSecrets(kubernetesClient) .watch(new SecretWatcher(renewService)) def stop(): Unit = { @@ -45,23 +46,24 @@ private class SecretFinder(renewService: ActorRef, } private class SecretScanner(renewService: ActorRef, - kubernetesClient: KubernetesClient) extends TimerTask { + kubernetesClient: KubernetesClient) extends TimerTask with Logging { override def run(): Unit = { - val secrets = kubernetesClient - .secrets - .withLabel(SECRET_LABEL_KEY_REFRESH_HADOOP_TOKENS) - renewService ! UpdateRefreshList(secrets.list.getItems.asScala.toList) + val secrets = SecretFinder.selectSecrets(kubernetesClient).list.getItems.asScala.toList + logInfo(s"Scanned ${secrets.map(_.getMetadata.getSelfLink).mkString}") + renewService ! UpdateRefreshList(secrets) } } -private class SecretWatcher(renewService: ActorRef) extends Watcher[Secret] { +private class SecretWatcher(renewService: ActorRef) extends Watcher[Secret] with Logging { override def eventReceived(action: Action, secret: Secret): Unit = { action match { case Action.ADDED => + logInfo(s"Found ${secret.getMetadata.getSelfLink} added") renewService ! StartRefresh(secret) case Action.DELETED => + logInfo(s"Found ${secret.getMetadata.getSelfLink} deleted") renewService ! StopRefresh(secret) } } @@ -78,4 +80,12 @@ private object SecretFinder { new Timer(SECRET_SCANNER_THREAD_NAME, IS_DAEMON_THREAD), new DefaultKubernetesClient) } + + def selectSecrets(kubernetesClient: KubernetesClient): + FilterWatchListDeletable[Secret, SecretList, lang.Boolean, Watch, Watcher[Secret]] = { + kubernetesClient + .secrets() + .inAnyNamespace() + .withLabel(SECRET_LABEL_KEY_REFRESH_HADOOP_TOKENS, SECRET_LABEL_VALUE_REFRESH_HADOOP_TOKENS) + } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala index 941f8630babf2..cb61bb851ea25 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala @@ -18,17 +18,17 @@ package org.apache.spark.security.kubernetes import scala.concurrent.Await import scala.concurrent.duration.Duration - import akka.actor.ActorSystem +import org.apache.log4j.{Level, Logger} private class Server { private val actorSystem = ActorSystem("TokenRefreshServer") - private var secretFinder : SecretFinder = _ + private var secretFinder : Option[SecretFinder] = None def start(): Unit = { val renewService = TokenRefreshService(actorSystem) - secretFinder = SecretFinder(renewService) + secretFinder = Some(SecretFinder(renewService)) } def join() : Unit = { @@ -39,16 +39,17 @@ private class Server { def stop(): Unit = { actorSystem.terminate() - secretFinder.stop() + secretFinder.foreach(_.stop()) } } object TokenRefreshServer { def main(args: Array[String]): Unit = { + Logger.getRootLogger.setLevel(Level.INFO) val server = new Server - server.start() try { + server.start() server.join() } finally { server.stop() diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index 8e6941ff7733d..32558d5339a83 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -29,9 +29,10 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.security.token.{Token, TokenIdentifier} +import org.apache.spark.internal.Logging import org.apache.spark.security.kubernetes.constants._ -private class TokenRefreshService extends Actor { +private class TokenRefreshService extends Actor with Logging { private val scheduler = newScheduler() // Keyed by secret UID. @@ -39,11 +40,11 @@ private class TokenRefreshService extends Actor { private val hadoopConf = new Configuration private val clock = new Clock - def receive: PartialFunction[Any, Unit] = { + override def receive: PartialFunction[Any, Unit] = { case StartRefresh(secret) => addStarterTask(secret) case StopRefresh(secret) => removeRefreshTask(secret) case UpdateRefreshList(secrets) => updateRefreshTaskSet(secrets) - case renew @ Renew(nextExpireTime, expireTimeByToken, secret) => scheduleRenewTask(renew) + case renew @ Renew(nextExpireTime, expireTimeByToken, secret, _) => scheduleRenewTask(renew) case _ => } @@ -58,14 +59,20 @@ private class TokenRefreshService extends Actor { private def addStarterTask(secret: Secret) = { taskHandleBySecret.getOrElseUpdate(getSecretUid(secret), { - val task = new StarterTask(secret, hadoopConf, self) - scheduler.schedule(task, REFRESH_STARTER_TASK_INITIAL_DELAY_MILLIS, TimeUnit.MILLISECONDS) + val task = new StarterTask(secret, hadoopConf, self, clock) + val future = scheduler.schedule(task, REFRESH_STARTER_TASK_INITIAL_DELAY_MILLIS, + TimeUnit.MILLISECONDS) + logInfo(s"Started refresh of tokens in ${secret.getMetadata.getSelfLink} with ${future}") + future }) } private def removeRefreshTask(secret: Secret) = { - val task = taskHandleBySecret.remove(getSecretUid(secret)) - task.foreach(_.cancel(true)) // Interrupt if running. + val uid = getSecretUid(secret) + taskHandleBySecret.remove(uid).foreach(future => { + logInfo(s"Canceling refresh of tokens in ${secret.getMetadata.getSelfLink}") + future.cancel(true) // Interrupt if running. + }) } private def updateRefreshTaskSet(currentSecrets: List[Secret]) = { @@ -81,24 +88,49 @@ private class TokenRefreshService extends Actor { private def scheduleRenewTask(renew: Renew) = { val uid = getSecretUid(renew.secret) if (taskHandleBySecret.get(uid).nonEmpty) { - val renewTime = math.min(0L, - renew.expireTime - RENEW_TASK_SCHEDULE_AHEAD_MILLIS - clock.nowInMillis()) - val task = new RenewTask(renew, hadoopConf, self) - taskHandleBySecret.put(uid, scheduler.schedule(task, renewTime, TimeUnit.MILLISECONDS)) + val numConsecutiveErrors = renew.numConsecutiveErrors + if (numConsecutiveErrors < RENEW_TASK_MAX_CONSECUTIVE_ERRORS) { + val renewTime = math.max(0L, + renew.expireTime - RENEW_TASK_SCHEDULE_AHEAD_MILLIS - clock.nowInMillis()) + val task = new RenewTask(renew, hadoopConf, self, clock) + logInfo(s"Scheduling refresh of tokens with " + + s"${renew.secret.getMetadata.getSelfLink} at now + $renewTime millis.") + taskHandleBySecret.put(uid, scheduler.schedule(task, renewTime, TimeUnit.MILLISECONDS)) + } else { + logWarning(s"Got too many errors for ${renew.secret.getMetadata.getSelfLink}. Abandoning.") + val future = taskHandleBySecret.remove(uid) + future.foreach(_.cancel(true)) // Interrupt if running. + } + } else { + logWarning(s"Could not find an entry for renew task" + + s" ${renew.secret.getMetadata.getSelfLink}. Maybe the secret got deleted") } } private def getSecretUid(secret: Secret) = secret.getMetadata.getUid } -private class StarterTask(secret: Secret, hadoopConf: Configuration, refreshService: ActorRef) - extends Runnable { +private class StarterTask(secret: Secret, + hadoopConf: Configuration, + refreshService: ActorRef, + clock: Clock) extends Runnable with Logging { + + private var hasError = false override def run() : Unit = { val tokens = readHadoopTokens() + logInfo(s"Read Hadoop tokens: $tokens") val expireTimeByToken = renewTokens(tokens) - val nextExpireTime = expireTimeByToken.values.min - refreshService ! Renew(nextExpireTime, expireTimeByToken, secret) + val nextExpireTime = if (expireTimeByToken.nonEmpty) { + expireTimeByToken.values.min + } else { + logWarning(s"Got an empty token list with ${secret.getMetadata.getSelfLink}") + hasError = true + getRetryTime() + } + logInfo(s"Initial renew resulted with $expireTimeByToken. Next expire time $nextExpireTime") + val numConsecutiveErrors = if (hasError) 1 else 0 + refreshService ! Renew(nextExpireTime, expireTimeByToken, secret, numConsecutiveErrors) } private def readHadoopTokens() = { @@ -121,31 +153,66 @@ private class StarterTask(secret: Secret, hadoopConf: Configuration, refreshServ private def renewTokens(tokens: List[Token[_ <: TokenIdentifier]]) : Map[Token[_ <: TokenIdentifier], Long] = { - tokens.map(token => (token, token.renew(hadoopConf))).toMap + tokens.map(token => { + val expireTime = try { + token.renew(hadoopConf) + } catch { + case t: Throwable => + logWarning(t.getMessage, t) + hasError = true + + getRetryTime() + } + (token, expireTime) + }).toMap } + + private def getRetryTime() = clock.nowInMillis() + RENEW_TASK_DEADLINE_LOOK_AHEAD_MILLIS } -private class RenewTask(renew: Renew, hadoopConf: Configuration, refreshService: ActorRef) - extends Runnable { +private class RenewTask(renew: Renew, + hadoopConf: Configuration, + refreshService: ActorRef, + clock: Clock) extends Runnable with Logging { + + private var hasError = false override def run() : Unit = { - val deadline = renew.expireTime + RENEW_TASK_SCHEDULE_AHEAD_MILLIS + val deadline = renew.expireTime + RENEW_TASK_DEADLINE_LOOK_AHEAD_MILLIS val newExpireTimeByToken : Map[Token[_ <: TokenIdentifier], Long] = renew.expireTimeByToken.map { item => val token = item._1 val expireTime = item._2 - val newExpireTime = if (expireTime <= deadline) { - token.renew(hadoopConf) + val newExpireTime = + if (expireTime <= deadline) { + try { + token.renew(hadoopConf) + } catch { + case t: Throwable => + logWarning(t.getMessage, t) + hasError = true + + getRetryTime() + } } else { expireTime } (token, newExpireTime) } .toMap - val nextExpireTime = newExpireTimeByToken.values.min - refreshService ! Renew(nextExpireTime, newExpireTimeByToken, renew.secret) + if (newExpireTimeByToken.nonEmpty) { + val nextExpireTime = newExpireTimeByToken.values.min + logInfo(s"Renewed with the result $newExpireTimeByToken. Next expire time $nextExpireTime") + val numConsecutiveErrors = if (hasError) renew.numConsecutiveErrors + 1 else 0 + refreshService ! Renew(nextExpireTime, newExpireTimeByToken, renew.secret, + numConsecutiveErrors) + } else { + logWarning(s"Got an empty token list with ${renew.secret.getMetadata.getSelfLink}") + } } + + private def getRetryTime() = clock.nowInMillis() + RENEW_TASK_DEADLINE_LOOK_AHEAD_MILLIS } private class Clock { @@ -157,7 +224,8 @@ private case class UpdateRefreshList(secrets: List[Secret]) private case class StartRefresh(secret: Secret) private case class Renew(expireTime: Long, expireTimeByToken: Map[Token[_ <: TokenIdentifier], Long], - secret: Secret) + secret: Secret, + numConsecutiveErrors: Int) private case class StopRefresh(secret: Secret) private object TokenRefreshService { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/constants.scala index 8715656390318..05a72af8e66f1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/constants.scala @@ -19,16 +19,18 @@ package org.apache.spark.security.kubernetes package object constants { val REFRESH_SERVER_KERBEROS_PRINCIPAL = "kimoonkim" - val REFRESH_SERVER_KERBEROS_KEYTAB_PATH = "/etc/security/krb5.keytab" + val REFRESH_SERVER_KERBEROS_KEYTAB_PATH = "/mnt/secrets/krb5.keytab" val SECRET_LABEL_KEY_REFRESH_HADOOP_TOKENS = "refresh-hadoop-tokens" val SECRET_LABEL_VALUE_REFRESH_HADOOP_TOKENS = "yes" - val SECRET_DATA_KEY_PREFIX_HADOOP_TOKENS = "hadoop-token-" + val SECRET_DATA_KEY_PREFIX_HADOOP_TOKENS = "hadoop-tokens-" val REFERSH_TASKS_NUM_THREADS = 10 val REFRESH_TASK_THREAD_NAME = "token-renewer" val REFRESH_STARTER_TASK_INITIAL_DELAY_MILLIS = 0L + val RENEW_TASK_DEADLINE_LOOK_AHEAD_MILLIS = 10000L val RENEW_TASK_SCHEDULE_AHEAD_MILLIS = 10000L + val RENEW_TASK_MAX_CONSECUTIVE_ERRORS = 3 val SECRET_SCANNER_THREAD_NAME = "secret-scanner" val SECRET_SCANNER_INITIAL_DELAY_MILLIS = 10 * 1000L From cbe2777053a949d63fb22908c01a568ae89ca898 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Tue, 22 Aug 2017 15:58:21 -0700 Subject: [PATCH 07/43] Deployment config file --- ...ubernetes-hadoop-token-refresh-server.yaml | 60 +++++++++++++++++++ 1 file changed, 60 insertions(+) create mode 100644 conf/kubernetes-hadoop-token-refresh-server.yaml diff --git a/conf/kubernetes-hadoop-token-refresh-server.yaml b/conf/kubernetes-hadoop-token-refresh-server.yaml new file mode 100644 index 0000000000000..a2ec0250385df --- /dev/null +++ b/conf/kubernetes-hadoop-token-refresh-server.yaml @@ -0,0 +1,60 @@ +# +# 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. +# +--- +apiVersion: extensions/v1beta1 +kind: Deployment +metadata: + name: hadoop-token-refresh-server +spec: + replicas: 1 + template: + metadata: + labels: + hadoop-token-refresh-server-instance: default + spec: + volumes: + - name: kerberos-config + configMap: + name: hadoop-token-refresh-server-kerberos-config + - name: hadoop-config + configMap: + name: hadoop-token-refresh-server-hadoop-config + - name: kerberos-keytab + secret: + name: hadoop-token-refresh-server-kerberos-keytab + containers: + - name: hadoop-token-refresh-server + image: docker:5000/hadoop-token-refresh-server:kimoon-0821-1 + resources: + requests: + cpu: 100m + memory: 256Mi + limits: + cpu: 100m + memory: 256Mi + volumeMounts: + - name: kerberos-config + mountPath: '/etc/krb5.conf' + subPath: krb5.conf + readOnly: true + - name: hadoop-config + mountPath: '/etc/hadoop/conf' + readOnly: true + - name: kereros-keytab + mountPath: '/mnt/secrets/krb5.keytab' + subPath: krb5.keytab + readOnly: true From 4f36793b3297a0b9b742a9af483d60039bb81034 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Tue, 22 Aug 2017 16:15:03 -0700 Subject: [PATCH 08/43] Fix Dockerfile to match names --- .../Dockerfile | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) rename resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/{hadoop-token-renew-server => hadoop-token-refresh-server}/Dockerfile (86%) diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/hadoop-token-renew-server/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/hadoop-token-refresh-server/Dockerfile similarity index 86% rename from resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/hadoop-token-renew-server/Dockerfile rename to resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/hadoop-token-refresh-server/Dockerfile index fd76818738fc7..4248310ee20d3 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/hadoop-token-renew-server/Dockerfile +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/hadoop-token-refresh-server/Dockerfile @@ -20,6 +20,6 @@ FROM spark-base # If this docker file is being used in the context of building your images from a Spark distribution, the docker build # command should be invoked from the top level directory of the Spark distribution. E.g.: -# docker build -t spark-resource-staging-server:latest -f dockerfiles/resource-staging-server/Dockerfile . +# docker build -t spark-hadoop-token-refresh-server:latest -f dockerfiles/hadoop-token-refresh-server/Dockerfile . -ENTRYPOINT [ "/sbin/tini", "--", "bin/spark-class", "org.apache.spark.security.kubernetes.HadoopTokenRenewServer" ] +ENTRYPOINT [ "/sbin/tini", "--", "bin/spark-class", "org.apache.spark.security.kubernetes.TokenRefreshServer" ] From 874b8e98aa0769ded82da7031bb204c3bc864715 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Mon, 11 Sep 2017 16:55:57 -0700 Subject: [PATCH 09/43] Add as independent project with own pom.xml --- resource-managers/kubernetes/core/pom.xml | 5 -- .../kubernetes/token-refresh-server/pom.xml | 82 +++++++++++++++++++ .../src/main/assembly/assembly.xml | 33 ++++++++ .../src/main/docker}/Dockerfile | 0 .../spark/security/kubernetes/Logging.scala | 19 +++++ .../security/kubernetes/SecretFinder.scala | 1 - .../kubernetes/TokenRefreshServer.scala | 1 + .../kubernetes/TokenRefreshService.scala | 4 +- .../spark/security/kubernetes/constants.scala | 0 9 files changed, 136 insertions(+), 9 deletions(-) create mode 100644 resource-managers/kubernetes/token-refresh-server/pom.xml create mode 100644 resource-managers/kubernetes/token-refresh-server/src/main/assembly/assembly.xml rename resource-managers/kubernetes/{docker-minimal-bundle/src/main/docker/hadoop-token-refresh-server => token-refresh-server/src/main/docker}/Dockerfile (100%) create mode 100644 resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/Logging.scala rename resource-managers/kubernetes/{core => token-refresh-server}/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala (98%) rename resource-managers/kubernetes/{core => token-refresh-server}/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala (99%) rename resource-managers/kubernetes/{core => token-refresh-server}/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala (99%) rename resource-managers/kubernetes/{core => token-refresh-server}/src/main/scala/org/apache/spark/security/kubernetes/constants.scala (100%) diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 7563809c80484..01c94670b807a 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -76,11 +76,6 @@ jackson-dataformat-yaml ${fasterxml.jackson.version} - - com.typesafe.akka - akka-actor_${scala.binary.version} - 2.5.4 - org.glassfish.jersey.containers jersey-container-servlet diff --git a/resource-managers/kubernetes/token-refresh-server/pom.xml b/resource-managers/kubernetes/token-refresh-server/pom.xml new file mode 100644 index 0000000000000..3c53d447ab717 --- /dev/null +++ b/resource-managers/kubernetes/token-refresh-server/pom.xml @@ -0,0 +1,82 @@ + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.2.0-k8s-0.3.0-SNAPSHOT + ../../../pom.xml + + + token-refresh-server-kubernetes_2.11 + jar + Hadoop Token Refresh Server on Kubernetes + + 2.5.4 + 1.2 + 2.2.13 + + + + com.typesafe.akka + akka-actor_${scala.binary.version} + ${akka.actor.version} + + + io.fabric8 + kubernetes-client + ${kubernetes.client.version} + + + log4j + log4j + + + org.apache.hadoop + hadoop-client + + + commons-logging + commons-logging + ${commons-logging.version} + + + + + + org.apache.maven.plugins + maven-assembly-plugin + + + src/main/assembly/assembly.xml + + + + + make-assembly + package + + single + + + + + + + diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/assembly/assembly.xml b/resource-managers/kubernetes/token-refresh-server/src/main/assembly/assembly.xml new file mode 100644 index 0000000000000..920ff527b0dd4 --- /dev/null +++ b/resource-managers/kubernetes/token-refresh-server/src/main/assembly/assembly.xml @@ -0,0 +1,33 @@ + + + bin + + jar + + false + + + true + compile + + + true + provided + + + diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/hadoop-token-refresh-server/Dockerfile b/resource-managers/kubernetes/token-refresh-server/src/main/docker/Dockerfile similarity index 100% rename from resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/hadoop-token-refresh-server/Dockerfile rename to resource-managers/kubernetes/token-refresh-server/src/main/docker/Dockerfile diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/Logging.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/Logging.scala new file mode 100644 index 0000000000000..d34b592b86aa4 --- /dev/null +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/Logging.scala @@ -0,0 +1,19 @@ +package org.apache.spark.security.kubernetes + +import org.apache.log4j.{LogManager, Logger, Priority} + +trait Logging { + + private var log : Logger = LogManager.getLogger(this.getClass) + + protected def logDebug(msg: => String) = if (log.isDebugEnabled) log.debug(msg) + + protected def logInfo(msg: => String) = if (log.isInfoEnabled) log.info(msg) + + protected def logWarning(msg: => String) = if (log.isEnabledFor(Priority.WARN)) log.warn(msg) + + protected def logWarning(msg: => String, throwable: Throwable) = + if (log.isEnabledFor(Priority.WARN)) log.warn(msg, throwable) + + protected def logError(msg: => String) = if (log.isEnabledFor(Priority.ERROR)) log.error(msg) +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala similarity index 98% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala rename to resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala index e17da5469ad3b..26edf7b002187 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala @@ -27,7 +27,6 @@ import io.fabric8.kubernetes.client._ import io.fabric8.kubernetes.client.Watcher.Action import io.fabric8.kubernetes.client.dsl.FilterWatchListDeletable -import org.apache.spark.internal.Logging import org.apache.spark.security.kubernetes.constants._ private class SecretFinder(renewService: ActorRef, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala similarity index 99% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala rename to resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala index cb61bb851ea25..809a7c448da34 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala @@ -21,6 +21,7 @@ import scala.concurrent.duration.Duration import akka.actor.ActorSystem import org.apache.log4j.{Level, Logger} + private class Server { private val actorSystem = ActorSystem("TokenRefreshServer") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala similarity index 99% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala rename to resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index 32558d5339a83..1ba5e780fab80 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -21,15 +21,13 @@ import java.util.concurrent.{Executors, ScheduledFuture, ThreadFactory, TimeUnit import scala.collection.JavaConverters._ import scala.collection.mutable - import akka.actor.{Actor, ActorRef, ActorSystem, Props} + import io.fabric8.kubernetes.api.model.Secret import org.apache.commons.codec.binary.Base64 import org.apache.hadoop.conf.Configuration import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.security.token.{Token, TokenIdentifier} - -import org.apache.spark.internal.Logging import org.apache.spark.security.kubernetes.constants._ private class TokenRefreshService extends Actor with Logging { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/constants.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala similarity index 100% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/security/kubernetes/constants.scala rename to resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala From 5dc49ca0614b66916eef229d7596ff9bd051dc62 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Tue, 12 Sep 2017 14:49:18 -0700 Subject: [PATCH 10/43] Add working Dockerfile and deployment yaml file --- .../kubernetes/token-refresh-server/README.md | 61 +++++++++++++++++++ ...ubernetes-hadoop-token-refresh-server.yaml | 11 ++-- .../src/main/docker/Dockerfile | 26 ++++++-- 3 files changed, 88 insertions(+), 10 deletions(-) create mode 100644 resource-managers/kubernetes/token-refresh-server/README.md rename {conf => resource-managers/kubernetes/token-refresh-server/src/main/conf}/kubernetes-hadoop-token-refresh-server.yaml (88%) diff --git a/resource-managers/kubernetes/token-refresh-server/README.md b/resource-managers/kubernetes/token-refresh-server/README.md new file mode 100644 index 0000000000000..63515f145d1ef --- /dev/null +++ b/resource-managers/kubernetes/token-refresh-server/README.md @@ -0,0 +1,61 @@ +--- +layout: global +title: Hadoop Token Refresh Server on Kubernetes +--- + +Spark on Kubernetes may use Kerberized Hadoop data sources such as secure HDFS or Kafka. If the job +runs for days or weeks, someone should extend the lifetime of Hadoop delegation tokens, which +expire every 24 hours. The Hadoop Token Refresh Server is a Kubernetes microservice that renews +token lifetime and puts the replacement tokens in place. + +# Building the Refresh Server + +To build the refresh server jar, simply run Maven. For example: + + mvn clean package + +The target directory will have a uber jar that includes the project class files as well as +3rd party classes from the dependencies. The jar name would end with `-bin.jar`. For example: + + target/token-refresh-server-kubernetes_2.11-2.2.0-k8s-0.3.0-SNAPSHOT-bin.jar + +# Running the Refresh Server + +To run the server, follow the steps below. + +1. Build and push the docker image: + + docker build -t hadoop-token-refresh-server:latest \ + -f src/main/docker/Dockerfile . + docker tag hadoop-token-refresh-server:latest : + docker push : + +2. Create a k8s `configmap` containing Hadoop config files. This should enable Kerberos and secure Hadoop. + It should also include the Hadoop servers that would issue delegation tokens such as the HDFS namenode + address: + + kubectl create configmap hadoop-token-refresh-server-hadoop-config \ + --from-file=/usr/local/hadoop/conf/core-site.xml + +3. Create another k8s `configmap` containing Kerberos config files. This should include + the kerberos server address and the correct realm name for Kerberos principals: + + kubectl create configmap hadoop-token-refresh-server-kerberos-config \ + --from-file=/etc/krb5.conf + +4. Create a k8s `secret` containing the Kerberos keytab file. The keytab file should include + the password for the system user Kerberos principal that the refresh server is using to + extend Hadoop delegation tokens. + + kubectl create secret generic hadoop-token-refresh-server-kerberos-keytab \ + --from-file /mnt/secrets/krb5.keytab + +5. Create a k8s `service account` and `clusterrolebinding` that the service pod will use. + The service account should have `edit` capability for job `secret`s that contains + the Hadoop delegation tokens. + +6. Finally, edit the config file for k8s `deployment` and launch the service pod + using the deployment. The config file should include the right docker image tag + and the correct k8s `service account` name. + + kubectl create -f src/main/conf/kubernetes-hadoop-token-refresh-server.yaml diff --git a/conf/kubernetes-hadoop-token-refresh-server.yaml b/resource-managers/kubernetes/token-refresh-server/src/main/conf/kubernetes-hadoop-token-refresh-server.yaml similarity index 88% rename from conf/kubernetes-hadoop-token-refresh-server.yaml rename to resource-managers/kubernetes/token-refresh-server/src/main/conf/kubernetes-hadoop-token-refresh-server.yaml index a2ec0250385df..34bb7b99ac70d 100644 --- a/conf/kubernetes-hadoop-token-refresh-server.yaml +++ b/resource-managers/kubernetes/token-refresh-server/src/main/conf/kubernetes-hadoop-token-refresh-server.yaml @@ -26,6 +26,7 @@ spec: labels: hadoop-token-refresh-server-instance: default spec: + serviceAccountName: YOUR-SERVICE-ACCOUT volumes: - name: kerberos-config configMap: @@ -35,17 +36,17 @@ spec: name: hadoop-token-refresh-server-hadoop-config - name: kerberos-keytab secret: - name: hadoop-token-refresh-server-kerberos-keytab + secretName: hadoop-token-refresh-server-kerberos-keytab containers: - name: hadoop-token-refresh-server - image: docker:5000/hadoop-token-refresh-server:kimoon-0821-1 + image: YOUR-REPO:YOUR-TAG resources: requests: cpu: 100m - memory: 256Mi + memory: 512Mi limits: cpu: 100m - memory: 256Mi + memory: 512Mi volumeMounts: - name: kerberos-config mountPath: '/etc/krb5.conf' @@ -54,7 +55,7 @@ spec: - name: hadoop-config mountPath: '/etc/hadoop/conf' readOnly: true - - name: kereros-keytab + - name: kerberos-keytab mountPath: '/mnt/secrets/krb5.keytab' subPath: krb5.keytab readOnly: true diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/docker/Dockerfile b/resource-managers/kubernetes/token-refresh-server/src/main/docker/Dockerfile index 4248310ee20d3..daf63dc1a77e3 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/docker/Dockerfile +++ b/resource-managers/kubernetes/token-refresh-server/src/main/docker/Dockerfile @@ -15,11 +15,27 @@ # limitations under the License. # -FROM spark-base +FROM openjdk:8-alpine -# If this docker file is being used in the context of building your images from a Spark distribution, the docker build -# command should be invoked from the top level directory of the Spark distribution. E.g.: -# docker build -t spark-hadoop-token-refresh-server:latest -f dockerfiles/hadoop-token-refresh-server/Dockerfile . +RUN apk upgrade --no-cache && \ + apk add --no-cache bash tini && \ + rm /bin/sh && \ + ln -sv /bin/bash /bin/sh && \ + chgrp root /etc/passwd && \ + chmod ug+rw /etc/passwd && \ + mkdir -p /opt/token-refresh-server && \ + mkdir -p /opt/token-refresh-server/jars && \ + mkdir -p /opt/token-refresh-server/work-dir -ENTRYPOINT [ "/sbin/tini", "--", "bin/spark-class", "org.apache.spark.security.kubernetes.TokenRefreshServer" ] +ADD target/token-refresh-server-kubernetes_2.11-2.2.0-k8s-0.3.0-SNAPSHOT-bin.jar \ + /opt/token-refresh-server/jars +WORKDIR /opt/token-refresh-server/work-dir + +# The docker build command should be invoked from the top level directory of +# the token-refresh-server project. E.g.: +# docker build -t hadoop-token-refresh-server:latest \ +# -f src/main/docker/Dockerfile . + +CMD /sbin/tini -s -- /usr/bin/java -cp '/opt/token-refresh-server/jars/*' \ + org.apache.spark.security.kubernetes.TokenRefreshServer From 388063a0b5baca227e91a534348357ba53637000 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Tue, 12 Sep 2017 17:33:10 -0700 Subject: [PATCH 11/43] Fix a bug by including hadoop conf dir in the classpath --- ...ubernetes-hadoop-token-refresh-server.yaml | 7 +++++- .../src/main/docker/Dockerfile | 6 +++-- .../kubernetes/TokenRefreshServer.scala | 23 ++++++++++++++++++- 3 files changed, 32 insertions(+), 4 deletions(-) diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/conf/kubernetes-hadoop-token-refresh-server.yaml b/resource-managers/kubernetes/token-refresh-server/src/main/conf/kubernetes-hadoop-token-refresh-server.yaml index 34bb7b99ac70d..e58cf6916fbd3 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/conf/kubernetes-hadoop-token-refresh-server.yaml +++ b/resource-managers/kubernetes/token-refresh-server/src/main/conf/kubernetes-hadoop-token-refresh-server.yaml @@ -26,7 +26,7 @@ spec: labels: hadoop-token-refresh-server-instance: default spec: - serviceAccountName: YOUR-SERVICE-ACCOUT + serviceAccountName: YOUR-SERVICE-ACCOUNT volumes: - name: kerberos-config configMap: @@ -40,6 +40,11 @@ spec: containers: - name: hadoop-token-refresh-server image: YOUR-REPO:YOUR-TAG + env: + - name: HADOOP_CONF_DIR + value: /etc/hadoop/conf + - name: TOKEN_REFRESH_SERVER_ARGS + value: --verbose resources: requests: cpu: 100m diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/docker/Dockerfile b/resource-managers/kubernetes/token-refresh-server/src/main/docker/Dockerfile index daf63dc1a77e3..1d4a1415e27af 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/docker/Dockerfile +++ b/resource-managers/kubernetes/token-refresh-server/src/main/docker/Dockerfile @@ -37,5 +37,7 @@ WORKDIR /opt/token-refresh-server/work-dir # docker build -t hadoop-token-refresh-server:latest \ # -f src/main/docker/Dockerfile . -CMD /sbin/tini -s -- /usr/bin/java -cp '/opt/token-refresh-server/jars/*' \ - org.apache.spark.security.kubernetes.TokenRefreshServer +CMD /sbin/tini -s -- /usr/bin/java \ + -cp $HADOOP_CONF_DIR:'/opt/token-refresh-server/jars/*' \ + org.apache.spark.security.kubernetes.TokenRefreshServer \ + $TOKEN_REFRESH_SERVER_ARGS diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala index 809a7c448da34..7d501caca9726 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala @@ -21,6 +21,8 @@ import scala.concurrent.duration.Duration import akka.actor.ActorSystem import org.apache.log4j.{Level, Logger} +import scala.annotation.tailrec + private class Server { @@ -46,8 +48,27 @@ private class Server { object TokenRefreshServer { + private class Arguments(args: List[String]) { + + var logLevel: Level = Level.WARN + + parse(args) + + @tailrec + private def parse(args: List[String]): Unit = args match { + case ("--verbose" | "-v") :: tail => + logLevel = Level.INFO + parse(tail) + case ("--debug" | "-d") :: tail => + logLevel = Level.DEBUG + parse(tail) + case _ => + } + } + def main(args: Array[String]): Unit = { - Logger.getRootLogger.setLevel(Level.INFO) + val parsedArgs = new Arguments(args.toList) + Logger.getRootLogger.setLevel(parsedArgs.logLevel) val server = new Server try { server.start() From 1426523bbb855ecf200115e74c4afa99e8f24081 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Wed, 20 Sep 2017 09:47:25 -0700 Subject: [PATCH 12/43] Add token-refresh-server as extra build-only module --- assembly/pom.xml | 3 +++ pom.xml | 11 +++++++++++ resource-managers/kubernetes/README.md | 20 +++++++++++++++++--- 3 files changed, 31 insertions(+), 3 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 3c097bf1a23ab..cc726825c91df 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -150,6 +150,9 @@ kubernetes + org.apache.spark diff --git a/pom.xml b/pom.xml index 6063e03bb74fe..5e28d77b64b18 100644 --- a/pom.xml +++ b/pom.xml @@ -2619,6 +2619,17 @@ + + + kubernetes-extra + + + resource-managers/kubernetes/token-refresh-server + + + kubernetes-integration-tests diff --git a/resource-managers/kubernetes/README.md b/resource-managers/kubernetes/README.md index 31b721d193362..816f1ffdcf81c 100644 --- a/resource-managers/kubernetes/README.md +++ b/resource-managers/kubernetes/README.md @@ -28,11 +28,13 @@ building Spark normally. For example, to build Spark against Hadoop 2.7 and Kube dev/make-distribution.sh --tgz -Phadoop-2.7 -Pkubernetes -# Kubernetes Code Modules +# Kubernetes Modules Below is a list of the submodules for this cluster manager and what they do. * `core`: Implementation of the Kubernetes cluster manager support. +* `token-refresh-server`: Extra Kubernetes service that refreshes Hadoop + tokens for long-running Spark jobs accessing secure data source like HDFS. * `integration-tests`: Integration tests for the project. * `docker-minimal-bundle`: Base Dockerfiles for the driver and the executors. The Dockerfiles are used for integration tests as well as being provided in packaged distributions of Spark. @@ -40,6 +42,18 @@ Below is a list of the submodules for this cluster manager and what they do. * `integration-tests-spark-jobs-helpers`: Dependencies for the spark jobs used in integration tests. These dependencies are separated out to facilitate testing the shipping of jars to drivers running on Kubernetes clusters. +# Building Extra Submodules for Spark with Kubernetes + +There are non-core extra submodules such as token-refresh-server. To build +those, use the `kubernetes-extra` profile when invoking Maven. For example, +to build the token-refresh-server submodule: + + build/mvn package -Pkubernetes-extra -pl resource-managers/kubernetes/token-refresh-rver -am + +Some of these submodules are helper Kubernetes services. They need not be part +of the Spark distribution. The distribution build script will not include +artifacts from these submodules. + # Running the Kubernetes Integration Tests Note that the integration test framework is currently being heavily revised and is subject to change. @@ -64,7 +78,7 @@ build/mvn integration-test \ -pl resource-managers/kubernetes/integration-tests -am ``` -# Running against an arbitrary cluster +## Running against an arbitrary cluster In order to run against any cluster, use the following: ```sh @@ -74,7 +88,7 @@ build/mvn integration-test \ -DextraScalaTestArgs="-Dspark.kubernetes.test.master=k8s://https:// -Dspark.docker.test.driverImage= -Dspark.docker.test.executorImage=" ``` -# Preserve the Minikube VM +## Preserve the Minikube VM The integration tests make use of [Minikube](https://github.com/kubernetes/minikube), which fires up a virtual machine and setup a single-node kubernetes cluster within it. By default the vm is destroyed after the tests are finished. From 50c3a66c93b8aef8f87a073195d63f3d199a855c Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Tue, 3 Oct 2017 15:08:00 -0700 Subject: [PATCH 13/43] Use akka scheduler for renew tasks --- .../kubernetes/TokenRefreshService.scala | 41 +++++++++---------- 1 file changed, 19 insertions(+), 22 deletions(-) diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index 1ba5e780fab80..7e8b9ea40b9ad 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -17,24 +17,28 @@ package org.apache.spark.security.kubernetes import java.io.{ByteArrayInputStream, DataInputStream} -import java.util.concurrent.{Executors, ScheduledFuture, ThreadFactory, TimeUnit} +import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ import scala.collection.mutable -import akka.actor.{Actor, ActorRef, ActorSystem, Props} +import scala.concurrent.duration.Duration +import scala.concurrent.ExecutionContext.Implicits.global +import akka.actor.{Actor, ActorRef, ActorSystem, Cancellable, Props} import io.fabric8.kubernetes.api.model.Secret import org.apache.commons.codec.binary.Base64 import org.apache.hadoop.conf.Configuration import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.security.token.{Token, TokenIdentifier} + import org.apache.spark.security.kubernetes.constants._ + private class TokenRefreshService extends Actor with Logging { - private val scheduler = newScheduler() + private val scheduler = context.system.scheduler // Keyed by secret UID. - private val taskHandleBySecret = mutable.HashMap[String, ScheduledFuture[_]]() + private val taskHandleBySecret = mutable.HashMap[String, Cancellable]() private val hadoopConf = new Configuration private val clock = new Clock @@ -46,30 +50,22 @@ private class TokenRefreshService extends Actor with Logging { case _ => } - private def newScheduler() = Executors.newScheduledThreadPool(REFERSH_TASKS_NUM_THREADS, - new ThreadFactory { - override def newThread(r: Runnable): Thread = { - val thread = new Thread(r, REFRESH_TASK_THREAD_NAME) - thread.setDaemon(true) - thread - } - }) - private def addStarterTask(secret: Secret) = { taskHandleBySecret.getOrElseUpdate(getSecretUid(secret), { val task = new StarterTask(secret, hadoopConf, self, clock) - val future = scheduler.schedule(task, REFRESH_STARTER_TASK_INITIAL_DELAY_MILLIS, - TimeUnit.MILLISECONDS) - logInfo(s"Started refresh of tokens in ${secret.getMetadata.getSelfLink} with ${future}") - future + val cancellable = scheduler.scheduleOnce( + Duration(REFRESH_STARTER_TASK_INITIAL_DELAY_MILLIS, TimeUnit.MILLISECONDS), + task) + logInfo(s"Started refresh of tokens in ${secret.getMetadata.getSelfLink} with ${cancellable}") + cancellable }) } private def removeRefreshTask(secret: Secret) = { val uid = getSecretUid(secret) - taskHandleBySecret.remove(uid).foreach(future => { + taskHandleBySecret.remove(uid).foreach(cancellable => { logInfo(s"Canceling refresh of tokens in ${secret.getMetadata.getSelfLink}") - future.cancel(true) // Interrupt if running. + cancellable.cancel() }) } @@ -93,11 +89,12 @@ private class TokenRefreshService extends Actor with Logging { val task = new RenewTask(renew, hadoopConf, self, clock) logInfo(s"Scheduling refresh of tokens with " + s"${renew.secret.getMetadata.getSelfLink} at now + $renewTime millis.") - taskHandleBySecret.put(uid, scheduler.schedule(task, renewTime, TimeUnit.MILLISECONDS)) + val cancellable = scheduler.scheduleOnce(Duration(renewTime, TimeUnit.MILLISECONDS), task) + taskHandleBySecret.put(uid, cancellable) } else { logWarning(s"Got too many errors for ${renew.secret.getMetadata.getSelfLink}. Abandoning.") - val future = taskHandleBySecret.remove(uid) - future.foreach(_.cancel(true)) // Interrupt if running. + val maybeCancellable = taskHandleBySecret.remove(uid) + maybeCancellable.foreach(_.cancel()) } } else { logWarning(s"Could not find an entry for renew task" + From c2ccaa9b87a87b73f705337702883c97a4330ceb Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Tue, 3 Oct 2017 15:42:41 -0700 Subject: [PATCH 14/43] Relogin to Kerberos periodically --- .../kubernetes/TokenRefreshService.scala | 32 +++++++++++++++---- .../spark/security/kubernetes/constants.scala | 1 + 2 files changed, 26 insertions(+), 7 deletions(-) diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index 7e8b9ea40b9ad..4e2c9be0fa53c 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -43,6 +43,7 @@ private class TokenRefreshService extends Actor with Logging { private val clock = new Clock override def receive: PartialFunction[Any, Unit] = { + case Relogin => launchReloginTask() case StartRefresh(secret) => addStarterTask(secret) case StopRefresh(secret) => removeRefreshTask(secret) case UpdateRefreshList(secrets) => updateRefreshTaskSet(secrets) @@ -50,6 +51,11 @@ private class TokenRefreshService extends Actor with Logging { case _ => } + private def launchReloginTask() = { + val task = new ReloginTask + scheduler.scheduleOnce(Duration(0L, TimeUnit.MILLISECONDS), task) + } + private def addStarterTask(secret: Secret) = { taskHandleBySecret.getOrElseUpdate(getSecretUid(secret), { val task = new StarterTask(secret, hadoopConf, self, clock) @@ -61,7 +67,7 @@ private class TokenRefreshService extends Actor with Logging { }) } - private def removeRefreshTask(secret: Secret) = { + private def removeRefreshTask(secret: Secret) : Unit = { val uid = getSecretUid(secret) taskHandleBySecret.remove(uid).foreach(cancellable => { logInfo(s"Canceling refresh of tokens in ${secret.getMetadata.getSelfLink}") @@ -69,7 +75,7 @@ private class TokenRefreshService extends Actor with Logging { }) } - private def updateRefreshTaskSet(currentSecrets: List[Secret]) = { + private def updateRefreshTaskSet(currentSecrets: List[Secret]) : Unit = { val secretByUid = currentSecrets.map(secret => (getSecretUid(secret), secret)).toMap val currentUids = secretByUid.keySet val priorUids = taskHandleBySecret.keySet @@ -105,6 +111,13 @@ private class TokenRefreshService extends Actor with Logging { private def getSecretUid(secret: Secret) = secret.getMetadata.getUid } +private class ReloginTask extends Runnable { + + override def run() : Unit = { + UserGroupInformation.getLoginUser.checkTGTAndReloginFromKeytab() + } +} + private class StarterTask(secret: Secret, hadoopConf: Configuration, refreshService: ActorRef, @@ -215,13 +228,15 @@ private class Clock { def nowInMillis() : Long = System.currentTimeMillis() } -private case class UpdateRefreshList(secrets: List[Secret]) -private case class StartRefresh(secret: Secret) +private sealed trait Command +private case object Relogin extends Command +private case class UpdateRefreshList(secrets: List[Secret]) extends Command +private case class StartRefresh(secret: Secret) extends Command private case class Renew(expireTime: Long, expireTimeByToken: Map[Token[_ <: TokenIdentifier], Long], secret: Secret, - numConsecutiveErrors: Int) -private case class StopRefresh(secret: Secret) + numConsecutiveErrors: Int) extends Command +private case class StopRefresh(secret: Secret) extends Command private object TokenRefreshService { @@ -229,6 +244,9 @@ private object TokenRefreshService { UserGroupInformation.loginUserFromKeytab( REFRESH_SERVER_KERBEROS_PRINCIPAL, REFRESH_SERVER_KERBEROS_KEYTAB_PATH) - system.actorOf(Props[TokenRefreshService]) + val actor = system.actorOf(Props[TokenRefreshService]) + val duration = Duration(REFRESH_SERVER_KERBEROS_RELOGIN_PERIOD_MILLIS, TimeUnit.MILLISECONDS) + system.scheduler.schedule(duration, duration, actor, Relogin) + actor } } diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala index 05a72af8e66f1..6951d6a74dfef 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala @@ -20,6 +20,7 @@ package object constants { val REFRESH_SERVER_KERBEROS_PRINCIPAL = "kimoonkim" val REFRESH_SERVER_KERBEROS_KEYTAB_PATH = "/mnt/secrets/krb5.keytab" + val REFRESH_SERVER_KERBEROS_RELOGIN_PERIOD_MILLIS = 60 * 60 * 1000L val SECRET_LABEL_KEY_REFRESH_HADOOP_TOKENS = "refresh-hadoop-tokens" val SECRET_LABEL_VALUE_REFRESH_HADOOP_TOKENS = "yes" From a2aec2bb230a9b1f26ad433f9777aaa13e44a6b2 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Tue, 3 Oct 2017 16:02:49 -0700 Subject: [PATCH 15/43] Renew at 90% mark of deadline --- .../security/kubernetes/TokenRefreshService.scala | 14 ++++++++------ .../spark/security/kubernetes/constants.scala | 1 + 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index 4e2c9be0fa53c..f8aebcf5a9569 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -90,12 +90,14 @@ private class TokenRefreshService extends Actor with Logging { if (taskHandleBySecret.get(uid).nonEmpty) { val numConsecutiveErrors = renew.numConsecutiveErrors if (numConsecutiveErrors < RENEW_TASK_MAX_CONSECUTIVE_ERRORS) { - val renewTime = math.max(0L, - renew.expireTime - RENEW_TASK_SCHEDULE_AHEAD_MILLIS - clock.nowInMillis()) + val durationTillExpire = math.max(0L, renew.expireTime - clock.nowInMillis()) + val renewTime = math.max(0L, renew.expireTime - durationTillExpire / 10) // 90% mark. + val durationTillRenew = math.max(0L, renewTime - clock.nowInMillis()) val task = new RenewTask(renew, hadoopConf, self, clock) logInfo(s"Scheduling refresh of tokens with " + - s"${renew.secret.getMetadata.getSelfLink} at now + $renewTime millis.") - val cancellable = scheduler.scheduleOnce(Duration(renewTime, TimeUnit.MILLISECONDS), task) + s"${renew.secret.getMetadata.getSelfLink} at now + $durationTillRenew millis.") + val cancellable = scheduler.scheduleOnce( + Duration(durationTillRenew, TimeUnit.MILLISECONDS), task) taskHandleBySecret.put(uid, cancellable) } else { logWarning(s"Got too many errors for ${renew.secret.getMetadata.getSelfLink}. Abandoning.") @@ -175,7 +177,7 @@ private class StarterTask(secret: Secret, }).toMap } - private def getRetryTime() = clock.nowInMillis() + RENEW_TASK_DEADLINE_LOOK_AHEAD_MILLIS + private def getRetryTime() = clock.nowInMillis() + RENEW_TASK_RETRY_TIME_MILLIS } private class RenewTask(renew: Renew, @@ -220,7 +222,7 @@ private class RenewTask(renew: Renew, } } - private def getRetryTime() = clock.nowInMillis() + RENEW_TASK_DEADLINE_LOOK_AHEAD_MILLIS + private def getRetryTime() = clock.nowInMillis() + RENEW_TASK_RETRY_TIME_MILLIS } private class Clock { diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala index 6951d6a74dfef..e25ef9c8a08df 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala @@ -29,6 +29,7 @@ package object constants { val REFERSH_TASKS_NUM_THREADS = 10 val REFRESH_TASK_THREAD_NAME = "token-renewer" val REFRESH_STARTER_TASK_INITIAL_DELAY_MILLIS = 0L + val RENEW_TASK_RETRY_TIME_MILLIS = 10000L val RENEW_TASK_DEADLINE_LOOK_AHEAD_MILLIS = 10000L val RENEW_TASK_SCHEDULE_AHEAD_MILLIS = 10000L val RENEW_TASK_MAX_CONSECUTIVE_ERRORS = 3 From ec70b47dce4c93d93cce0c8745d986b21c580117 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Tue, 3 Oct 2017 16:47:28 -0700 Subject: [PATCH 16/43] Get renew time from data item key --- .../kubernetes/TokenRefreshService.scala | 49 +++++++------------ .../spark/security/kubernetes/constants.scala | 1 + 2 files changed, 18 insertions(+), 32 deletions(-) diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index f8aebcf5a9569..12bbba174ff6b 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -128,15 +128,14 @@ private class StarterTask(secret: Secret, private var hasError = false override def run() : Unit = { - val tokens = readHadoopTokens() - logInfo(s"Read Hadoop tokens: $tokens") - val expireTimeByToken = renewTokens(tokens) + val expireTimeByToken = readHadoopTokens() + logInfo(s"Read Hadoop tokens: $expireTimeByToken") val nextExpireTime = if (expireTimeByToken.nonEmpty) { expireTimeByToken.values.min } else { logWarning(s"Got an empty token list with ${secret.getMetadata.getSelfLink}") hasError = true - getRetryTime() + getRetryTime } logInfo(s"Initial renew resulted with $expireTimeByToken. Next expire time $nextExpireTime") val numConsecutiveErrors = if (hasError) 1 else 0 @@ -146,38 +145,24 @@ private class StarterTask(secret: Secret, private def readHadoopTokens() = { val hadoopSecretData = secret.getData.asScala.filterKeys( _.startsWith(SECRET_DATA_KEY_PREFIX_HADOOP_TOKENS)) - val latestData = if (hadoopSecretData.nonEmpty) Some(hadoopSecretData.max._2) else None - val credentials = latestData.map { - data => + val latestData = if (hadoopSecretData.nonEmpty) Some(hadoopSecretData.max) else None + val tokens = latestData.map { + item => + val key = item._1 + val data = item._2 + val createTimeAndDuration = key.split(SECRET_DATA_KEY_REGEX_HADOOP_TOKENS, 2) + val expireTime = createTimeAndDuration(0).toLong + createTimeAndDuration(1).toLong val creds = new Credentials creds.readTokenStorageStream(new DataInputStream(new ByteArrayInputStream( Base64.decodeBase64(data)))) - creds - } - val tokens = credentials.map { - creds => - creds.getAllTokens.asScala.toList - } - tokens.getOrElse(Nil) - } - - private def renewTokens(tokens: List[Token[_ <: TokenIdentifier]]) - : Map[Token[_ <: TokenIdentifier], Long] = { - tokens.map(token => { - val expireTime = try { - token.renew(hadoopConf) - } catch { - case t: Throwable => - logWarning(t.getMessage, t) - hasError = true - - getRetryTime() + creds.getAllTokens.asScala.toList.map { + (_, expireTime) } - (token, expireTime) - }).toMap + } + tokens.getOrElse(Nil).toMap } - private def getRetryTime() = clock.nowInMillis() + RENEW_TASK_RETRY_TIME_MILLIS + private def getRetryTime = clock.nowInMillis() + RENEW_TASK_RETRY_TIME_MILLIS } private class RenewTask(renew: Renew, @@ -203,7 +188,7 @@ private class RenewTask(renew: Renew, logWarning(t.getMessage, t) hasError = true - getRetryTime() + getRetryTime } } else { expireTime @@ -222,7 +207,7 @@ private class RenewTask(renew: Renew, } } - private def getRetryTime() = clock.nowInMillis() + RENEW_TASK_RETRY_TIME_MILLIS + private def getRetryTime = clock.nowInMillis() + RENEW_TASK_RETRY_TIME_MILLIS } private class Clock { diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala index e25ef9c8a08df..571ebda58fb28 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala @@ -25,6 +25,7 @@ package object constants { val SECRET_LABEL_KEY_REFRESH_HADOOP_TOKENS = "refresh-hadoop-tokens" val SECRET_LABEL_VALUE_REFRESH_HADOOP_TOKENS = "yes" val SECRET_DATA_KEY_PREFIX_HADOOP_TOKENS = "hadoop-tokens-" + val SECRET_DATA_KEY_REGEX_HADOOP_TOKENS = "hadoop-tokens-(\\d+)-(\\d+)" val REFERSH_TASKS_NUM_THREADS = 10 val REFRESH_TASK_THREAD_NAME = "token-renewer" From 0b049fd3dc738b6472ad44b58b459b5bed6d3663 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Tue, 3 Oct 2017 17:07:24 -0700 Subject: [PATCH 17/43] Fix compile error --- .../spark/security/kubernetes/TokenRefreshService.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index 12bbba174ff6b..af522c3e08e6a 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -142,11 +142,11 @@ private class StarterTask(secret: Secret, refreshService ! Renew(nextExpireTime, expireTimeByToken, secret, numConsecutiveErrors) } - private def readHadoopTokens() = { + private def readHadoopTokens() : Map[Token[_ <: TokenIdentifier], Long] = { val hadoopSecretData = secret.getData.asScala.filterKeys( _.startsWith(SECRET_DATA_KEY_PREFIX_HADOOP_TOKENS)) val latestData = if (hadoopSecretData.nonEmpty) Some(hadoopSecretData.max) else None - val tokens = latestData.map { + latestData.map { item => val key = item._1 val data = item._2 @@ -158,8 +158,7 @@ private class StarterTask(secret: Secret, creds.getAllTokens.asScala.toList.map { (_, expireTime) } - } - tokens.getOrElse(Nil).toMap + }.toList.flatten.toMap } private def getRetryTime = clock.nowInMillis() + RENEW_TASK_RETRY_TIME_MILLIS From d42c568e6402d409c062b91cdb00f93f97561fc4 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Wed, 4 Oct 2017 14:28:13 -0700 Subject: [PATCH 18/43] Obtain new tokens --- .../kubernetes/TokenRefreshService.scala | 108 +++++++++++++----- .../spark/security/kubernetes/constants.scala | 10 +- 2 files changed, 87 insertions(+), 31 deletions(-) diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index af522c3e08e6a..b0283b248507a 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -17,20 +17,22 @@ package org.apache.spark.security.kubernetes import java.io.{ByteArrayInputStream, DataInputStream} +import java.security.{PrivilegedActionException, PrivilegedExceptionAction} import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ import scala.collection.mutable import scala.concurrent.duration.Duration import scala.concurrent.ExecutionContext.Implicits.global - import akka.actor.{Actor, ActorRef, ActorSystem, Cancellable, Props} import io.fabric8.kubernetes.api.model.Secret import org.apache.commons.codec.binary.Base64 import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier +import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.security.token.{Token, TokenIdentifier} - import org.apache.spark.security.kubernetes.constants._ @@ -60,7 +62,7 @@ private class TokenRefreshService extends Actor with Logging { taskHandleBySecret.getOrElseUpdate(getSecretUid(secret), { val task = new StarterTask(secret, hadoopConf, self, clock) val cancellable = scheduler.scheduleOnce( - Duration(REFRESH_STARTER_TASK_INITIAL_DELAY_MILLIS, TimeUnit.MILLISECONDS), + Duration(STARTER_TASK_INITIAL_DELAY_MILLIS, TimeUnit.MILLISECONDS), task) logInfo(s"Started refresh of tokens in ${secret.getMetadata.getSelfLink} with ${cancellable}") cancellable @@ -128,18 +130,18 @@ private class StarterTask(secret: Secret, private var hasError = false override def run() : Unit = { - val expireTimeByToken = readHadoopTokens() - logInfo(s"Read Hadoop tokens: $expireTimeByToken") - val nextExpireTime = if (expireTimeByToken.nonEmpty) { - expireTimeByToken.values.min + val tokenToExpireTime = readHadoopTokens() + logInfo(s"Read Hadoop tokens: $tokenToExpireTime") + val nextExpireTime = if (tokenToExpireTime.nonEmpty) { + tokenToExpireTime.values.min } else { logWarning(s"Got an empty token list with ${secret.getMetadata.getSelfLink}") hasError = true getRetryTime } - logInfo(s"Initial renew resulted with $expireTimeByToken. Next expire time $nextExpireTime") + logInfo(s"Initial renew resulted with $tokenToExpireTime. Next expire time $nextExpireTime") val numConsecutiveErrors = if (hasError) 1 else 0 - refreshService ! Renew(nextExpireTime, expireTimeByToken, secret, numConsecutiveErrors) + refreshService ! Renew(nextExpireTime, tokenToExpireTime, secret, numConsecutiveErrors) } private def readHadoopTokens() : Map[Token[_ <: TokenIdentifier], Long] = { @@ -173,26 +175,14 @@ private class RenewTask(renew: Renew, override def run() : Unit = { val deadline = renew.expireTime + RENEW_TASK_DEADLINE_LOOK_AHEAD_MILLIS + val nowMillis = clock.nowInMillis() val newExpireTimeByToken : Map[Token[_ <: TokenIdentifier], Long] = - renew.expireTimeByToken.map { + renew.tokenToExpireTime.map { item => val token = item._1 val expireTime = item._2 - val newExpireTime = - if (expireTime <= deadline) { - try { - token.renew(hadoopConf) - } catch { - case t: Throwable => - logWarning(t.getMessage, t) - hasError = true - - getRetryTime - } - } else { - expireTime - } - (token, newExpireTime) + val (maybeNewToken, maybeNewExpireTime) = refresh(token, expireTime, deadline, nowMillis) + (maybeNewToken, maybeNewExpireTime) } .toMap if (newExpireTimeByToken.nonEmpty) { @@ -206,6 +196,72 @@ private class RenewTask(renew: Renew, } } + private def refresh(token: Token[_ <: TokenIdentifier], expireTime: Long, deadline: Long, + nowMillis: Long) = { + val maybeNewToken = maybeObtainNewToken(token, expireTime) + val maybeNewExpireTime = maybeRenewExpireTime(maybeNewToken, expireTime, deadline, nowMillis) + (token, maybeNewExpireTime) + } + + private def maybeObtainNewToken(token: Token[_ <: TokenIdentifier], expireTime: Long) = { + val maybeNewToken = if (token.getKind.equals(DelegationTokenIdentifier.HDFS_DELEGATION_KIND)) { + val identifier = token.decodeIdentifier().asInstanceOf[AbstractDelegationTokenIdentifier] + val maxDate = identifier.getMaxDate + if (maxDate - expireTime < RENEW_TASK_REMAINING_TIME_BEFORE_NEW_TOKEN_MILLIS) { + val newToken = obtainNewToken(token, identifier) + logInfo(s"Obtained token $newToken") + newToken + } else { + token + } + } else { + token + } + maybeNewToken + } + + private def maybeRenewExpireTime(token: Token[_ <: TokenIdentifier], expireTime: Long, + deadline: Long, nowMillis: Long) = { + if (expireTime <= deadline || expireTime <= nowMillis) { + try { + val newExpireTime = token.renew(hadoopConf) + logInfo(s"Renewed token $token. Next expire time $newExpireTime") + newExpireTime + } catch { + case t: Throwable => + logWarning(t.getMessage, t) + hasError = true + + getRetryTime + } + } else { + expireTime + } + } + + private def obtainNewToken(token: Token[_ <: TokenIdentifier], + identifier: AbstractDelegationTokenIdentifier) = { + val owner = identifier.getOwner + val realUser = identifier.getRealUser + val user = if (realUser == null || realUser.toString.isEmpty || realUser.equals(owner)) { + owner.toString + } else { + realUser.toString + } + val credentials = new Credentials() + val ugi = UserGroupInformation.createProxyUser(user, UserGroupInformation.getLoginUser) + val newToken = ugi.doAs(new PrivilegedExceptionAction[Token[_ <: TokenIdentifier]] { + + override def run() : Token[_ <: TokenIdentifier] = { + val fs = FileSystem.get(hadoopConf) + val tokens = fs.addDelegationTokens(UserGroupInformation.getLoginUser.getUserName, + credentials) + tokens(0) + } + }) + newToken + } + private def getRetryTime = clock.nowInMillis() + RENEW_TASK_RETRY_TIME_MILLIS } @@ -219,7 +275,7 @@ private case object Relogin extends Command private case class UpdateRefreshList(secrets: List[Secret]) extends Command private case class StartRefresh(secret: Secret) extends Command private case class Renew(expireTime: Long, - expireTimeByToken: Map[Token[_ <: TokenIdentifier], Long], + tokenToExpireTime: Map[Token[_ <: TokenIdentifier], Long], secret: Secret, numConsecutiveErrors: Int) extends Command private case class StopRefresh(secret: Secret) extends Command diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala index 571ebda58fb28..02694dc45f159 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala @@ -27,13 +27,13 @@ package object constants { val SECRET_DATA_KEY_PREFIX_HADOOP_TOKENS = "hadoop-tokens-" val SECRET_DATA_KEY_REGEX_HADOOP_TOKENS = "hadoop-tokens-(\\d+)-(\\d+)" - val REFERSH_TASKS_NUM_THREADS = 10 - val REFRESH_TASK_THREAD_NAME = "token-renewer" - val REFRESH_STARTER_TASK_INITIAL_DELAY_MILLIS = 0L - val RENEW_TASK_RETRY_TIME_MILLIS = 10000L - val RENEW_TASK_DEADLINE_LOOK_AHEAD_MILLIS = 10000L + val STARTER_TASK_INITIAL_DELAY_MILLIS = 0L + val RENEW_TASK_SCHEDULE_AHEAD_MILLIS = 10000L + val RENEW_TASK_RETRY_TIME_MILLIS = 10000L val RENEW_TASK_MAX_CONSECUTIVE_ERRORS = 3 + val RENEW_TASK_DEADLINE_LOOK_AHEAD_MILLIS = 10000L + val RENEW_TASK_REMAINING_TIME_BEFORE_NEW_TOKEN_MILLIS = 3 * 60 * 60 * 1000L val SECRET_SCANNER_THREAD_NAME = "secret-scanner" val SECRET_SCANNER_INITIAL_DELAY_MILLIS = 10 * 1000L From 5d9687933a86c9015c5a1558131eeda9d2455e6e Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Wed, 4 Oct 2017 14:35:06 -0700 Subject: [PATCH 19/43] Fix bugs --- .../kubernetes/TokenRefreshService.scala | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index b0283b248507a..44a9d7761619f 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -149,9 +149,7 @@ private class StarterTask(secret: Secret, _.startsWith(SECRET_DATA_KEY_PREFIX_HADOOP_TOKENS)) val latestData = if (hadoopSecretData.nonEmpty) Some(hadoopSecretData.max) else None latestData.map { - item => - val key = item._1 - val data = item._2 + case (key, data) => val createTimeAndDuration = key.split(SECRET_DATA_KEY_REGEX_HADOOP_TOKENS, 2) val expireTime = createTimeAndDuration(0).toLong + createTimeAndDuration(1).toLong val creds = new Credentials @@ -178,9 +176,7 @@ private class RenewTask(renew: Renew, val nowMillis = clock.nowInMillis() val newExpireTimeByToken : Map[Token[_ <: TokenIdentifier], Long] = renew.tokenToExpireTime.map { - item => - val token = item._1 - val expireTime = item._2 + case (token, expireTime) => val (maybeNewToken, maybeNewExpireTime) = refresh(token, expireTime, deadline, nowMillis) (maybeNewToken, maybeNewExpireTime) } @@ -198,16 +194,18 @@ private class RenewTask(renew: Renew, private def refresh(token: Token[_ <: TokenIdentifier], expireTime: Long, deadline: Long, nowMillis: Long) = { - val maybeNewToken = maybeObtainNewToken(token, expireTime) + val maybeNewToken = maybeObtainNewToken(token, expireTime, nowMillis) val maybeNewExpireTime = maybeRenewExpireTime(maybeNewToken, expireTime, deadline, nowMillis) - (token, maybeNewExpireTime) + (maybeNewToken, maybeNewExpireTime) } - private def maybeObtainNewToken(token: Token[_ <: TokenIdentifier], expireTime: Long) = { + private def maybeObtainNewToken(token: Token[_ <: TokenIdentifier], expireTime: Long, + nowMills: Long) = { val maybeNewToken = if (token.getKind.equals(DelegationTokenIdentifier.HDFS_DELEGATION_KIND)) { val identifier = token.decodeIdentifier().asInstanceOf[AbstractDelegationTokenIdentifier] val maxDate = identifier.getMaxDate - if (maxDate - expireTime < RENEW_TASK_REMAINING_TIME_BEFORE_NEW_TOKEN_MILLIS) { + if (maxDate - expireTime < RENEW_TASK_REMAINING_TIME_BEFORE_NEW_TOKEN_MILLIS || + maxDate <= nowMills) { val newToken = obtainNewToken(token, identifier) logInfo(s"Obtained token $newToken") newToken From c0e28d4e6cfd45cb5db8d185cfd3d0277df4a4eb Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Wed, 4 Oct 2017 15:58:58 -0700 Subject: [PATCH 20/43] Write back tokens to K8s secret --- .../kubernetes/TokenRefreshService.scala | 34 ++++++++++++++++--- 1 file changed, 30 insertions(+), 4 deletions(-) diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index 44a9d7761619f..c3c9df4d6aba1 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.security.kubernetes -import java.io.{ByteArrayInputStream, DataInputStream} +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} import java.security.{PrivilegedActionException, PrivilegedExceptionAction} import java.util.concurrent.TimeUnit @@ -25,7 +25,7 @@ import scala.collection.mutable import scala.concurrent.duration.Duration import scala.concurrent.ExecutionContext.Implicits.global import akka.actor.{Actor, ActorRef, ActorSystem, Cancellable, Props} -import io.fabric8.kubernetes.api.model.Secret +import io.fabric8.kubernetes.api.model.{Secret, SecretBuilder} import org.apache.commons.codec.binary.Base64 import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem @@ -130,7 +130,7 @@ private class StarterTask(secret: Secret, private var hasError = false override def run() : Unit = { - val tokenToExpireTime = readHadoopTokens() + val tokenToExpireTime = readTokensFromSecret() logInfo(s"Read Hadoop tokens: $tokenToExpireTime") val nextExpireTime = if (tokenToExpireTime.nonEmpty) { tokenToExpireTime.values.min @@ -144,7 +144,7 @@ private class StarterTask(secret: Secret, refreshService ! Renew(nextExpireTime, tokenToExpireTime, secret, numConsecutiveErrors) } - private def readHadoopTokens() : Map[Token[_ <: TokenIdentifier], Long] = { + private def readTokensFromSecret() : Map[Token[_ <: TokenIdentifier], Long] = { val hadoopSecretData = secret.getData.asScala.filterKeys( _.startsWith(SECRET_DATA_KEY_PREFIX_HADOOP_TOKENS)) val latestData = if (hadoopSecretData.nonEmpty) Some(hadoopSecretData.max) else None @@ -182,6 +182,8 @@ private class RenewTask(renew: Renew, } .toMap if (newExpireTimeByToken.nonEmpty) { + val newTokens = newExpireTimeByToken.keySet -- renew.tokenToExpireTime.keySet + if (newTokens.nonEmpty) writeTokensToSecret(newExpireTimeByToken, nowMillis) val nextExpireTime = newExpireTimeByToken.values.min logInfo(s"Renewed with the result $newExpireTimeByToken. Next expire time $nextExpireTime") val numConsecutiveErrors = if (hasError) renew.numConsecutiveErrors + 1 else 0 @@ -260,6 +262,30 @@ private class RenewTask(renew: Renew, newToken } + private def writeTokensToSecret(tokenToExpire: Map[Token[_ <: TokenIdentifier], Long], + nowMillis: Long) = { + val durationUntilExpire = tokenToExpire.values.min - nowMillis + val key = s"$SECRET_DATA_KEY_PREFIX_HADOOP_TOKENS$nowMillis-$durationUntilExpire" + val credentials = new Credentials() + tokenToExpire.keys.foreach(token => credentials.addToken(token.getService, token)) + val serialized = serializeCredentials(credentials) + val value = Base64.encodeBase64String(serialized) + val builder = new SecretBuilder(renew.secret) + .addToData(key, value) + val hadoopSecretData = builder.getData.asScala.filterKeys( + _.startsWith(SECRET_DATA_KEY_PREFIX_HADOOP_TOKENS)) + hadoopSecretData.keys.dropRight(2).foreach(builder.removeFromData) + builder.build() + } + + private def serializeCredentials(credentials: Credentials) = { + val byteStream = new ByteArrayOutputStream + val dataStream = new DataOutputStream(byteStream) + credentials.writeTokenStorageToStream(dataStream) + dataStream.flush() + byteStream.toByteArray + } + private def getRetryTime = clock.nowInMillis() + RENEW_TASK_RETRY_TIME_MILLIS } From 57c847e3cbc5df7d7c9ad15f7f56ffe291be798d Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Thu, 5 Oct 2017 10:00:59 -0700 Subject: [PATCH 21/43] Handle recently added secrets --- .../security/kubernetes/SecretFinder.scala | 4 +- .../kubernetes/TokenRefreshService.scala | 41 +++++++++++-------- 2 files changed, 25 insertions(+), 20 deletions(-) diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala index 26edf7b002187..9541747a6dd4d 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala @@ -50,7 +50,7 @@ private class SecretScanner(renewService: ActorRef, override def run(): Unit = { val secrets = SecretFinder.selectSecrets(kubernetesClient).list.getItems.asScala.toList logInfo(s"Scanned ${secrets.map(_.getMetadata.getSelfLink).mkString}") - renewService ! UpdateRefreshList(secrets) + renewService ! UpdateSecretsToTrack(secrets) } } @@ -68,7 +68,7 @@ private class SecretWatcher(renewService: ActorRef) extends Watcher[Secret] with } override def onClose(cause: KubernetesClientException): Unit = { - // FIXME. TBD. + // Do nothing } } diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index c3c9df4d6aba1..5b72c1fa6f95b 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -17,7 +17,7 @@ package org.apache.spark.security.kubernetes import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} -import java.security.{PrivilegedActionException, PrivilegedExceptionAction} +import java.security.PrivilegedExceptionAction import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ @@ -39,18 +39,22 @@ import org.apache.spark.security.kubernetes.constants._ private class TokenRefreshService extends Actor with Logging { private val scheduler = context.system.scheduler - // Keyed by secret UID. - private val taskHandleBySecret = mutable.HashMap[String, Cancellable]() + private val secretUidToTaskHandle = mutable.HashMap[String, Cancellable]() + private val recentlyAddedSecretUids = mutable.HashSet[String]() private val hadoopConf = new Configuration private val clock = new Clock override def receive: PartialFunction[Any, Unit] = { - case Relogin => launchReloginTask() - case StartRefresh(secret) => addStarterTask(secret) - case StopRefresh(secret) => removeRefreshTask(secret) - case UpdateRefreshList(secrets) => updateRefreshTaskSet(secrets) - case renew @ Renew(nextExpireTime, expireTimeByToken, secret, _) => scheduleRenewTask(renew) - case _ => + case Relogin => + launchReloginTask() + case StartRefresh(secret) => + addStarterTask(secret) + case StopRefresh(secret) => + removeRefreshTask(secret) + case UpdateSecretsToTrack(secrets) => + updateSecretsToTrack(secrets) + case renew: Renew => + scheduleRenewTask(renew) } private def launchReloginTask() = { @@ -59,7 +63,7 @@ private class TokenRefreshService extends Actor with Logging { } private def addStarterTask(secret: Secret) = { - taskHandleBySecret.getOrElseUpdate(getSecretUid(secret), { + secretUidToTaskHandle.getOrElseUpdate(getSecretUid(secret), { val task = new StarterTask(secret, hadoopConf, self, clock) val cancellable = scheduler.scheduleOnce( Duration(STARTER_TASK_INITIAL_DELAY_MILLIS, TimeUnit.MILLISECONDS), @@ -71,25 +75,26 @@ private class TokenRefreshService extends Actor with Logging { private def removeRefreshTask(secret: Secret) : Unit = { val uid = getSecretUid(secret) - taskHandleBySecret.remove(uid).foreach(cancellable => { + secretUidToTaskHandle.remove(uid).foreach(cancellable => { logInfo(s"Canceling refresh of tokens in ${secret.getMetadata.getSelfLink}") cancellable.cancel() }) } - private def updateRefreshTaskSet(currentSecrets: List[Secret]) : Unit = { + private def updateSecretsToTrack(currentSecrets: List[Secret]) : Unit = { val secretByUid = currentSecrets.map(secret => (getSecretUid(secret), secret)).toMap val currentUids = secretByUid.keySet - val priorUids = taskHandleBySecret.keySet + val priorUids = secretUidToTaskHandle.keySet val uidsToAdd = currentUids -- priorUids uidsToAdd.foreach(uid => addStarterTask(secretByUid(uid))) - val uidsToRemove = priorUids -- currentUids + val uidsToRemove = priorUids -- currentUids -- recentlyAddedSecretUids uidsToRemove.foreach(uid => removeRefreshTask(secretByUid(uid))) + recentlyAddedSecretUids.clear() } private def scheduleRenewTask(renew: Renew) = { val uid = getSecretUid(renew.secret) - if (taskHandleBySecret.get(uid).nonEmpty) { + if (secretUidToTaskHandle.get(uid).nonEmpty) { val numConsecutiveErrors = renew.numConsecutiveErrors if (numConsecutiveErrors < RENEW_TASK_MAX_CONSECUTIVE_ERRORS) { val durationTillExpire = math.max(0L, renew.expireTime - clock.nowInMillis()) @@ -100,10 +105,10 @@ private class TokenRefreshService extends Actor with Logging { s"${renew.secret.getMetadata.getSelfLink} at now + $durationTillRenew millis.") val cancellable = scheduler.scheduleOnce( Duration(durationTillRenew, TimeUnit.MILLISECONDS), task) - taskHandleBySecret.put(uid, cancellable) + secretUidToTaskHandle.put(uid, cancellable) } else { logWarning(s"Got too many errors for ${renew.secret.getMetadata.getSelfLink}. Abandoning.") - val maybeCancellable = taskHandleBySecret.remove(uid) + val maybeCancellable = secretUidToTaskHandle.remove(uid) maybeCancellable.foreach(_.cancel()) } } else { @@ -296,7 +301,7 @@ private class Clock { private sealed trait Command private case object Relogin extends Command -private case class UpdateRefreshList(secrets: List[Secret]) extends Command +private case class UpdateSecretsToTrack(secrets: List[Secret]) extends Command private case class StartRefresh(secret: Secret) extends Command private case class Renew(expireTime: Long, tokenToExpireTime: Map[Token[_ <: TokenIdentifier], Long], From ce1bb7f86c4c32cc6f2c1bf6f8a952faf06cf018 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Thu, 5 Oct 2017 10:41:18 -0700 Subject: [PATCH 22/43] Use k8s client editable to update secret data --- .../security/kubernetes/SecretFinder.scala | 4 +- .../kubernetes/TokenRefreshServer.scala | 6 ++- .../kubernetes/TokenRefreshService.scala | 40 +++++++++++-------- .../spark/security/kubernetes/constants.scala | 4 +- 4 files changed, 31 insertions(+), 23 deletions(-) diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala index 9541747a6dd4d..0bb3f2d55bc2c 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala @@ -74,10 +74,10 @@ private class SecretWatcher(renewService: ActorRef) extends Watcher[Secret] with private object SecretFinder { - def apply(renewService: ActorRef) : SecretFinder = { + def apply(renewService: ActorRef, kubernetesClient: KubernetesClient) : SecretFinder = { new SecretFinder(renewService, new Timer(SECRET_SCANNER_THREAD_NAME, IS_DAEMON_THREAD), - new DefaultKubernetesClient) + kubernetesClient) } def selectSecrets(kubernetesClient: KubernetesClient): diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala index 7d501caca9726..632c709b10770 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala @@ -19,6 +19,7 @@ package org.apache.spark.security.kubernetes import scala.concurrent.Await import scala.concurrent.duration.Duration import akka.actor.ActorSystem +import io.fabric8.kubernetes.client.DefaultKubernetesClient import org.apache.log4j.{Level, Logger} import scala.annotation.tailrec @@ -27,11 +28,12 @@ import scala.annotation.tailrec private class Server { private val actorSystem = ActorSystem("TokenRefreshServer") + private val kubernetesClient = new DefaultKubernetesClient private var secretFinder : Option[SecretFinder] = None def start(): Unit = { - val renewService = TokenRefreshService(actorSystem) - secretFinder = Some(SecretFinder(renewService)) + val renewService = TokenRefreshService(actorSystem, kubernetesClient) + secretFinder = Some(SecretFinder(renewService, kubernetesClient)) } def join() : Unit = { diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index 5b72c1fa6f95b..02ec560ed0b43 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -21,11 +21,12 @@ import java.security.PrivilegedExceptionAction import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ -import scala.collection.mutable +import scala.collection.{SortedSet, mutable} import scala.concurrent.duration.Duration import scala.concurrent.ExecutionContext.Implicits.global import akka.actor.{Actor, ActorRef, ActorSystem, Cancellable, Props} import io.fabric8.kubernetes.api.model.{Secret, SecretBuilder} +import io.fabric8.kubernetes.client.KubernetesClient import org.apache.commons.codec.binary.Base64 import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem @@ -36,7 +37,7 @@ import org.apache.hadoop.security.token.{Token, TokenIdentifier} import org.apache.spark.security.kubernetes.constants._ -private class TokenRefreshService extends Actor with Logging { +private class TokenRefreshService(kubernetesClient: KubernetesClient) extends Actor with Logging { private val scheduler = context.system.scheduler private val secretUidToTaskHandle = mutable.HashMap[String, Cancellable]() @@ -100,7 +101,7 @@ private class TokenRefreshService extends Actor with Logging { val durationTillExpire = math.max(0L, renew.expireTime - clock.nowInMillis()) val renewTime = math.max(0L, renew.expireTime - durationTillExpire / 10) // 90% mark. val durationTillRenew = math.max(0L, renewTime - clock.nowInMillis()) - val task = new RenewTask(renew, hadoopConf, self, clock) + val task = new RenewTask(renew, hadoopConf, self, kubernetesClient, clock) logInfo(s"Scheduling refresh of tokens with " + s"${renew.secret.getMetadata.getSelfLink} at now + $durationTillRenew millis.") val cancellable = scheduler.scheduleOnce( @@ -150,12 +151,12 @@ private class StarterTask(secret: Secret, } private def readTokensFromSecret() : Map[Token[_ <: TokenIdentifier], Long] = { - val hadoopSecretData = secret.getData.asScala.filterKeys( - _.startsWith(SECRET_DATA_KEY_PREFIX_HADOOP_TOKENS)) - val latestData = if (hadoopSecretData.nonEmpty) Some(hadoopSecretData.max) else None - latestData.map { + val dataItems = secret.getData.asScala.filterKeys( + _.startsWith(SECRET_DATA_ITEM_KEY_PREFIX_HADOOP_TOKENS)).toSeq.sorted + val latestDataItem = if (dataItems.nonEmpty) Some(dataItems.max) else None + latestDataItem.map { case (key, data) => - val createTimeAndDuration = key.split(SECRET_DATA_KEY_REGEX_HADOOP_TOKENS, 2) + val createTimeAndDuration = key.split(SECRET_DATA_ITEM_KEY_REGEX_HADOOP_TOKENS, 2) val expireTime = createTimeAndDuration(0).toLong + createTimeAndDuration(1).toLong val creds = new Credentials creds.readTokenStorageStream(new DataInputStream(new ByteArrayInputStream( @@ -172,6 +173,7 @@ private class StarterTask(secret: Secret, private class RenewTask(renew: Renew, hadoopConf: Configuration, refreshService: ActorRef, + kubernetesClient: KubernetesClient, clock: Clock) extends Runnable with Logging { private var hasError = false @@ -270,17 +272,21 @@ private class RenewTask(renew: Renew, private def writeTokensToSecret(tokenToExpire: Map[Token[_ <: TokenIdentifier], Long], nowMillis: Long) = { val durationUntilExpire = tokenToExpire.values.min - nowMillis - val key = s"$SECRET_DATA_KEY_PREFIX_HADOOP_TOKENS$nowMillis-$durationUntilExpire" + val key = s"$SECRET_DATA_ITEM_KEY_PREFIX_HADOOP_TOKENS$nowMillis-$durationUntilExpire" val credentials = new Credentials() tokenToExpire.keys.foreach(token => credentials.addToken(token.getService, token)) val serialized = serializeCredentials(credentials) val value = Base64.encodeBase64String(serialized) - val builder = new SecretBuilder(renew.secret) - .addToData(key, value) - val hadoopSecretData = builder.getData.asScala.filterKeys( - _.startsWith(SECRET_DATA_KEY_PREFIX_HADOOP_TOKENS)) - hadoopSecretData.keys.dropRight(2).foreach(builder.removeFromData) - builder.build() + val secret = renew.secret + val editor = kubernetesClient.secrets + .inNamespace(secret.getMetadata.getNamespace) + .withName(secret.getMetadata.getName) + .edit() + editor.addToData(key, value) + val dataItemKeys = editor.getData.keySet().asScala.filter( + _.startsWith(SECRET_DATA_ITEM_KEY_PREFIX_HADOOP_TOKENS)).toSeq.sorted + dataItemKeys.dropRight(2).foreach(editor.removeFromData) + editor.done } private def serializeCredentials(credentials: Credentials) = { @@ -311,11 +317,11 @@ private case class StopRefresh(secret: Secret) extends Command private object TokenRefreshService { - def apply(system: ActorSystem) : ActorRef = { + def apply(system: ActorSystem, kubernetesClient: KubernetesClient) : ActorRef = { UserGroupInformation.loginUserFromKeytab( REFRESH_SERVER_KERBEROS_PRINCIPAL, REFRESH_SERVER_KERBEROS_KEYTAB_PATH) - val actor = system.actorOf(Props[TokenRefreshService]) + val actor = system.actorOf(Props(classOf[TokenRefreshService], kubernetesClient)) val duration = Duration(REFRESH_SERVER_KERBEROS_RELOGIN_PERIOD_MILLIS, TimeUnit.MILLISECONDS) system.scheduler.schedule(duration, duration, actor, Relogin) actor diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala index 02694dc45f159..e9c1c8fe4dde3 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala @@ -24,8 +24,8 @@ package object constants { val SECRET_LABEL_KEY_REFRESH_HADOOP_TOKENS = "refresh-hadoop-tokens" val SECRET_LABEL_VALUE_REFRESH_HADOOP_TOKENS = "yes" - val SECRET_DATA_KEY_PREFIX_HADOOP_TOKENS = "hadoop-tokens-" - val SECRET_DATA_KEY_REGEX_HADOOP_TOKENS = "hadoop-tokens-(\\d+)-(\\d+)" + val SECRET_DATA_ITEM_KEY_PREFIX_HADOOP_TOKENS = "hadoop-tokens-" + val SECRET_DATA_ITEM_KEY_REGEX_HADOOP_TOKENS = "hadoop-tokens-(\\d+)-(\\d+)" val STARTER_TASK_INITIAL_DELAY_MILLIS = 0L From 5162339cbbc602af7b4eae224f874475da90793e Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Thu, 5 Oct 2017 10:47:52 -0700 Subject: [PATCH 23/43] Add a comment --- .../apache/spark/security/kubernetes/TokenRefreshService.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index 02ec560ed0b43..74f65af3c2a6b 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -285,6 +285,7 @@ private class RenewTask(renew: Renew, editor.addToData(key, value) val dataItemKeys = editor.getData.keySet().asScala.filter( _.startsWith(SECRET_DATA_ITEM_KEY_PREFIX_HADOOP_TOKENS)).toSeq.sorted + // Remove data items except the latest two data items. dataItemKeys.dropRight(2).foreach(editor.removeFromData) editor.done } From 196cd8a0f7bc86e4c74fb324e39b42b16d5ab3be Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Thu, 5 Oct 2017 11:19:09 -0700 Subject: [PATCH 24/43] Keep only secret metadata in memory --- .../kubernetes/TokenRefreshService.scala | 37 ++++++++++--------- 1 file changed, 19 insertions(+), 18 deletions(-) diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index 74f65af3c2a6b..664ff813e5128 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -21,11 +21,11 @@ import java.security.PrivilegedExceptionAction import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ -import scala.collection.{SortedSet, mutable} +import scala.collection.mutable import scala.concurrent.duration.Duration import scala.concurrent.ExecutionContext.Implicits.global import akka.actor.{Actor, ActorRef, ActorSystem, Cancellable, Props} -import io.fabric8.kubernetes.api.model.{Secret, SecretBuilder} +import io.fabric8.kubernetes.api.model.{ObjectMeta, Secret} import io.fabric8.kubernetes.client.KubernetesClient import org.apache.commons.codec.binary.Base64 import org.apache.hadoop.conf.Configuration @@ -64,18 +64,18 @@ private class TokenRefreshService(kubernetesClient: KubernetesClient) extends Ac } private def addStarterTask(secret: Secret) = { - secretUidToTaskHandle.getOrElseUpdate(getSecretUid(secret), { + secretUidToTaskHandle.getOrElseUpdate(getSecretUid(secret.getMetadata), { val task = new StarterTask(secret, hadoopConf, self, clock) val cancellable = scheduler.scheduleOnce( Duration(STARTER_TASK_INITIAL_DELAY_MILLIS, TimeUnit.MILLISECONDS), task) - logInfo(s"Started refresh of tokens in ${secret.getMetadata.getSelfLink} with ${cancellable}") + logInfo(s"Started refresh of tokens in ${secret.getMetadata.getSelfLink} with $cancellable") cancellable }) } private def removeRefreshTask(secret: Secret) : Unit = { - val uid = getSecretUid(secret) + val uid = getSecretUid(secret.getMetadata) secretUidToTaskHandle.remove(uid).foreach(cancellable => { logInfo(s"Canceling refresh of tokens in ${secret.getMetadata.getSelfLink}") cancellable.cancel() @@ -83,7 +83,7 @@ private class TokenRefreshService(kubernetesClient: KubernetesClient) extends Ac } private def updateSecretsToTrack(currentSecrets: List[Secret]) : Unit = { - val secretByUid = currentSecrets.map(secret => (getSecretUid(secret), secret)).toMap + val secretByUid = currentSecrets.map(secret => (getSecretUid(secret.getMetadata), secret)).toMap val currentUids = secretByUid.keySet val priorUids = secretUidToTaskHandle.keySet val uidsToAdd = currentUids -- priorUids @@ -94,7 +94,7 @@ private class TokenRefreshService(kubernetesClient: KubernetesClient) extends Ac } private def scheduleRenewTask(renew: Renew) = { - val uid = getSecretUid(renew.secret) + val uid = getSecretUid(renew.secretMeta) if (secretUidToTaskHandle.get(uid).nonEmpty) { val numConsecutiveErrors = renew.numConsecutiveErrors if (numConsecutiveErrors < RENEW_TASK_MAX_CONSECUTIVE_ERRORS) { @@ -103,22 +103,22 @@ private class TokenRefreshService(kubernetesClient: KubernetesClient) extends Ac val durationTillRenew = math.max(0L, renewTime - clock.nowInMillis()) val task = new RenewTask(renew, hadoopConf, self, kubernetesClient, clock) logInfo(s"Scheduling refresh of tokens with " + - s"${renew.secret.getMetadata.getSelfLink} at now + $durationTillRenew millis.") + s"${renew.secretMeta.getSelfLink} at now + $durationTillRenew millis.") val cancellable = scheduler.scheduleOnce( Duration(durationTillRenew, TimeUnit.MILLISECONDS), task) secretUidToTaskHandle.put(uid, cancellable) } else { - logWarning(s"Got too many errors for ${renew.secret.getMetadata.getSelfLink}. Abandoning.") + logWarning(s"Got too many errors for ${renew.secretMeta.getSelfLink}. Abandoning.") val maybeCancellable = secretUidToTaskHandle.remove(uid) maybeCancellable.foreach(_.cancel()) } } else { logWarning(s"Could not find an entry for renew task" + - s" ${renew.secret.getMetadata.getSelfLink}. Maybe the secret got deleted") + s" ${renew.secretMeta.getSelfLink}. Maybe the secret got deleted") } } - private def getSecretUid(secret: Secret) = secret.getMetadata.getUid + private def getSecretUid(secret: ObjectMeta) = secret.getUid } private class ReloginTask extends Runnable { @@ -147,7 +147,8 @@ private class StarterTask(secret: Secret, } logInfo(s"Initial renew resulted with $tokenToExpireTime. Next expire time $nextExpireTime") val numConsecutiveErrors = if (hasError) 1 else 0 - refreshService ! Renew(nextExpireTime, tokenToExpireTime, secret, numConsecutiveErrors) + refreshService ! Renew(nextExpireTime, tokenToExpireTime, secret.getMetadata, + numConsecutiveErrors) } private def readTokensFromSecret() : Map[Token[_ <: TokenIdentifier], Long] = { @@ -194,10 +195,10 @@ private class RenewTask(renew: Renew, val nextExpireTime = newExpireTimeByToken.values.min logInfo(s"Renewed with the result $newExpireTimeByToken. Next expire time $nextExpireTime") val numConsecutiveErrors = if (hasError) renew.numConsecutiveErrors + 1 else 0 - refreshService ! Renew(nextExpireTime, newExpireTimeByToken, renew.secret, + refreshService ! Renew(nextExpireTime, newExpireTimeByToken, renew.secretMeta, numConsecutiveErrors) } else { - logWarning(s"Got an empty token list with ${renew.secret.getMetadata.getSelfLink}") + logWarning(s"Got an empty token list with ${renew.secretMeta.getSelfLink}") } } @@ -277,10 +278,10 @@ private class RenewTask(renew: Renew, tokenToExpire.keys.foreach(token => credentials.addToken(token.getService, token)) val serialized = serializeCredentials(credentials) val value = Base64.encodeBase64String(serialized) - val secret = renew.secret + val secretMeta = renew.secretMeta val editor = kubernetesClient.secrets - .inNamespace(secret.getMetadata.getNamespace) - .withName(secret.getMetadata.getName) + .inNamespace(secretMeta.getNamespace) + .withName(secretMeta.getName) .edit() editor.addToData(key, value) val dataItemKeys = editor.getData.keySet().asScala.filter( @@ -312,7 +313,7 @@ private case class UpdateSecretsToTrack(secrets: List[Secret]) extends Command private case class StartRefresh(secret: Secret) extends Command private case class Renew(expireTime: Long, tokenToExpireTime: Map[Token[_ <: TokenIdentifier], Long], - secret: Secret, + secretMeta: ObjectMeta, numConsecutiveErrors: Int) extends Command private case class StopRefresh(secret: Secret) extends Command From 56ef8e66ed6b32970950f238e786292e0ae4ffac Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Thu, 5 Oct 2017 12:00:09 -0700 Subject: [PATCH 25/43] Fix a regex match bug --- .../security/kubernetes/TokenRefreshService.scala | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index 664ff813e5128..b1fb5d979a5e9 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -19,6 +19,7 @@ package org.apache.spark.security.kubernetes import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} import java.security.PrivilegedExceptionAction import java.util.concurrent.TimeUnit +import java.util.regex.Pattern import scala.collection.JavaConverters._ import scala.collection.mutable @@ -145,7 +146,6 @@ private class StarterTask(secret: Secret, hasError = true getRetryTime } - logInfo(s"Initial renew resulted with $tokenToExpireTime. Next expire time $nextExpireTime") val numConsecutiveErrors = if (hasError) 1 else 0 refreshService ! Renew(nextExpireTime, tokenToExpireTime, secret.getMetadata, numConsecutiveErrors) @@ -157,8 +157,12 @@ private class StarterTask(secret: Secret, val latestDataItem = if (dataItems.nonEmpty) Some(dataItems.max) else None latestDataItem.map { case (key, data) => - val createTimeAndDuration = key.split(SECRET_DATA_ITEM_KEY_REGEX_HADOOP_TOKENS, 2) - val expireTime = createTimeAndDuration(0).toLong + createTimeAndDuration(1).toLong + val matcher = TokenRefreshService.hadoopTokenPattern.matcher(key) + val matches = matcher.matches() + logInfo(s"Matching secret data $key, result $matches") + val createTime = matcher.group(1).toLong + val duration = matcher.group(2).toLong + val expireTime = createTime + duration val creds = new Credentials creds.readTokenStorageStream(new DataInputStream(new ByteArrayInputStream( Base64.decodeBase64(data)))) @@ -216,6 +220,7 @@ private class RenewTask(renew: Renew, val maxDate = identifier.getMaxDate if (maxDate - expireTime < RENEW_TASK_REMAINING_TIME_BEFORE_NEW_TOKEN_MILLIS || maxDate <= nowMills) { + logInfo(s"Obtaining a new token") val newToken = obtainNewToken(token, identifier) logInfo(s"Obtained token $newToken") newToken @@ -319,6 +324,8 @@ private case class StopRefresh(secret: Secret) extends Command private object TokenRefreshService { + val hadoopTokenPattern = Pattern.compile(SECRET_DATA_ITEM_KEY_REGEX_HADOOP_TOKENS) + def apply(system: ActorSystem, kubernetesClient: KubernetesClient) : ActorRef = { UserGroupInformation.loginUserFromKeytab( REFRESH_SERVER_KERBEROS_PRINCIPAL, From 93b2acf21b09fad3b553401ebfb60aff26f769b0 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Thu, 5 Oct 2017 13:53:07 -0700 Subject: [PATCH 26/43] Tested --- .../kubernetes/token-refresh-server/README.md | 6 +++--- .../src/main/assembly/assembly.xml | 8 +++---- .../src/main/docker/Dockerfile | 2 +- .../security/kubernetes/SecretFinder.scala | 1 + .../kubernetes/TokenRefreshService.scala | 21 ++++++++++++++----- 5 files changed, 25 insertions(+), 13 deletions(-) diff --git a/resource-managers/kubernetes/token-refresh-server/README.md b/resource-managers/kubernetes/token-refresh-server/README.md index 63515f145d1ef..1d6e1a16f3daf 100644 --- a/resource-managers/kubernetes/token-refresh-server/README.md +++ b/resource-managers/kubernetes/token-refresh-server/README.md @@ -14,10 +14,10 @@ To build the refresh server jar, simply run Maven. For example: mvn clean package -The target directory will have a uber jar that includes the project class files as well as -3rd party classes from the dependencies. The jar name would end with `-bin.jar`. For example: +The target directory will have a tarball that includes the project jar file as well as +3rd party dependency jars. The tarball name would end with `-assembly.tar.gz`. For example: - target/token-refresh-server-kubernetes_2.11-2.2.0-k8s-0.3.0-SNAPSHOT-bin.jar + target/token-refresh-server-kubernetes_2.11-2.2.0-k8s-0.3.0-SNAPSHOT-assembly.tar.gz # Running the Refresh Server diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/assembly/assembly.xml b/resource-managers/kubernetes/token-refresh-server/src/main/assembly/assembly.xml index 920ff527b0dd4..2bbc040bc5fa9 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/assembly/assembly.xml +++ b/resource-managers/kubernetes/token-refresh-server/src/main/assembly/assembly.xml @@ -15,18 +15,18 @@ ~ limitations under the License. --> - bin + assembly - jar + tar.gz false - true + false compile - true + false provided diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/docker/Dockerfile b/resource-managers/kubernetes/token-refresh-server/src/main/docker/Dockerfile index 1d4a1415e27af..f6e330a24c57a 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/docker/Dockerfile +++ b/resource-managers/kubernetes/token-refresh-server/src/main/docker/Dockerfile @@ -28,7 +28,7 @@ RUN apk upgrade --no-cache && \ mkdir -p /opt/token-refresh-server/jars && \ mkdir -p /opt/token-refresh-server/work-dir -ADD target/token-refresh-server-kubernetes_2.11-2.2.0-k8s-0.3.0-SNAPSHOT-bin.jar \ +ADD target/token-refresh-server-kubernetes_2.11-2.2.0-k8s-0.3.0-SNAPSHOT-assembly.tar.gz \ /opt/token-refresh-server/jars WORKDIR /opt/token-refresh-server/work-dir diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala index 0bb3f2d55bc2c..e43d43fac78bd 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala @@ -64,6 +64,7 @@ private class SecretWatcher(renewService: ActorRef) extends Watcher[Secret] with case Action.DELETED => logInfo(s"Found ${secret.getMetadata.getSelfLink} deleted") renewService ! StopRefresh(secret) + case _ => // Do nothing } } diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index b1fb5d979a5e9..7557daf570355 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -59,6 +59,11 @@ private class TokenRefreshService(kubernetesClient: KubernetesClient) extends Ac scheduleRenewTask(renew) } + override def postStop(): Unit = { + super.postStop() + secretUidToTaskHandle.values.map(_.cancel) + } + private def launchReloginTask() = { val task = new ReloginTask scheduler.scheduleOnce(Duration(0L, TimeUnit.MILLISECONDS), task) @@ -195,9 +200,11 @@ private class RenewTask(renew: Renew, .toMap if (newExpireTimeByToken.nonEmpty) { val newTokens = newExpireTimeByToken.keySet -- renew.tokenToExpireTime.keySet - if (newTokens.nonEmpty) writeTokensToSecret(newExpireTimeByToken, nowMillis) + if (newTokens.nonEmpty) { + writeTokensToSecret(newExpireTimeByToken, nowMillis) + } val nextExpireTime = newExpireTimeByToken.values.min - logInfo(s"Renewed with the result $newExpireTimeByToken. Next expire time $nextExpireTime") + logInfo(s"Renewed tokens $newExpireTimeByToken. Next expire time $nextExpireTime") val numConsecutiveErrors = if (hasError) renew.numConsecutiveErrors + 1 else 0 refreshService ! Renew(nextExpireTime, newExpireTimeByToken, renew.secretMeta, numConsecutiveErrors) @@ -220,7 +227,8 @@ private class RenewTask(renew: Renew, val maxDate = identifier.getMaxDate if (maxDate - expireTime < RENEW_TASK_REMAINING_TIME_BEFORE_NEW_TOKEN_MILLIS || maxDate <= nowMills) { - logInfo(s"Obtaining a new token") + logDebug(s"Obtaining a new token with maxData $maxDate," + + s" expireTime $expireTime, now $nowMills") val newToken = obtainNewToken(token, identifier) logInfo(s"Obtained token $newToken") newToken @@ -237,8 +245,10 @@ private class RenewTask(renew: Renew, deadline: Long, nowMillis: Long) = { if (expireTime <= deadline || expireTime <= nowMillis) { try { + logDebug(s"Renewing token $token with current expire time $expireTime," + + s" deadline $deadline, now $nowMillis") val newExpireTime = token.renew(hadoopConf) - logInfo(s"Renewed token $token. Next expire time $newExpireTime") + logDebug(s"Renewed token $token. Next expire time $newExpireTime") newExpireTime } catch { case t: Throwable => @@ -276,7 +286,7 @@ private class RenewTask(renew: Renew, } private def writeTokensToSecret(tokenToExpire: Map[Token[_ <: TokenIdentifier], Long], - nowMillis: Long) = { + nowMillis: Long) : Unit = { val durationUntilExpire = tokenToExpire.values.min - nowMillis val key = s"$SECRET_DATA_ITEM_KEY_PREFIX_HADOOP_TOKENS$nowMillis-$durationUntilExpire" val credentials = new Credentials() @@ -294,6 +304,7 @@ private class RenewTask(renew: Renew, // Remove data items except the latest two data items. dataItemKeys.dropRight(2).foreach(editor.removeFromData) editor.done + logInfo(s"Wrote new tokens $tokenToExpire to a data item $key in ${secretMeta.getSelfLink}") } private def serializeCredentials(credentials: Credentials) = { From ba2e79a45679ddab1ff28c219e509408b6baa1bd Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Thu, 5 Oct 2017 14:02:32 -0700 Subject: [PATCH 27/43] Updated parent version --- resource-managers/kubernetes/token-refresh-server/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/token-refresh-server/pom.xml b/resource-managers/kubernetes/token-refresh-server/pom.xml index 3c53d447ab717..d99276c526510 100644 --- a/resource-managers/kubernetes/token-refresh-server/pom.xml +++ b/resource-managers/kubernetes/token-refresh-server/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.2.0-k8s-0.3.0-SNAPSHOT + 2.2.0-k8s-0.5.0-SNAPSHOT ../../../pom.xml From 1d745795c49012161b52f3bd97dae3e9b9defe14 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Fri, 6 Oct 2017 11:58:49 -0700 Subject: [PATCH 28/43] Address review comments --- resource-managers/kubernetes/README.md | 3 +- ...ubernetes-hadoop-token-refresh-server.yaml | 2 +- .../kubernetes/TokenRefreshService.scala | 32 +++++++++++++------ 3 files changed, 25 insertions(+), 12 deletions(-) diff --git a/resource-managers/kubernetes/README.md b/resource-managers/kubernetes/README.md index 816f1ffdcf81c..e653a93f22bcc 100644 --- a/resource-managers/kubernetes/README.md +++ b/resource-managers/kubernetes/README.md @@ -34,7 +34,8 @@ Below is a list of the submodules for this cluster manager and what they do. * `core`: Implementation of the Kubernetes cluster manager support. * `token-refresh-server`: Extra Kubernetes service that refreshes Hadoop - tokens for long-running Spark jobs accessing secure data source like HDFS. + tokens for long-running Spark jobs accessing secure data sources like + Kerberized HDFS. * `integration-tests`: Integration tests for the project. * `docker-minimal-bundle`: Base Dockerfiles for the driver and the executors. The Dockerfiles are used for integration tests as well as being provided in packaged distributions of Spark. diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/conf/kubernetes-hadoop-token-refresh-server.yaml b/resource-managers/kubernetes/token-refresh-server/src/main/conf/kubernetes-hadoop-token-refresh-server.yaml index e58cf6916fbd3..afb646ebfb1ec 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/conf/kubernetes-hadoop-token-refresh-server.yaml +++ b/resource-managers/kubernetes/token-refresh-server/src/main/conf/kubernetes-hadoop-token-refresh-server.yaml @@ -15,7 +15,7 @@ # limitations under the License. # --- -apiVersion: extensions/v1beta1 +apiVersion: apps/v1beta1 kind: Deployment metadata: name: hadoop-token-refresh-server diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index 7557daf570355..4a78faf9107c9 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -50,7 +50,7 @@ private class TokenRefreshService(kubernetesClient: KubernetesClient) extends Ac case Relogin => launchReloginTask() case StartRefresh(secret) => - addStarterTask(secret) + startRefresh(secret) case StopRefresh(secret) => removeRefreshTask(secret) case UpdateSecretsToTrack(secrets) => @@ -69,7 +69,12 @@ private class TokenRefreshService(kubernetesClient: KubernetesClient) extends Ac scheduler.scheduleOnce(Duration(0L, TimeUnit.MILLISECONDS), task) } - private def addStarterTask(secret: Secret) = { + private def startRefresh(secret: Secret) = { + recentlyAddedSecretUids.add(getSecretUid(secret.getMetadata)) + addRefreshTask(secret) + } + + private def addRefreshTask(secret: Secret) = { secretUidToTaskHandle.getOrElseUpdate(getSecretUid(secret.getMetadata), { val task = new StarterTask(secret, hadoopConf, self, clock) val cancellable = scheduler.scheduleOnce( @@ -89,13 +94,14 @@ private class TokenRefreshService(kubernetesClient: KubernetesClient) extends Ac } private def updateSecretsToTrack(currentSecrets: List[Secret]) : Unit = { - val secretByUid = currentSecrets.map(secret => (getSecretUid(secret.getMetadata), secret)).toMap - val currentUids = secretByUid.keySet + val secretsByUids = currentSecrets.map(secret => (getSecretUid(secret.getMetadata), secret)) + .toMap + val currentUids = secretsByUids.keySet val priorUids = secretUidToTaskHandle.keySet val uidsToAdd = currentUids -- priorUids - uidsToAdd.foreach(uid => addStarterTask(secretByUid(uid))) + uidsToAdd.foreach(uid => addRefreshTask(secretsByUids(uid))) val uidsToRemove = priorUids -- currentUids -- recentlyAddedSecretUids - uidsToRemove.foreach(uid => removeRefreshTask(secretByUid(uid))) + uidsToRemove.foreach(uid => removeRefreshTask(secretsByUids(uid))) recentlyAddedSecretUids.clear() } @@ -216,13 +222,16 @@ private class RenewTask(renew: Renew, private def refresh(token: Token[_ <: TokenIdentifier], expireTime: Long, deadline: Long, nowMillis: Long) = { val maybeNewToken = maybeObtainNewToken(token, expireTime, nowMillis) - val maybeNewExpireTime = maybeRenewExpireTime(maybeNewToken, expireTime, deadline, nowMillis) + val maybeNewExpireTime = maybeGetNewExpireTime(maybeNewToken, expireTime, deadline, nowMillis) (maybeNewToken, maybeNewExpireTime) } private def maybeObtainNewToken(token: Token[_ <: TokenIdentifier], expireTime: Long, nowMills: Long) = { val maybeNewToken = if (token.getKind.equals(DelegationTokenIdentifier.HDFS_DELEGATION_KIND)) { + // The token can casted to AbstractDelegationTokenIdentifier below only if the token kind + // is HDFS_DELEGATION_KIND, according to the YARN resource manager code. See if this can be + // generalized beyond HDFS tokens. val identifier = token.decodeIdentifier().asInstanceOf[AbstractDelegationTokenIdentifier] val maxDate = identifier.getMaxDate if (maxDate - expireTime < RENEW_TASK_REMAINING_TIME_BEFORE_NEW_TOKEN_MILLIS || @@ -241,8 +250,9 @@ private class RenewTask(renew: Renew, maybeNewToken } - private def maybeRenewExpireTime(token: Token[_ <: TokenIdentifier], expireTime: Long, - deadline: Long, nowMillis: Long) = { + private def maybeGetNewExpireTime(token: Token[_ <: TokenIdentifier], expireTime: Long, + deadline: Long, + nowMillis: Long) = { if (expireTime <= deadline || expireTime <= nowMillis) { try { logDebug(s"Renewing token $token with current expire time $expireTime," + @@ -301,7 +311,9 @@ private class RenewTask(renew: Renew, editor.addToData(key, value) val dataItemKeys = editor.getData.keySet().asScala.filter( _.startsWith(SECRET_DATA_ITEM_KEY_PREFIX_HADOOP_TOKENS)).toSeq.sorted - // Remove data items except the latest two data items. + // Remove data items except the latest two data items. A K8s secret can hold only up to 1 MB + // data. We need to remove old data items. We keep the latest two items to avoid race conditions + // where some newly launching executors may access the previous token. dataItemKeys.dropRight(2).foreach(editor.removeFromData) editor.done logInfo(s"Wrote new tokens $tokenToExpire to a data item $key in ${secretMeta.getSelfLink}") From 95e68d32a7266ac6b6084ee39d2d95953cb8dbbd Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Fri, 6 Oct 2017 12:02:58 -0700 Subject: [PATCH 29/43] Add TODO for token status rest endpoint --- .../apache/spark/security/kubernetes/TokenRefreshServer.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala index 632c709b10770..f8c7b3d26f2b0 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala @@ -48,6 +48,9 @@ private class Server { } } +/* + * TODO: Support REST endpoint for checking status of tokens. + */ object TokenRefreshServer { private class Arguments(args: List[String]) { From a006233f028445db3f194ab9f5a13b4edde4445f Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Fri, 6 Oct 2017 12:56:30 -0700 Subject: [PATCH 30/43] Address review comments --- .../spark/security/kubernetes/TokenRefreshService.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index 4a78faf9107c9..1c565eeb3c2f3 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -50,7 +50,7 @@ private class TokenRefreshService(kubernetesClient: KubernetesClient) extends Ac case Relogin => launchReloginTask() case StartRefresh(secret) => - startRefresh(secret) + startRefreshTask(secret) case StopRefresh(secret) => removeRefreshTask(secret) case UpdateSecretsToTrack(secrets) => @@ -69,7 +69,7 @@ private class TokenRefreshService(kubernetesClient: KubernetesClient) extends Ac scheduler.scheduleOnce(Duration(0L, TimeUnit.MILLISECONDS), task) } - private def startRefresh(secret: Secret) = { + private def startRefreshTask(secret: Secret) = { recentlyAddedSecretUids.add(getSecretUid(secret.getMetadata)) addRefreshTask(secret) } From 9dc83453d369e6063a25fdf485225a8b50bf65c2 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Tue, 10 Oct 2017 15:33:33 -0700 Subject: [PATCH 31/43] Address review comments --- resource-managers/kubernetes/README.md | 2 +- resource-managers/kubernetes/token-refresh-server/README.md | 6 +++--- .../token-refresh-server/src/main/docker/Dockerfile | 2 +- .../spark/security/kubernetes/TokenRefreshService.scala | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/resource-managers/kubernetes/README.md b/resource-managers/kubernetes/README.md index e653a93f22bcc..a83a0311c0974 100644 --- a/resource-managers/kubernetes/README.md +++ b/resource-managers/kubernetes/README.md @@ -34,7 +34,7 @@ Below is a list of the submodules for this cluster manager and what they do. * `core`: Implementation of the Kubernetes cluster manager support. * `token-refresh-server`: Extra Kubernetes service that refreshes Hadoop - tokens for long-running Spark jobs accessing secure data sources like + tokens for long-running Spark jobs that access secure data sources like Kerberized HDFS. * `integration-tests`: Integration tests for the project. * `docker-minimal-bundle`: Base Dockerfiles for the driver and the executors. The Dockerfiles are used for integration diff --git a/resource-managers/kubernetes/token-refresh-server/README.md b/resource-managers/kubernetes/token-refresh-server/README.md index 1d6e1a16f3daf..9d313ac637d33 100644 --- a/resource-managers/kubernetes/token-refresh-server/README.md +++ b/resource-managers/kubernetes/token-refresh-server/README.md @@ -50,9 +50,9 @@ To run the server, follow the steps below. kubectl create secret generic hadoop-token-refresh-server-kerberos-keytab \ --from-file /mnt/secrets/krb5.keytab -5. Create a k8s `service account` and `clusterrolebinding` that the service pod will use. - The service account should have `edit` capability for job `secret`s that contains - the Hadoop delegation tokens. +5. Optionally, create a k8s `service account` and `clusterrolebinding` that + the service pod will use. The service account should have `edit` capability for + job `secret`s that contains the Hadoop delegation tokens. 6. Finally, edit the config file for k8s `deployment` and launch the service pod using the deployment. The config file should include the right docker image tag diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/docker/Dockerfile b/resource-managers/kubernetes/token-refresh-server/src/main/docker/Dockerfile index f6e330a24c57a..5696a5cff58ec 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/docker/Dockerfile +++ b/resource-managers/kubernetes/token-refresh-server/src/main/docker/Dockerfile @@ -28,7 +28,7 @@ RUN apk upgrade --no-cache && \ mkdir -p /opt/token-refresh-server/jars && \ mkdir -p /opt/token-refresh-server/work-dir -ADD target/token-refresh-server-kubernetes_2.11-2.2.0-k8s-0.3.0-SNAPSHOT-assembly.tar.gz \ +ADD target/token-refresh-server-kubernetes_2.11-2.2.0-k8s-0.5.0-SNAPSHOT-assembly.tar.gz \ /opt/token-refresh-server/jars WORKDIR /opt/token-refresh-server/work-dir diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index 1c565eeb3c2f3..1e354688d6d5d 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -281,7 +281,7 @@ private class RenewTask(renew: Renew, } else { realUser.toString } - val credentials = new Credentials() + val credentials = new Credentials val ugi = UserGroupInformation.createProxyUser(user, UserGroupInformation.getLoginUser) val newToken = ugi.doAs(new PrivilegedExceptionAction[Token[_ <: TokenIdentifier]] { From f4d5ee962b127c6092533c7c38026a4e2e14ef91 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Tue, 10 Oct 2017 16:37:06 -0700 Subject: [PATCH 32/43] Support configuration --- .../kubernetes/token-refresh-server/README.md | 19 ++++++--- .../src/main/conf/application.conf | 39 +++++++++++++++++++ ...ubernetes-hadoop-token-refresh-server.yaml | 14 +++++-- .../src/main/docker/Dockerfile | 2 +- .../security/kubernetes/SecretFinder.scala | 9 ++++- .../kubernetes/TokenRefreshServer.scala | 22 +++++++++-- .../kubernetes/TokenRefreshService.scala | 6 ++- .../spark/security/kubernetes/constants.scala | 1 - 8 files changed, 94 insertions(+), 18 deletions(-) create mode 100644 resource-managers/kubernetes/token-refresh-server/src/main/conf/application.conf diff --git a/resource-managers/kubernetes/token-refresh-server/README.md b/resource-managers/kubernetes/token-refresh-server/README.md index 9d313ac637d33..37d58109f8c62 100644 --- a/resource-managers/kubernetes/token-refresh-server/README.md +++ b/resource-managers/kubernetes/token-refresh-server/README.md @@ -30,31 +30,38 @@ To run the server, follow the steps below. docker tag hadoop-token-refresh-server:latest : docker push : -2. Create a k8s `configmap` containing Hadoop config files. This should enable Kerberos and secure Hadoop. +2. Edit the main application config file, src/main/conf/application.conf + and create a `configmap`: + + kubectl create configmap hadoop-token-refresh-server-application-conf \ + --from-file=src/main/conf/application.conf + +3. Create another k8s `configmap` containing Hadoop config files. This should enable Kerberos and secure Hadoop. It should also include the Hadoop servers that would issue delegation tokens such as the HDFS namenode address: kubectl create configmap hadoop-token-refresh-server-hadoop-config \ --from-file=/usr/local/hadoop/conf/core-site.xml -3. Create another k8s `configmap` containing Kerberos config files. This should include +4. Create yet another k8s `configmap` containing Kerberos config files. This should include the kerberos server address and the correct realm name for Kerberos principals: kubectl create configmap hadoop-token-refresh-server-kerberos-config \ --from-file=/etc/krb5.conf -4. Create a k8s `secret` containing the Kerberos keytab file. The keytab file should include +5. Create a k8s `secret` containing the Kerberos keytab file. The keytab file should include the password for the system user Kerberos principal that the refresh server is using to - extend Hadoop delegation tokens. + extend Hadoop delegation tokens. See + hadoop-token-refresh-server.kerberosPrincipal in the application.conf. kubectl create secret generic hadoop-token-refresh-server-kerberos-keytab \ --from-file /mnt/secrets/krb5.keytab -5. Optionally, create a k8s `service account` and `clusterrolebinding` that +6. Optionally, create a k8s `service account` and `clusterrolebinding` that the service pod will use. The service account should have `edit` capability for job `secret`s that contains the Hadoop delegation tokens. -6. Finally, edit the config file for k8s `deployment` and launch the service pod +7. Finally, edit the config file for k8s `deployment` and launch the service pod using the deployment. The config file should include the right docker image tag and the correct k8s `service account` name. diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/conf/application.conf b/resource-managers/kubernetes/token-refresh-server/src/main/conf/application.conf new file mode 100644 index 0000000000000..76f47ef73334e --- /dev/null +++ b/resource-managers/kubernetes/token-refresh-server/src/main/conf/application.conf @@ -0,0 +1,39 @@ +# +# 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. +# +# Main application config file for the Hadoop token refresh server. Override the values below +# as needed. +hadoop-token-refresh-server { + + # Kerberos principal that the refresh server should use as its login user. This principal should + # match the keytab file used for the refresh server. + # For a token to be renewed for the next 24 hours by the refresh server, the token should + # designate this refresh server principal as the renewer. To allow a brand new token to be + # obtained by the refresh server, the HDFS namenode configuration should specify this refresh + # server principal as the special proxy for the job users. See + # https://hadoop.apache.org/docs/r2.7.2/hadoop-project-dist/hadoop-common/Superusers.html#Configurations + # for details. + kerberosPrincipal = "MY-REFRESH-SERVER-KERBEROS-PRINCIPAL" + + # Set this to true if the refresh server should scan secrets across all namespaces. Set it to + # false and specify namespaceToScan if the refresh server should scan secrets only from + # the specific namespace. + scanAllNamespaces = true + + # Effective only if scanAllNamespaces is false. A specific namespace that the refresh server + # should scan secrets from. + namespaceToScan = "default" +} diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/conf/kubernetes-hadoop-token-refresh-server.yaml b/resource-managers/kubernetes/token-refresh-server/src/main/conf/kubernetes-hadoop-token-refresh-server.yaml index afb646ebfb1ec..ef2ef009e6615 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/conf/kubernetes-hadoop-token-refresh-server.yaml +++ b/resource-managers/kubernetes/token-refresh-server/src/main/conf/kubernetes-hadoop-token-refresh-server.yaml @@ -28,6 +28,9 @@ spec: spec: serviceAccountName: YOUR-SERVICE-ACCOUNT volumes: + - name: application-conf + configMap: + name: hadoop-token-refresh-server-application-conf - name: kerberos-config configMap: name: hadoop-token-refresh-server-kerberos-config @@ -41,6 +44,8 @@ spec: - name: hadoop-token-refresh-server image: YOUR-REPO:YOUR-TAG env: + - name: APPLICATION_CONF_DIR + value: /etc/token-refresh-server/conf - name: HADOOP_CONF_DIR value: /etc/hadoop/conf - name: TOKEN_REFRESH_SERVER_ARGS @@ -53,13 +58,16 @@ spec: cpu: 100m memory: 512Mi volumeMounts: - - name: kerberos-config - mountPath: '/etc/krb5.conf' - subPath: krb5.conf + - name: application-conf + mountPath: '/etc/token-refreh-server/conf readOnly: true - name: hadoop-config mountPath: '/etc/hadoop/conf' readOnly: true + - name: kerberos-config + mountPath: '/etc/krb5.conf' + subPath: krb5.conf + readOnly: true - name: kerberos-keytab mountPath: '/mnt/secrets/krb5.keytab' subPath: krb5.keytab diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/docker/Dockerfile b/resource-managers/kubernetes/token-refresh-server/src/main/docker/Dockerfile index 5696a5cff58ec..a89acaf3aec1d 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/docker/Dockerfile +++ b/resource-managers/kubernetes/token-refresh-server/src/main/docker/Dockerfile @@ -38,6 +38,6 @@ WORKDIR /opt/token-refresh-server/work-dir # -f src/main/docker/Dockerfile . CMD /sbin/tini -s -- /usr/bin/java \ - -cp $HADOOP_CONF_DIR:'/opt/token-refresh-server/jars/*' \ + -cp $APPLICATION_CONF_DIR:$HADOOP_CONF_DIR:'/opt/token-refresh-server/jars/*' \ org.apache.spark.security.kubernetes.TokenRefreshServer \ $TOKEN_REFRESH_SERVER_ARGS diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala index e43d43fac78bd..02099719d3383 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala @@ -83,9 +83,14 @@ private object SecretFinder { def selectSecrets(kubernetesClient: KubernetesClient): FilterWatchListDeletable[Secret, SecretList, lang.Boolean, Watch, Watcher[Secret]] = { - kubernetesClient + val selector = kubernetesClient .secrets() - .inAnyNamespace() + val namespacedSelector = if (Settings.shouldScanAllNamespaces) { + selector.inAnyNamespace() + } else { + selector.inNamespace(Settings.namespaceToScan) + } + namespacedSelector .withLabel(SECRET_LABEL_KEY_REFRESH_HADOOP_TOKENS, SECRET_LABEL_VALUE_REFRESH_HADOOP_TOKENS) } } diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala index f8c7b3d26f2b0..95e4fc6b4e190 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala @@ -16,15 +16,16 @@ */ package org.apache.spark.security.kubernetes +import scala.annotation.tailrec +import scala.collection.JavaConversions._ import scala.concurrent.Await import scala.concurrent.duration.Duration + import akka.actor.ActorSystem +import com.typesafe.config.{Config, ConfigFactory} import io.fabric8.kubernetes.client.DefaultKubernetesClient import org.apache.log4j.{Level, Logger} -import scala.annotation.tailrec - - private class Server { private val actorSystem = ActorSystem("TokenRefreshServer") @@ -32,6 +33,7 @@ private class Server { private var secretFinder : Option[SecretFinder] = None def start(): Unit = { + val config = ConfigFactory.load val renewService = TokenRefreshService(actorSystem, kubernetesClient) secretFinder = Some(SecretFinder(renewService, kubernetesClient)) } @@ -48,6 +50,20 @@ private class Server { } } +private object Settings { + + private val config: Config = ConfigFactory.load + + private val configKeyPrefix = "hadoop-token-refresh-server" + + val refreshServerKerberosPrincipal : String = config.getString( + s"$configKeyPrefix.kerberosPrincipal") + + val shouldScanAllNamespaces : Boolean = config.getBoolean(s"$configKeyPrefix.scanAllNamespaces") + + val namespaceToScan : String = config.getString(s"s$configKeyPrefix.namespaceToScan") +} + /* * TODO: Support REST endpoint for checking status of tokens. */ diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index 1e354688d6d5d..949db10a97d3b 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -25,7 +25,9 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import scala.concurrent.duration.Duration import scala.concurrent.ExecutionContext.Implicits.global + import akka.actor.{Actor, ActorRef, ActorSystem, Cancellable, Props} +import com.typesafe.config.Config import io.fabric8.kubernetes.api.model.{ObjectMeta, Secret} import io.fabric8.kubernetes.client.KubernetesClient import org.apache.commons.codec.binary.Base64 @@ -347,11 +349,11 @@ private case class StopRefresh(secret: Secret) extends Command private object TokenRefreshService { - val hadoopTokenPattern = Pattern.compile(SECRET_DATA_ITEM_KEY_REGEX_HADOOP_TOKENS) + val hadoopTokenPattern : Pattern = Pattern.compile(SECRET_DATA_ITEM_KEY_REGEX_HADOOP_TOKENS) def apply(system: ActorSystem, kubernetesClient: KubernetesClient) : ActorRef = { UserGroupInformation.loginUserFromKeytab( - REFRESH_SERVER_KERBEROS_PRINCIPAL, + Settings.refreshServerKerberosPrincipal, REFRESH_SERVER_KERBEROS_KEYTAB_PATH) val actor = system.actorOf(Props(classOf[TokenRefreshService], kubernetesClient)) val duration = Duration(REFRESH_SERVER_KERBEROS_RELOGIN_PERIOD_MILLIS, TimeUnit.MILLISECONDS) diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala index e9c1c8fe4dde3..b166446a2cbf0 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala @@ -18,7 +18,6 @@ package org.apache.spark.security.kubernetes package object constants { - val REFRESH_SERVER_KERBEROS_PRINCIPAL = "kimoonkim" val REFRESH_SERVER_KERBEROS_KEYTAB_PATH = "/mnt/secrets/krb5.keytab" val REFRESH_SERVER_KERBEROS_RELOGIN_PERIOD_MILLIS = 60 * 60 * 1000L From 1462d2cb0c92785adaccaecc3cc7ecac64316211 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Wed, 11 Oct 2017 11:54:00 -0700 Subject: [PATCH 33/43] Fix a typo --- .../apache/spark/security/kubernetes/TokenRefreshServer.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala index 95e4fc6b4e190..8ecbf9649e481 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala @@ -61,7 +61,7 @@ private object Settings { val shouldScanAllNamespaces : Boolean = config.getBoolean(s"$configKeyPrefix.scanAllNamespaces") - val namespaceToScan : String = config.getString(s"s$configKeyPrefix.namespaceToScan") + val namespaceToScan : String = config.getString(s"$configKeyPrefix.namespaceToScan") } /* From 193d0f9205ef98410b68a38daaebbbd0c793bf85 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Wed, 11 Oct 2017 18:45:14 -0700 Subject: [PATCH 34/43] Add some unit tests --- .../kubernetes/token-refresh-server/pom.xml | 11 ++ .../security/kubernetes/SecretFinder.scala | 81 ++++----- .../kubernetes/TokenRefreshServer.scala | 47 +++-- .../kubernetes/TokenRefreshService.scala | 17 +- .../spark/security/kubernetes/constants.scala | 2 - .../kubernetes/SecretFinderSuite.scala | 161 ++++++++++++++++++ .../kubernetes/TokenRefreshServerSuite.scala | 90 ++++++++++ 7 files changed, 348 insertions(+), 61 deletions(-) create mode 100644 resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/SecretFinderSuite.scala create mode 100644 resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/TokenRefreshServerSuite.scala diff --git a/resource-managers/kubernetes/token-refresh-server/pom.xml b/resource-managers/kubernetes/token-refresh-server/pom.xml index d99276c526510..ca53974285d4e 100644 --- a/resource-managers/kubernetes/token-refresh-server/pom.xml +++ b/resource-managers/kubernetes/token-refresh-server/pom.xml @@ -56,6 +56,17 @@ commons-logging ${commons-logging.version} + + com.typesafe.akka + akka-testkit_${scala.binary.version} + ${akka.actor.version} + test + + + org.mockito + mockito-core + test + diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala index 02099719d3383..bf77de9f704ff 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala @@ -17,53 +17,74 @@ package org.apache.spark.security.kubernetes import java.lang -import java.util.{Timer, TimerTask} +import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ +import scala.concurrent.ExecutionContext.Implicits.global -import akka.actor.ActorRef +import akka.actor.{ActorRef, Scheduler} import io.fabric8.kubernetes.api.model.{Secret, SecretList} import io.fabric8.kubernetes.client._ import io.fabric8.kubernetes.client.Watcher.Action import io.fabric8.kubernetes.client.dsl.FilterWatchListDeletable - import org.apache.spark.security.kubernetes.constants._ -private class SecretFinder(renewService: ActorRef, - timer: Timer, - kubernetesClient: KubernetesClient) { +import scala.concurrent.duration.Duration + +private trait SecretSelection { + + def selectSecrets(kubernetesClient: KubernetesClient, settings: Settings): + FilterWatchListDeletable[Secret, SecretList, lang.Boolean, Watch, Watcher[Secret]] = { + val selector = kubernetesClient.secrets() + val namespacedSelector = if (settings.shouldScanAllNamespaces) { + selector.inAnyNamespace() + } else { + selector.inNamespace(settings.namespaceToScan) + } + namespacedSelector + .withLabel(SECRET_LABEL_KEY_REFRESH_HADOOP_TOKENS, SECRET_LABEL_VALUE_REFRESH_HADOOP_TOKENS) + } +} - timer.schedule(new SecretScanner(renewService, kubernetesClient), - SECRET_SCANNER_INITIAL_DELAY_MILLIS, SECRET_SCANNER_PERIOD_MILLIS) - SecretFinder.selectSecrets(kubernetesClient) - .watch(new SecretWatcher(renewService)) +private class SecretFinder(refreshService: ActorRef, + scheduler: Scheduler, + kubernetesClient: KubernetesClient, + settings: Settings) extends SecretSelection { + + private val cancellable = scheduler.schedule( + Duration(SECRET_SCANNER_INITIAL_DELAY_MILLIS, TimeUnit.MILLISECONDS), + Duration(SECRET_SCANNER_PERIOD_MILLIS, TimeUnit.MILLISECONDS), + new SecretScanner(refreshService, kubernetesClient, settings)) + private val watched = selectSecrets(kubernetesClient, settings) + .watch(new SecretWatcher(refreshService)) def stop(): Unit = { - timer.cancel() - kubernetesClient.close() + cancellable.cancel() + watched.close() } } -private class SecretScanner(renewService: ActorRef, - kubernetesClient: KubernetesClient) extends TimerTask with Logging { +private class SecretScanner(refreshService: ActorRef, + kubernetesClient: KubernetesClient, + settings: Settings) extends Runnable with SecretSelection with Logging { override def run(): Unit = { - val secrets = SecretFinder.selectSecrets(kubernetesClient).list.getItems.asScala.toList - logInfo(s"Scanned ${secrets.map(_.getMetadata.getSelfLink).mkString}") - renewService ! UpdateSecretsToTrack(secrets) + val secrets = selectSecrets(kubernetesClient, settings).list.getItems.asScala.toList + logInfo(s"Scanned ${secrets.map(_.getMetadata.getSelfLink).mkString(", ")}") + refreshService ! UpdateSecretsToTrack(secrets) } } -private class SecretWatcher(renewService: ActorRef) extends Watcher[Secret] with Logging { +private class SecretWatcher(refreshService: ActorRef) extends Watcher[Secret] with Logging { override def eventReceived(action: Action, secret: Secret): Unit = { action match { case Action.ADDED => logInfo(s"Found ${secret.getMetadata.getSelfLink} added") - renewService ! StartRefresh(secret) + refreshService ! StartRefresh(secret) case Action.DELETED => logInfo(s"Found ${secret.getMetadata.getSelfLink} deleted") - renewService ! StopRefresh(secret) + refreshService ! StopRefresh(secret) case _ => // Do nothing } } @@ -75,22 +96,6 @@ private class SecretWatcher(renewService: ActorRef) extends Watcher[Secret] with private object SecretFinder { - def apply(renewService: ActorRef, kubernetesClient: KubernetesClient) : SecretFinder = { - new SecretFinder(renewService, - new Timer(SECRET_SCANNER_THREAD_NAME, IS_DAEMON_THREAD), - kubernetesClient) - } - - def selectSecrets(kubernetesClient: KubernetesClient): - FilterWatchListDeletable[Secret, SecretList, lang.Boolean, Watch, Watcher[Secret]] = { - val selector = kubernetesClient - .secrets() - val namespacedSelector = if (Settings.shouldScanAllNamespaces) { - selector.inAnyNamespace() - } else { - selector.inNamespace(Settings.namespaceToScan) - } - namespacedSelector - .withLabel(SECRET_LABEL_KEY_REFRESH_HADOOP_TOKENS, SECRET_LABEL_VALUE_REFRESH_HADOOP_TOKENS) - } + def apply(refreshService: ActorRef, scheduler: Scheduler, client: KubernetesClient, + settings: Settings) = new SecretFinder(refreshService, scheduler, client, settings) } diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala index 8ecbf9649e481..4922be1dcb719 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala @@ -17,25 +17,24 @@ package org.apache.spark.security.kubernetes import scala.annotation.tailrec -import scala.collection.JavaConversions._ import scala.concurrent.Await import scala.concurrent.duration.Duration - -import akka.actor.ActorSystem +import akka.actor.{ActorRef, ActorSystem, Scheduler} import com.typesafe.config.{Config, ConfigFactory} -import io.fabric8.kubernetes.client.DefaultKubernetesClient +import io.fabric8.kubernetes.client.{DefaultKubernetesClient, KubernetesClient} import org.apache.log4j.{Level, Logger} -private class Server { +private class Server(injector: Injector) { - private val actorSystem = ActorSystem("TokenRefreshServer") - private val kubernetesClient = new DefaultKubernetesClient + private val actorSystem = injector.newActorSystem() + private val kubernetesClient = injector.newKubernetesClient() + private val settings = injector.newSettings() private var secretFinder : Option[SecretFinder] = None def start(): Unit = { - val config = ConfigFactory.load - val renewService = TokenRefreshService(actorSystem, kubernetesClient) - secretFinder = Some(SecretFinder(renewService, kubernetesClient)) + val refreshService = injector.newTokenRefreshService(actorSystem, kubernetesClient, settings) + secretFinder = Some(injector.newSecretFinder(refreshService, kubernetesClient, + actorSystem.scheduler, settings)) } def join() : Unit = { @@ -47,12 +46,26 @@ private class Server { def stop(): Unit = { actorSystem.terminate() secretFinder.foreach(_.stop()) + kubernetesClient.close() } } -private object Settings { +private class Injector { + + def newActorSystem() = ActorSystem("TokenRefreshServer") + + def newKubernetesClient() : KubernetesClient = new DefaultKubernetesClient() - private val config: Config = ConfigFactory.load + def newSettings() = new Settings() + + def newTokenRefreshService(actorSystem: ActorSystem, client: KubernetesClient, settings: Settings) + = TokenRefreshService(actorSystem, client, settings) + + def newSecretFinder(refreshService: ActorRef, client: KubernetesClient, scheduler: Scheduler, + settings: Settings) = SecretFinder(refreshService, scheduler, client, settings) +} + +private class Settings(config: Config = ConfigFactory.load) { private val configKeyPrefix = "hadoop-token-refresh-server" @@ -83,14 +96,20 @@ object TokenRefreshServer { case ("--debug" | "-d") :: tail => logLevel = Level.DEBUG parse(tail) - case _ => + case unknown => + usage() + throw new IllegalArgumentException(s"Got an unknown argument: $unknown") + } + + private def usage(): Unit = { + println("Usage: TokenRefreshServer [--verbose | -v] [--debug | -d]") } } def main(args: Array[String]): Unit = { val parsedArgs = new Arguments(args.toList) Logger.getRootLogger.setLevel(parsedArgs.logLevel) - val server = new Server + val server = new Server(new Injector) try { server.start() server.join() diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index 949db10a97d3b..48d6405f98e9e 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -27,7 +27,6 @@ import scala.concurrent.duration.Duration import scala.concurrent.ExecutionContext.Implicits.global import akka.actor.{Actor, ActorRef, ActorSystem, Cancellable, Props} -import com.typesafe.config.Config import io.fabric8.kubernetes.api.model.{ObjectMeta, Secret} import io.fabric8.kubernetes.client.KubernetesClient import org.apache.commons.codec.binary.Base64 @@ -48,6 +47,12 @@ private class TokenRefreshService(kubernetesClient: KubernetesClient) extends Ac private val hadoopConf = new Configuration private val clock = new Clock + override def preStart(): Unit = { + super.preStart() + val duration = Duration(REFRESH_SERVER_KERBEROS_RELOGIN_PERIOD_MILLIS, TimeUnit.MILLISECONDS) + scheduler.schedule(duration, duration, self, Relogin) + } + override def receive: PartialFunction[Any, Unit] = { case Relogin => launchReloginTask() @@ -351,13 +356,11 @@ private object TokenRefreshService { val hadoopTokenPattern : Pattern = Pattern.compile(SECRET_DATA_ITEM_KEY_REGEX_HADOOP_TOKENS) - def apply(system: ActorSystem, kubernetesClient: KubernetesClient) : ActorRef = { + def apply(system: ActorSystem, kubernetesClient: KubernetesClient, + settings: Settings) : ActorRef = { UserGroupInformation.loginUserFromKeytab( - Settings.refreshServerKerberosPrincipal, + settings.refreshServerKerberosPrincipal, REFRESH_SERVER_KERBEROS_KEYTAB_PATH) - val actor = system.actorOf(Props(classOf[TokenRefreshService], kubernetesClient)) - val duration = Duration(REFRESH_SERVER_KERBEROS_RELOGIN_PERIOD_MILLIS, TimeUnit.MILLISECONDS) - system.scheduler.schedule(duration, duration, actor, Relogin) - actor + system.actorOf(Props(classOf[TokenRefreshService], kubernetesClient)) } } diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala index b166446a2cbf0..7f7728c4537b4 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala @@ -34,8 +34,6 @@ package object constants { val RENEW_TASK_DEADLINE_LOOK_AHEAD_MILLIS = 10000L val RENEW_TASK_REMAINING_TIME_BEFORE_NEW_TOKEN_MILLIS = 3 * 60 * 60 * 1000L - val SECRET_SCANNER_THREAD_NAME = "secret-scanner" val SECRET_SCANNER_INITIAL_DELAY_MILLIS = 10 * 1000L val SECRET_SCANNER_PERIOD_MILLIS = 60 * 60 * 1000L - val IS_DAEMON_THREAD = true } diff --git a/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/SecretFinderSuite.scala b/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/SecretFinderSuite.scala new file mode 100644 index 0000000000000..1f84874bbe168 --- /dev/null +++ b/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/SecretFinderSuite.scala @@ -0,0 +1,161 @@ +package org.apache.spark.security.kubernetes + +import java.util.concurrent.TimeUnit + +import scala.collection.JavaConverters._ +import scala.concurrent.ExecutionContext +import scala.concurrent.duration.{Duration, FiniteDuration} +import akka.actor.{ActorRef, ActorSystem, Cancellable, Props, Scheduler} +import akka.testkit.{ImplicitSender, TestActorRef, TestActors, TestKit, TestProbe} +import com.typesafe.config.ConfigFactory +import io.fabric8.kubernetes.api.model.{DoneableSecret, Secret, SecretList} +import io.fabric8.kubernetes.client.Watcher.Action +import io.fabric8.kubernetes.client.{KubernetesClient, Watch, Watcher} +import io.fabric8.kubernetes.client.dsl.{FilterWatchListDeletable, FilterWatchListMultiDeletable, MixedOperation, Resource} +import org.mockito._ +import org.mockito.Matchers.{any, anyString} +import org.mockito.Mockito.{doReturn, never, verify, when} +import org.scalatest.{BeforeAndAfter, FunSuite, FunSuiteLike} +import org.apache.spark.security.kubernetes.constants._ + + +class SecretFinderSuite extends TestKit(ActorSystem("test")) with ImplicitSender with FunSuiteLike + with BeforeAndAfter { + + private val configKeyPrefix = "hadoop-token-refresh-server" + + private val configMap1 = Map(s"$configKeyPrefix.kerberosPrincipal" -> "my-principla", + s"$configKeyPrefix.scanAllNamespaces" -> true, + s"$configKeyPrefix.namespaceToScan" -> "my-namespace") + private val configMap2 = configMap1.updated(s"$configKeyPrefix.scanAllNamespaces", false) + private val probe = TestProbe() + private val tokenRefreshService = TestActorRef(TestActors.forwardActorProps(probe.ref)) + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var kubernetesClient: KubernetesClient = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var secrets: MixedOperation[Secret, SecretList, DoneableSecret, + Resource[Secret, DoneableSecret]] = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var secretsInAnynamespace: FilterWatchListMultiDeletable[Secret, SecretList, Boolean, + Watch, Watcher[Secret]] = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var secretsInSpecifiedNamespace: FilterWatchListMultiDeletable[Secret, SecretList, + Boolean, Watch, Watcher[Secret]] = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var secretsWithLabel: FilterWatchListDeletable[Secret, SecretList, Boolean, + Watch, Watcher[Secret]] = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var secretList: SecretList = _ + @Mock(answer = Answers.RETURNS_DEEP_STUBS) + private var secret1: Secret = _ + @Mock(answer = Answers.RETURNS_DEEP_STUBS) + private var secret2: Secret = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var watch: Watch = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var scheduler: Scheduler = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var cancellable: Cancellable = _ + + before { + MockitoAnnotations.initMocks(this) + when(scheduler.schedule(any(classOf[FiniteDuration]), any(classOf[FiniteDuration]), + any(classOf[Runnable]))(any(classOf[ExecutionContext]))) + .thenReturn(cancellable) + when(kubernetesClient.secrets).thenReturn(secrets) + doReturn(secretsInAnynamespace).when(secrets).inAnyNamespace() + doReturn(secretsInSpecifiedNamespace).when(secrets).inNamespace("my-namespace") + doReturn(secretsWithLabel).when(secretsInAnynamespace).withLabel( + SECRET_LABEL_KEY_REFRESH_HADOOP_TOKENS, SECRET_LABEL_VALUE_REFRESH_HADOOP_TOKENS) + doReturn(secretsWithLabel).when(secretsInSpecifiedNamespace).withLabel( + SECRET_LABEL_KEY_REFRESH_HADOOP_TOKENS, SECRET_LABEL_VALUE_REFRESH_HADOOP_TOKENS) + when(secretsWithLabel.watch(any(classOf[Watcher[Secret]]))).thenReturn(watch) + } + + test("Secret finder sets up the scanner and watcher, by default for all namespaces") { + val config = ConfigFactory.parseMap(configMap1.asJava) + val settings = new Settings(config) + val secretFinder = SecretFinder(tokenRefreshService, scheduler, kubernetesClient, settings) + + verifyScannerScheduled() + verifyWatcherLaunched() + verify(secrets).inAnyNamespace() + verify(secrets, never).inNamespace(anyString) + } + + test("Secret finder sets up the scanner and watcher for a specified namespace") { + val config = ConfigFactory.parseMap(configMap2.asJava) + val settings = new Settings(config) + val secretFinder = SecretFinder(tokenRefreshService, scheduler, kubernetesClient, settings) + + verifyScannerScheduled() + verifyWatcherLaunched() + verify(secrets).inNamespace("my-namespace") + verify(secrets, never).inAnyNamespace() + } + + test("Stopping the secret finder cancels the scanner and watcher") { + val config = ConfigFactory.parseMap(configMap1.asJava) + val settings = new Settings(config) + val secretFinder = SecretFinder(tokenRefreshService, scheduler, kubernetesClient, settings) + val scanner = captureScanner() + secretFinder.stop() + + verify(cancellable).cancel() + verify(watch).close() + } + + test("Scanner sends the refresh service secrets to track ") { + when(secretsWithLabel.list()).thenReturn(secretList) + val secrets = List(secret1, secret2) + when(secretList.getItems).thenReturn(secrets.asJava) + val config = ConfigFactory.parseMap(configMap1.asJava) + val settings = new Settings(config) + val scanner = new SecretScanner(tokenRefreshService, kubernetesClient, settings) + scanner.run() + + probe.expectMsg(UpdateSecretsToTrack(secrets)) + } + + test("Watcher sends the refresh service new or deleted secret") { + val config = ConfigFactory.parseMap(configMap1.asJava) + val settings = new Settings(config) + val watcher = new SecretWatcher(tokenRefreshService) + + watcher.eventReceived(Action.ADDED, secret1) + probe.expectMsg(StartRefresh(secret1)) + + watcher.eventReceived(Action.DELETED, secret2) + probe.expectMsg(StopRefresh(secret2)) + + watcher.eventReceived(Action.MODIFIED, secret1) // Ignored. + watcher.eventReceived(Action.ERROR, secret1) // Ignored. + } + + private def verifyScannerScheduled() = { + val scanner = captureScanner() + assert(scanner.getClass == classOf[SecretScanner]) + } + + private def captureScanner() = { + val scannerCaptor: ArgumentCaptor[Runnable] = ArgumentCaptor.forClass(classOf[Runnable]) + verify(scheduler).schedule( + Matchers.eq(Duration(SECRET_SCANNER_INITIAL_DELAY_MILLIS, TimeUnit.MILLISECONDS)), + Matchers.eq(Duration(SECRET_SCANNER_PERIOD_MILLIS, TimeUnit.MILLISECONDS)), + scannerCaptor.capture())(any(classOf[ExecutionContext]) + ) + scannerCaptor.getValue + } + + private def verifyWatcherLaunched() = { + val watcher = captureWatcher() + assert(watcher.getClass == classOf[SecretWatcher]) + } + + private def captureWatcher() = { + val watcherCaptor: ArgumentCaptor[Watcher[Secret]] = ArgumentCaptor.forClass( + classOf[Watcher[Secret]]) + verify(secretsWithLabel).watch(watcherCaptor.capture()) + watcherCaptor.getValue + } +} diff --git a/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/TokenRefreshServerSuite.scala b/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/TokenRefreshServerSuite.scala new file mode 100644 index 0000000000000..8e92c45a44c7c --- /dev/null +++ b/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/TokenRefreshServerSuite.scala @@ -0,0 +1,90 @@ +/* + * 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.security.kubernetes + +import scala.concurrent.Future +import scala.collection.JavaConverters._ + +import akka.actor.{ActorRef, ActorSystem, Scheduler} +import com.typesafe.config.ConfigFactory +import io.fabric8.kubernetes.client.KubernetesClient +import org.mockito.{Answers, Mock, MockitoAnnotations} +import org.mockito.Mockito.{doReturn, never, verify, when} +import org.scalatest.{BeforeAndAfter, FunSuite} + + +class TokenRefreshServerSuite extends FunSuite with BeforeAndAfter { + + private val configKeyPrefix = "hadoop-token-refresh-server" + private val config = ConfigFactory.parseMap( + Map(s"$configKeyPrefix.kerberosPrincipal" -> "my-principla", + s"$configKeyPrefix.scanAllNamespaces" -> true, + s"$configKeyPrefix.namespaceToScan" -> "my-namespace").asJava) + private val settings = new Settings(config) + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var injector: Injector = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var actorSystem: ActorSystem = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var scheduler: Scheduler = _ + @Mock(answer = Answers.RETURNS_DEEP_STUBS) + private var actorSystemAwaitFuture: Future[Unit] = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var kubernetesClient: KubernetesClient = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var tokenRefreshServiceActorRef: ActorRef = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var secretFinder: SecretFinder = _ + private var server : Server = _ + + before { + MockitoAnnotations.initMocks(this) + when(injector.newActorSystem()).thenReturn(actorSystem) + when(actorSystem.scheduler).thenReturn(scheduler) + when(injector.newKubernetesClient()).thenReturn(kubernetesClient) + when(injector.newSettings()).thenReturn(settings) + when(injector.newTokenRefreshService(actorSystem, kubernetesClient, settings)) + .thenReturn(tokenRefreshServiceActorRef) + when(injector.newSecretFinder(tokenRefreshServiceActorRef, kubernetesClient, scheduler, + settings)) + .thenReturn(secretFinder) + doReturn(actorSystemAwaitFuture).when(actorSystem).whenTerminated + server = new Server(injector) + } + + test("The token refresh server starts the refresh service actor and secret finder") { + server.start() + verify(injector).newTokenRefreshService(actorSystem, kubernetesClient, settings) + verify(injector).newSecretFinder(tokenRefreshServiceActorRef, kubernetesClient, scheduler, + settings) + verify(actorSystem, never()).whenTerminated + } + + test("The token refresh server waits until the refresh service finishes") { + server.start() + server.join() + verify(actorSystem).whenTerminated + } + + test("The token refresh server stops the refresh service and secret finder") { + server.start() + server.stop() + verify(actorSystem).terminate() + verify(kubernetesClient).close() + verify(secretFinder).stop() + } +} From eb10c4ad6cf421ecd1b147c7f0ad7941a27de1be Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Thu, 12 Oct 2017 10:22:19 -0700 Subject: [PATCH 35/43] Add more tests --- .../kubernetes/TokenRefreshServer.scala | 58 +++++++++++-------- .../src/test/resources/log4j.properties | 31 ++++++++++ .../kubernetes/SecretFinderSuite.scala | 6 +- .../kubernetes/TokenRefreshServerSuite.scala | 52 ++++++++++++++++- 4 files changed, 120 insertions(+), 27 deletions(-) create mode 100644 resource-managers/kubernetes/token-refresh-server/src/test/resources/log4j.properties diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala index 4922be1dcb719..8dd55cfce953e 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala @@ -19,6 +19,7 @@ package org.apache.spark.security.kubernetes import scala.annotation.tailrec import scala.concurrent.Await import scala.concurrent.duration.Duration + import akka.actor.{ActorRef, ActorSystem, Scheduler} import com.typesafe.config.{Config, ConfigFactory} import io.fabric8.kubernetes.client.{DefaultKubernetesClient, KubernetesClient} @@ -77,39 +78,35 @@ private class Settings(config: Config = ConfigFactory.load) { val namespaceToScan : String = config.getString(s"$configKeyPrefix.namespaceToScan") } -/* - * TODO: Support REST endpoint for checking status of tokens. - */ -object TokenRefreshServer { - - private class Arguments(args: List[String]) { +private class CommandLine(args: List[String]) { - var logLevel: Level = Level.WARN + var logLevel: Level = Level.WARN - parse(args) + parse(args) - @tailrec - private def parse(args: List[String]): Unit = args match { - case ("--verbose" | "-v") :: tail => - logLevel = Level.INFO - parse(tail) - case ("--debug" | "-d") :: tail => - logLevel = Level.DEBUG - parse(tail) - case unknown => + @tailrec + private def parse(args: List[String]): Unit = args match { + case ("--verbose" | "-v") :: tail => + logLevel = Level.INFO + parse(tail) + case ("--debug" | "-d") :: tail => + logLevel = Level.DEBUG + parse(tail) + case unknown if unknown.nonEmpty => usage() throw new IllegalArgumentException(s"Got an unknown argument: $unknown") - } + case _ => + } - private def usage(): Unit = { - println("Usage: TokenRefreshServer [--verbose | -v] [--debug | -d]") - } + private def usage(): Unit = { + println("Usage: TokenRefreshServer [--verbose | -v] [--debug | -d]") } +} - def main(args: Array[String]): Unit = { - val parsedArgs = new Arguments(args.toList) +private class Launcher(parsedArgs: CommandLine, server: Server) { + + def launch(): Unit = { Logger.getRootLogger.setLevel(parsedArgs.logLevel) - val server = new Server(new Injector) try { server.start() server.join() @@ -118,3 +115,16 @@ object TokenRefreshServer { } } } + +/* + * TODO: Support REST endpoint for checking status of tokens. + */ +object TokenRefreshServer { + + def main(args: Array[String]): Unit = { + val parsedArgs = new CommandLine(args.toList) + val server = new Server(new Injector) + val launcher = new Launcher(parsedArgs, server) + launcher.launch() + } +} diff --git a/resource-managers/kubernetes/token-refresh-server/src/test/resources/log4j.properties b/resource-managers/kubernetes/token-refresh-server/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..ad95fadb7c0c0 --- /dev/null +++ b/resource-managers/kubernetes/token-refresh-server/src/test/resources/log4j.properties @@ -0,0 +1,31 @@ +# +# 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 file target/unit-tests.log +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Ignore messages below warning level from a few verbose libraries. +log4j.logger.com.sun.jersey=WARN +log4j.logger.org.apache.hadoop=WARN +log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.mortbay=WARN +log4j.logger.org.spark_project.jetty=WARN diff --git a/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/SecretFinderSuite.scala b/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/SecretFinderSuite.scala index 1f84874bbe168..f22afe8151c5a 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/SecretFinderSuite.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/SecretFinderSuite.scala @@ -5,7 +5,8 @@ import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ import scala.concurrent.ExecutionContext import scala.concurrent.duration.{Duration, FiniteDuration} -import akka.actor.{ActorRef, ActorSystem, Cancellable, Props, Scheduler} + +import akka.actor.{ActorSystem, Cancellable, Scheduler} import akka.testkit.{ImplicitSender, TestActorRef, TestActors, TestKit, TestProbe} import com.typesafe.config.ConfigFactory import io.fabric8.kubernetes.api.model.{DoneableSecret, Secret, SecretList} @@ -15,7 +16,8 @@ import io.fabric8.kubernetes.client.dsl.{FilterWatchListDeletable, FilterWatchLi import org.mockito._ import org.mockito.Matchers.{any, anyString} import org.mockito.Mockito.{doReturn, never, verify, when} -import org.scalatest.{BeforeAndAfter, FunSuite, FunSuiteLike} +import org.scalatest.{BeforeAndAfter, FunSuiteLike} + import org.apache.spark.security.kubernetes.constants._ diff --git a/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/TokenRefreshServerSuite.scala b/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/TokenRefreshServerSuite.scala index 8e92c45a44c7c..62eb84d3e3323 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/TokenRefreshServerSuite.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/TokenRefreshServerSuite.scala @@ -18,10 +18,10 @@ package org.apache.spark.security.kubernetes import scala.concurrent.Future import scala.collection.JavaConverters._ - import akka.actor.{ActorRef, ActorSystem, Scheduler} import com.typesafe.config.ConfigFactory import io.fabric8.kubernetes.client.KubernetesClient +import org.apache.log4j.Level import org.mockito.{Answers, Mock, MockitoAnnotations} import org.mockito.Mockito.{doReturn, never, verify, when} import org.scalatest.{BeforeAndAfter, FunSuite} @@ -49,6 +49,8 @@ class TokenRefreshServerSuite extends FunSuite with BeforeAndAfter { private var tokenRefreshServiceActorRef: ActorRef = _ @Mock(answer = Answers.RETURNS_SMART_NULLS) private var secretFinder: SecretFinder = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var mockServer: Server = _ private var server : Server = _ before { @@ -87,4 +89,52 @@ class TokenRefreshServerSuite extends FunSuite with BeforeAndAfter { verify(kubernetesClient).close() verify(secretFinder).stop() } + + test("The command line parses properly") { + var parsedArgs = new CommandLine(List()) + assert(parsedArgs.logLevel == Level.WARN) + + parsedArgs = new CommandLine(List("-v")) + assert(parsedArgs.logLevel == Level.INFO) + parsedArgs = new CommandLine(List("--verbose")) + assert(parsedArgs.logLevel == Level.INFO) + + parsedArgs = new CommandLine(List("-d")) + assert(parsedArgs.logLevel == Level.DEBUG) + parsedArgs = new CommandLine(List("--debug")) + assert(parsedArgs.logLevel == Level.DEBUG) + } + + test("Unknown command line arguments throws") { + intercept[IllegalArgumentException] { + new CommandLine(List("")) + } + intercept[IllegalArgumentException] { + new CommandLine(List("-f")) + } + intercept[IllegalArgumentException] { + new CommandLine(List("--unknown")) + } + } + + test("The server launches properly") { + val launcher = new Launcher(new CommandLine(List()), mockServer) + launcher.launch() + + verify(mockServer).start() + verify(mockServer).join() + verify(mockServer).stop() + } + + test("The server stops properly upon error") { + when(mockServer.stop()).thenThrow(new RuntimeException) + val launcher = new Launcher(new CommandLine(List()), mockServer) + intercept[RuntimeException] { + launcher.launch() + } + + verify(mockServer).start() + verify(mockServer).join() + verify(mockServer).stop() + } } From 078ac2af9577233901393bf21f36f1280ad8989c Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Thu, 12 Oct 2017 10:27:24 -0700 Subject: [PATCH 36/43] Clean up --- .../security/kubernetes/SecretFinderSuite.scala | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/SecretFinderSuite.scala b/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/SecretFinderSuite.scala index f22afe8151c5a..2607ffe0ee039 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/SecretFinderSuite.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/SecretFinderSuite.scala @@ -7,7 +7,7 @@ import scala.concurrent.ExecutionContext import scala.concurrent.duration.{Duration, FiniteDuration} import akka.actor.{ActorSystem, Cancellable, Scheduler} -import akka.testkit.{ImplicitSender, TestActorRef, TestActors, TestKit, TestProbe} +import akka.testkit.{TestKit, TestProbe} import com.typesafe.config.ConfigFactory import io.fabric8.kubernetes.api.model.{DoneableSecret, Secret, SecretList} import io.fabric8.kubernetes.client.Watcher.Action @@ -21,8 +21,7 @@ import org.scalatest.{BeforeAndAfter, FunSuiteLike} import org.apache.spark.security.kubernetes.constants._ -class SecretFinderSuite extends TestKit(ActorSystem("test")) with ImplicitSender with FunSuiteLike - with BeforeAndAfter { +class SecretFinderSuite extends TestKit(ActorSystem("test")) with FunSuiteLike with BeforeAndAfter { private val configKeyPrefix = "hadoop-token-refresh-server" @@ -30,8 +29,8 @@ class SecretFinderSuite extends TestKit(ActorSystem("test")) with ImplicitSender s"$configKeyPrefix.scanAllNamespaces" -> true, s"$configKeyPrefix.namespaceToScan" -> "my-namespace") private val configMap2 = configMap1.updated(s"$configKeyPrefix.scanAllNamespaces", false) - private val probe = TestProbe() - private val tokenRefreshService = TestActorRef(TestActors.forwardActorProps(probe.ref)) + private val tokenRefreshServiceProbe = TestProbe() + private val tokenRefreshService = tokenRefreshServiceProbe.ref @Mock(answer = Answers.RETURNS_SMART_NULLS) private var kubernetesClient: KubernetesClient = _ @Mock(answer = Answers.RETURNS_SMART_NULLS) @@ -116,7 +115,7 @@ class SecretFinderSuite extends TestKit(ActorSystem("test")) with ImplicitSender val scanner = new SecretScanner(tokenRefreshService, kubernetesClient, settings) scanner.run() - probe.expectMsg(UpdateSecretsToTrack(secrets)) + tokenRefreshServiceProbe.expectMsg(UpdateSecretsToTrack(secrets)) } test("Watcher sends the refresh service new or deleted secret") { @@ -125,10 +124,10 @@ class SecretFinderSuite extends TestKit(ActorSystem("test")) with ImplicitSender val watcher = new SecretWatcher(tokenRefreshService) watcher.eventReceived(Action.ADDED, secret1) - probe.expectMsg(StartRefresh(secret1)) + tokenRefreshServiceProbe.expectMsg(StartRefresh(secret1)) watcher.eventReceived(Action.DELETED, secret2) - probe.expectMsg(StopRefresh(secret2)) + tokenRefreshServiceProbe.expectMsg(StopRefresh(secret2)) watcher.eventReceived(Action.MODIFIED, secret1) // Ignored. watcher.eventReceived(Action.ERROR, secret1) // Ignored. From aeb269aa5ab88d78aff4b511d723187980a672c8 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Thu, 12 Oct 2017 15:52:33 -0700 Subject: [PATCH 37/43] Add more tests --- .../security/kubernetes/SecretFinder.scala | 2 +- .../kubernetes/TokenRefreshService.scala | 118 ++++++--- .../spark/security/kubernetes/constants.scala | 4 +- .../kubernetes/SecretFinderSuite.scala | 24 +- .../kubernetes/TokenRefreshServerSuite.scala | 2 +- .../kubernetes/TokenRefreshServiceSuite.scala | 234 ++++++++++++++++++ 6 files changed, 339 insertions(+), 45 deletions(-) create mode 100644 resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/TokenRefreshServiceSuite.scala diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala index bf77de9f704ff..e96e1829c9f6c 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala @@ -53,7 +53,7 @@ private class SecretFinder(refreshService: ActorRef, private val cancellable = scheduler.schedule( Duration(SECRET_SCANNER_INITIAL_DELAY_MILLIS, TimeUnit.MILLISECONDS), - Duration(SECRET_SCANNER_PERIOD_MILLIS, TimeUnit.MILLISECONDS), + Duration(SECRET_SCANNER_INTERVAL_MILLIS, TimeUnit.MILLISECONDS), new SecretScanner(refreshService, kubernetesClient, settings)) private val watched = selectSecrets(kubernetesClient, settings) .watch(new SecretWatcher(refreshService)) diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index 48d6405f98e9e..9349ecaf6858a 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -25,8 +25,8 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import scala.concurrent.duration.Duration import scala.concurrent.ExecutionContext.Implicits.global - -import akka.actor.{Actor, ActorRef, ActorSystem, Cancellable, Props} +import akka.actor.{Actor, ActorRef, ActorSystem, Cancellable, Props, Scheduler} +import com.google.common.annotations.VisibleForTesting import io.fabric8.kubernetes.api.model.{ObjectMeta, Secret} import io.fabric8.kubernetes.client.KubernetesClient import org.apache.commons.codec.binary.Base64 @@ -39,18 +39,24 @@ import org.apache.hadoop.security.token.{Token, TokenIdentifier} import org.apache.spark.security.kubernetes.constants._ -private class TokenRefreshService(kubernetesClient: KubernetesClient) extends Actor with Logging { +private class TokenRefreshService(kubernetesClient: KubernetesClient, scheduler: Scheduler, + ugi: UgiUtil, + settings: Settings, + clock: Clock) extends Actor with Logging { - private val scheduler = context.system.scheduler private val secretUidToTaskHandle = mutable.HashMap[String, Cancellable]() private val recentlyAddedSecretUids = mutable.HashSet[String]() + private val extraCancellableByClass = mutable.HashMap[Class[_], Cancellable]() private val hadoopConf = new Configuration - private val clock = new Clock + + ugi.loginUserFromKeytab(settings.refreshServerKerberosPrincipal, + REFRESH_SERVER_KERBEROS_KEYTAB_PATH) override def preStart(): Unit = { super.preStart() - val duration = Duration(REFRESH_SERVER_KERBEROS_RELOGIN_PERIOD_MILLIS, TimeUnit.MILLISECONDS) - scheduler.schedule(duration, duration, self, Relogin) + val duration = Duration(REFRESH_SERVER_KERBEROS_RELOGIN_INTERVAL_MILLIS, TimeUnit.MILLISECONDS) + extraCancellableByClass.put(Relogin.getClass, + scheduler.schedule(duration, duration, self, Relogin)) } override def receive: PartialFunction[Any, Unit] = { @@ -69,11 +75,14 @@ private class TokenRefreshService(kubernetesClient: KubernetesClient) extends Ac override def postStop(): Unit = { super.postStop() secretUidToTaskHandle.values.map(_.cancel) + extraCancellableByClass.values.map(_.cancel) } private def launchReloginTask() = { val task = new ReloginTask - scheduler.scheduleOnce(Duration(0L, TimeUnit.MILLISECONDS), task) + extraCancellableByClass.remove(task.getClass).foreach(_.cancel) // Cancel in case of hanging + extraCancellableByClass.put(task.getClass, + scheduler.scheduleOnce(Duration(0L, TimeUnit.MILLISECONDS), task)) } private def startRefreshTask(secret: Secret) = { @@ -82,67 +91,101 @@ private class TokenRefreshService(kubernetesClient: KubernetesClient) extends Ac } private def addRefreshTask(secret: Secret) = { - secretUidToTaskHandle.getOrElseUpdate(getSecretUid(secret.getMetadata), { + val secretUid = getSecretUid(secret.getMetadata) + secretUidToTaskHandle.remove(secretUid).foreach(_.cancel) // Cancel in case of hanging + secretUidToTaskHandle.put(secretUid, { val task = new StarterTask(secret, hadoopConf, self, clock) val cancellable = scheduler.scheduleOnce( Duration(STARTER_TASK_INITIAL_DELAY_MILLIS, TimeUnit.MILLISECONDS), task) - logInfo(s"Started refresh of tokens in ${secret.getMetadata.getSelfLink} with $cancellable") + logInfo(s"Started refresh of tokens in $secretUid of ${secret.getMetadata.getSelfLink}" + + s" with $cancellable") cancellable }) } - private def removeRefreshTask(secret: Secret) : Unit = { - val uid = getSecretUid(secret.getMetadata) - secretUidToTaskHandle.remove(uid).foreach(cancellable => { - logInfo(s"Canceling refresh of tokens in ${secret.getMetadata.getSelfLink}") + private def removeRefreshTask(secret: Secret): Unit = + removeRefreshTask(getSecretUid(secret.getMetadata)) + + private def removeRefreshTask(secretUid: String): Unit = { + secretUidToTaskHandle.remove(secretUid).foreach(cancellable => { + logInfo(s"Canceling refresh of tokens in $secretUid") cancellable.cancel() }) } - private def updateSecretsToTrack(currentSecrets: List[Secret]) : Unit = { + private def updateSecretsToTrack(currentSecrets: List[Secret]): Unit = { val secretsByUids = currentSecrets.map(secret => (getSecretUid(secret.getMetadata), secret)) .toMap val currentUids = secretsByUids.keySet - val priorUids = secretUidToTaskHandle.keySet + val priorUids = secretUidToTaskHandle.keys.toSet val uidsToAdd = currentUids -- priorUids uidsToAdd.foreach(uid => addRefreshTask(secretsByUids(uid))) val uidsToRemove = priorUids -- currentUids -- recentlyAddedSecretUids - uidsToRemove.foreach(uid => removeRefreshTask(secretsByUids(uid))) + uidsToRemove.foreach(uid => removeRefreshTask(uid)) recentlyAddedSecretUids.clear() } private def scheduleRenewTask(renew: Renew) = { - val uid = getSecretUid(renew.secretMeta) - if (secretUidToTaskHandle.get(uid).nonEmpty) { + val secretUid = getSecretUid(renew.secretMeta) + val priorTask = secretUidToTaskHandle.remove(secretUid) + if (priorTask.nonEmpty) { + priorTask.get.cancel() // Cancel in case of hanging. val numConsecutiveErrors = renew.numConsecutiveErrors if (numConsecutiveErrors < RENEW_TASK_MAX_CONSECUTIVE_ERRORS) { val durationTillExpire = math.max(0L, renew.expireTime - clock.nowInMillis()) - val renewTime = math.max(0L, renew.expireTime - durationTillExpire / 10) // 90% mark. + val renewTime = math.max(0L, renew.expireTime - durationTillExpire / 10) // 90% mark. val durationTillRenew = math.max(0L, renewTime - clock.nowInMillis()) val task = new RenewTask(renew, hadoopConf, self, kubernetesClient, clock) - logInfo(s"Scheduling refresh of tokens with " + + logInfo(s"Scheduling refresh of tokens in $secretUid of " + s"${renew.secretMeta.getSelfLink} at now + $durationTillRenew millis.") val cancellable = scheduler.scheduleOnce( Duration(durationTillRenew, TimeUnit.MILLISECONDS), task) - secretUidToTaskHandle.put(uid, cancellable) + secretUidToTaskHandle.put(secretUid, cancellable) } else { - logWarning(s"Got too many errors for ${renew.secretMeta.getSelfLink}. Abandoning.") - val maybeCancellable = secretUidToTaskHandle.remove(uid) + logWarning(s"Got too many errors for secret $secretUid of" + + s" ${renew.secretMeta.getSelfLink}. Abandoning.") + val maybeCancellable = secretUidToTaskHandle.remove(secretUid) maybeCancellable.foreach(_.cancel()) } } else { - logWarning(s"Could not find an entry for renew task" + + logWarning(s"Could not find a StarterTask entry for a renew work for secret $secretUid of " + s" ${renew.secretMeta.getSelfLink}. Maybe the secret got deleted") } } private def getSecretUid(secret: ObjectMeta) = secret.getUid + + @VisibleForTesting + private[kubernetes] def numExtraCancellables() = extraCancellableByClass.size + + @VisibleForTesting + private[kubernetes] def hasExtraCancellable(key: Class[_], expected: Cancellable): Boolean = { + val value = extraCancellableByClass.get(key) + value.nonEmpty && expected == value.get + } + + @VisibleForTesting + private[kubernetes] def numPendingSecretTasks() = secretUidToTaskHandle.size + + @VisibleForTesting + private[kubernetes] def hasSecretTaskCancellable(secretUid: String, expected: Cancellable) + : Boolean = { + val value = secretUidToTaskHandle.get(secretUid) + value.nonEmpty && expected == value.get + } +} + +private class UgiUtil { + + def loginUserFromKeytab(kerberosPrincipal: String, kerberosKeytabPath: String): Unit = + UserGroupInformation.loginUserFromKeytab(kerberosPrincipal, kerberosKeytabPath) + } private class ReloginTask extends Runnable { - override def run() : Unit = { + override def run(): Unit = { UserGroupInformation.getLoginUser.checkTGTAndReloginFromKeytab() } } @@ -154,13 +197,14 @@ private class StarterTask(secret: Secret, private var hasError = false - override def run() : Unit = { + override def run(): Unit = { val tokenToExpireTime = readTokensFromSecret() logInfo(s"Read Hadoop tokens: $tokenToExpireTime") val nextExpireTime = if (tokenToExpireTime.nonEmpty) { tokenToExpireTime.values.min } else { - logWarning(s"Got an empty token list with ${secret.getMetadata.getSelfLink}") + logWarning(s"Got an empty token list with secret ${secret.getMetadata.getUid} of" + + s" ${secret.getMetadata.getSelfLink}") hasError = true getRetryTime } @@ -169,7 +213,7 @@ private class StarterTask(secret: Secret, numConsecutiveErrors) } - private def readTokensFromSecret() : Map[Token[_ <: TokenIdentifier], Long] = { + private def readTokensFromSecret(): Map[Token[_ <: TokenIdentifier], Long] = { val dataItems = secret.getData.asScala.filterKeys( _.startsWith(SECRET_DATA_ITEM_KEY_PREFIX_HADOOP_TOKENS)).toSeq.sorted val latestDataItem = if (dataItems.nonEmpty) Some(dataItems.max) else None @@ -201,7 +245,7 @@ private class RenewTask(renew: Renew, private var hasError = false - override def run() : Unit = { + override def run(): Unit = { val deadline = renew.expireTime + RENEW_TASK_DEADLINE_LOOK_AHEAD_MILLIS val nowMillis = clock.nowInMillis() val newExpireTimeByToken : Map[Token[_ <: TokenIdentifier], Long] = @@ -222,7 +266,8 @@ private class RenewTask(renew: Renew, refreshService ! Renew(nextExpireTime, newExpireTimeByToken, renew.secretMeta, numConsecutiveErrors) } else { - logWarning(s"Got an empty token list with ${renew.secretMeta.getSelfLink}") + logWarning(s"Got an empty token list with secret ${renew.secretMeta.getUid} of" + + s" ${renew.secretMeta.getSelfLink}") } } @@ -303,7 +348,7 @@ private class RenewTask(renew: Renew, } private def writeTokensToSecret(tokenToExpire: Map[Token[_ <: TokenIdentifier], Long], - nowMillis: Long) : Unit = { + nowMillis: Long): Unit = { val durationUntilExpire = tokenToExpire.values.min - nowMillis val key = s"$SECRET_DATA_ITEM_KEY_PREFIX_HADOOP_TOKENS$nowMillis-$durationUntilExpire" val credentials = new Credentials() @@ -323,7 +368,8 @@ private class RenewTask(renew: Renew, // where some newly launching executors may access the previous token. dataItemKeys.dropRight(2).foreach(editor.removeFromData) editor.done - logInfo(s"Wrote new tokens $tokenToExpire to a data item $key in ${secretMeta.getSelfLink}") + logInfo(s"Wrote new tokens $tokenToExpire to a data item $key in secret ${secretMeta.getUid}" + + s" of ${secretMeta.getSelfLink}") } private def serializeCredentials(credentials: Credentials) = { @@ -358,9 +404,7 @@ private object TokenRefreshService { def apply(system: ActorSystem, kubernetesClient: KubernetesClient, settings: Settings) : ActorRef = { - UserGroupInformation.loginUserFromKeytab( - settings.refreshServerKerberosPrincipal, - REFRESH_SERVER_KERBEROS_KEYTAB_PATH) - system.actorOf(Props(classOf[TokenRefreshService], kubernetesClient)) + system.actorOf(Props(classOf[TokenRefreshService], kubernetesClient, system.scheduler, + new UgiUtil, settings, new Clock)) } } diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala index 7f7728c4537b4..5a0ba70af192a 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/constants.scala @@ -19,7 +19,7 @@ package org.apache.spark.security.kubernetes package object constants { val REFRESH_SERVER_KERBEROS_KEYTAB_PATH = "/mnt/secrets/krb5.keytab" - val REFRESH_SERVER_KERBEROS_RELOGIN_PERIOD_MILLIS = 60 * 60 * 1000L + val REFRESH_SERVER_KERBEROS_RELOGIN_INTERVAL_MILLIS = 60 * 60 * 1000L val SECRET_LABEL_KEY_REFRESH_HADOOP_TOKENS = "refresh-hadoop-tokens" val SECRET_LABEL_VALUE_REFRESH_HADOOP_TOKENS = "yes" @@ -35,5 +35,5 @@ package object constants { val RENEW_TASK_REMAINING_TIME_BEFORE_NEW_TOKEN_MILLIS = 3 * 60 * 60 * 1000L val SECRET_SCANNER_INITIAL_DELAY_MILLIS = 10 * 1000L - val SECRET_SCANNER_PERIOD_MILLIS = 60 * 60 * 1000L + val SECRET_SCANNER_INTERVAL_MILLIS = 60 * 60 * 1000L } diff --git a/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/SecretFinderSuite.scala b/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/SecretFinderSuite.scala index 2607ffe0ee039..dcbb4ade4e6d8 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/SecretFinderSuite.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/SecretFinderSuite.scala @@ -1,3 +1,19 @@ +/* + * 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.security.kubernetes import java.util.concurrent.TimeUnit @@ -24,8 +40,7 @@ import org.apache.spark.security.kubernetes.constants._ class SecretFinderSuite extends TestKit(ActorSystem("test")) with FunSuiteLike with BeforeAndAfter { private val configKeyPrefix = "hadoop-token-refresh-server" - - private val configMap1 = Map(s"$configKeyPrefix.kerberosPrincipal" -> "my-principla", + private val configMap1 = Map(s"$configKeyPrefix.kerberosPrincipal" -> "my-principal", s"$configKeyPrefix.scanAllNamespaces" -> true, s"$configKeyPrefix.namespaceToScan" -> "my-namespace") private val configMap2 = configMap1.updated(s"$configKeyPrefix.scanAllNamespaces", false) @@ -142,8 +157,9 @@ class SecretFinderSuite extends TestKit(ActorSystem("test")) with FunSuiteLike w val scannerCaptor: ArgumentCaptor[Runnable] = ArgumentCaptor.forClass(classOf[Runnable]) verify(scheduler).schedule( Matchers.eq(Duration(SECRET_SCANNER_INITIAL_DELAY_MILLIS, TimeUnit.MILLISECONDS)), - Matchers.eq(Duration(SECRET_SCANNER_PERIOD_MILLIS, TimeUnit.MILLISECONDS)), - scannerCaptor.capture())(any(classOf[ExecutionContext]) + Matchers.eq(Duration(SECRET_SCANNER_INTERVAL_MILLIS, TimeUnit.MILLISECONDS)), + scannerCaptor.capture())( + any(classOf[ExecutionContext]) ) scannerCaptor.getValue } diff --git a/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/TokenRefreshServerSuite.scala b/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/TokenRefreshServerSuite.scala index 62eb84d3e3323..7807bab7d8bd2 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/TokenRefreshServerSuite.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/TokenRefreshServerSuite.scala @@ -31,7 +31,7 @@ class TokenRefreshServerSuite extends FunSuite with BeforeAndAfter { private val configKeyPrefix = "hadoop-token-refresh-server" private val config = ConfigFactory.parseMap( - Map(s"$configKeyPrefix.kerberosPrincipal" -> "my-principla", + Map(s"$configKeyPrefix.kerberosPrincipal" -> "my-principal", s"$configKeyPrefix.scanAllNamespaces" -> true, s"$configKeyPrefix.namespaceToScan" -> "my-namespace").asJava) private val settings = new Settings(config) diff --git a/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/TokenRefreshServiceSuite.scala b/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/TokenRefreshServiceSuite.scala new file mode 100644 index 0000000000000..e9c5adfa708ff --- /dev/null +++ b/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/TokenRefreshServiceSuite.scala @@ -0,0 +1,234 @@ +/* + * 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.security.kubernetes + +import java.util.concurrent.TimeUnit + +import scala.collection.JavaConverters._ +import akka.actor.{ActorRef, ActorSystem, Cancellable, Props, Scheduler} +import akka.testkit.{TestActorRef, TestKit} +import com.typesafe.config.ConfigFactory +import io.fabric8.kubernetes.api.model.SecretBuilder +import io.fabric8.kubernetes.client.KubernetesClient +import org.apache.hadoop.security.token.{Token, TokenIdentifier} +import org.mockito._ +import org.mockito.Matchers.{any, same} +import org.mockito.Mockito.{verify, verifyNoMoreInteractions, when} +import org.scalatest.{BeforeAndAfter, FunSuiteLike} +import org.apache.spark.security.kubernetes.constants._ + +import scala.concurrent.ExecutionContext +import scala.concurrent.duration.{Duration, FiniteDuration} + +class TokenRefreshServiceSuite extends TestKit(ActorSystem("test")) + with FunSuiteLike with BeforeAndAfter { + + private val configKeyPrefix = "hadoop-token-refresh-server" + private val configMap = Map(s"$configKeyPrefix.kerberosPrincipal" -> "my-principal", + s"$configKeyPrefix.scanAllNamespaces" -> true, + s"$configKeyPrefix.namespaceToScan" -> "my-namespace") + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var kubernetesClient: KubernetesClient = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var scheduler: Scheduler = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var ugi: UgiUtil = _ + private val config = ConfigFactory.parseMap(configMap.asJava) + private val settings = new Settings(config) + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var clock: Clock = _ + private val nowMillis = 1500000000L + private var actorRef: TestActorRef[TokenRefreshService] = _ + private val reloginInterval = Duration(REFRESH_SERVER_KERBEROS_RELOGIN_INTERVAL_MILLIS, + TimeUnit.MILLISECONDS) + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var reloginCommandCancellable: Cancellable = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var reloginTaskCancellable: Cancellable = _ + private val starterTaskDelay = Duration(STARTER_TASK_INITIAL_DELAY_MILLIS, TimeUnit.MILLISECONDS) + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var starterTaskCancellable: Cancellable = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var starterTask1Cancellable: Cancellable = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var starterTask2Cancellable: Cancellable = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var starterTask3Cancellable: Cancellable = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var renewTaskCancellable: Cancellable = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var token1: Token[_ <: TokenIdentifier] = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var token2: Token[_ <: TokenIdentifier] = _ + + before { + MockitoAnnotations.initMocks(this) + when(scheduler.schedule(Matchers.eq(reloginInterval), Matchers.eq(reloginInterval), + any(classOf[ActorRef]), + Matchers.eq(Relogin))( + any(classOf[ExecutionContext]), + any(classOf[ActorRef]))) + .thenReturn(reloginCommandCancellable) + when(clock.nowInMillis()).thenReturn(nowMillis) + actorRef = TestActorRef(Props(classOf[TokenRefreshService], kubernetesClient, scheduler, + ugi, settings, clock)) + } + + test("The token refresh service actor starts properly") { + verify(ugi).loginUserFromKeytab("my-principal", REFRESH_SERVER_KERBEROS_KEYTAB_PATH) + verify(scheduler).schedule(Matchers.eq(reloginInterval), Matchers.eq(reloginInterval), + same(actorRef), + Matchers.eq(Relogin))( + any(classOf[ExecutionContext]), + same(actorRef)) + val actor: TokenRefreshService = actorRef.underlyingActor + assert(actor.numExtraCancellables() == 1) + assert(actor.hasExtraCancellable(Relogin.getClass, reloginCommandCancellable)) + assert(actor.numPendingSecretTasks() == 0) + verifyNoMoreInteractions(scheduler) + } + + test("The Relogin command launches a ReloginTask") { + when(scheduler.scheduleOnce(any(classOf[FiniteDuration]), + any(classOf[Runnable]))( + any(classOf[ExecutionContext]))) + .thenReturn(reloginTaskCancellable) + actorRef ! Relogin + + val taskCaptor: ArgumentCaptor[Runnable] = ArgumentCaptor.forClass(classOf[Runnable]) + verify(scheduler).scheduleOnce(Matchers.eq(Duration(0, TimeUnit.MILLISECONDS)), + taskCaptor.capture())( + any(classOf[ExecutionContext])) + val task = taskCaptor.getValue + assert(task.getClass == classOf[ReloginTask]) + val actor: TokenRefreshService = actorRef.underlyingActor + assert(actor.numExtraCancellables() == 2) // Relogin and ReloginTask + assert(actor.hasExtraCancellable(classOf[ReloginTask], reloginTaskCancellable)) + assert(actor.numPendingSecretTasks() == 0) + } + + test("The StartRefresh command launches a StarterTask") { + when(scheduler.scheduleOnce(any(classOf[FiniteDuration]), + any(classOf[Runnable]))( + any(classOf[ExecutionContext]))) + .thenReturn(starterTaskCancellable) + val secret = new SecretBuilder() + .withNewMetadata() + .withUid("uid-0101") + .endMetadata() + .build() + actorRef ! StartRefresh(secret) + + val taskCaptor: ArgumentCaptor[Runnable] = ArgumentCaptor.forClass(classOf[Runnable]) + verify(scheduler).scheduleOnce(Matchers.eq(starterTaskDelay), + taskCaptor.capture())( + any(classOf[ExecutionContext])) + val task = taskCaptor.getValue + assert(task.getClass == classOf[StarterTask]) + val actor: TokenRefreshService = actorRef.underlyingActor + assert(actor.numExtraCancellables() == 1) // Relogin + assert(actor.numPendingSecretTasks() == 1) + assert(actor.hasSecretTaskCancellable("uid-0101", starterTaskCancellable)) + } + + test("The Renew command launches a RenewTask") { + when(scheduler.scheduleOnce(any(classOf[FiniteDuration]), + any(classOf[Runnable]))( + any(classOf[ExecutionContext]))) + .thenReturn(starterTaskCancellable, renewTaskCancellable) + val secret = new SecretBuilder() + .withNewMetadata() + .withUid("uid-0101") + .endMetadata() + .build() + actorRef ! StartRefresh(secret) + actorRef ! Renew(expireTime = nowMillis + 10000L, Map(), secret.getMetadata, + numConsecutiveErrors = 0) + + val taskCaptor: ArgumentCaptor[Runnable] = ArgumentCaptor.forClass(classOf[Runnable]) + val renewTaskDelay = Duration((10000L * 0.9).toLong, // 90% of expire time from now. + TimeUnit.MILLISECONDS) + verify(scheduler).scheduleOnce(Matchers.eq(renewTaskDelay), + taskCaptor.capture())( + any(classOf[ExecutionContext])) + val task = taskCaptor.getValue + assert(task.getClass == classOf[RenewTask]) + + val actor: TokenRefreshService = actorRef.underlyingActor + assert(actor.numExtraCancellables() == 1) // Relogin + assert(actor.numPendingSecretTasks() == 1) + assert(actor.hasSecretTaskCancellable("uid-0101", renewTaskCancellable)) + } + + test("The StopRefresh command cancels a RenewTask") { + when(scheduler.scheduleOnce(any(classOf[FiniteDuration]), + any(classOf[Runnable]))( + any(classOf[ExecutionContext]))) + .thenReturn(starterTaskCancellable, renewTaskCancellable) + val secret = new SecretBuilder() + .withNewMetadata() + .withUid("uid-0101") + .endMetadata() + .build() + actorRef ! StartRefresh(secret) + actorRef ! Renew(expireTime = nowMillis + 10000L, Map(), secret.getMetadata, + numConsecutiveErrors = 0) + actorRef ! StopRefresh(secret) + + verify(renewTaskCancellable).cancel() + val actor: TokenRefreshService = actorRef.underlyingActor + assert(actor.numExtraCancellables() == 1) // Relogin + assert(actor.numPendingSecretTasks() == 0) + } + + test("The UpdateSecretsToTrack command resets tasks set") { + val secret1 = new SecretBuilder() + .withNewMetadata() + .withUid("uid-0101") + .endMetadata() + .build() + val secret2 = new SecretBuilder() + .withNewMetadata() + .withUid("uid-0202") + .endMetadata() + .build() + val secret3 = new SecretBuilder() + .withNewMetadata() + .withUid("uid-0303") + .endMetadata() + .build() + when(scheduler.scheduleOnce(any(classOf[FiniteDuration]), + any(classOf[Runnable]))( + any(classOf[ExecutionContext]))) + .thenReturn(starterTask1Cancellable, // for secret1 + starterTask2Cancellable, // for secret2 + renewTaskCancellable, // for secret2 + starterTask3Cancellable) // for secret3 + actorRef ! UpdateSecretsToTrack(List(secret1)) + actorRef ! StartRefresh(secret2) + actorRef ! Renew(expireTime = nowMillis + 10000L, Map(), secret2.getMetadata, + numConsecutiveErrors = 0) + actorRef ! UpdateSecretsToTrack(List(secret2, secret3)) + + verify(starterTask1Cancellable).cancel() + val actor: TokenRefreshService = actorRef.underlyingActor + assert(actor.numExtraCancellables() == 1) // Relogin + assert(actor.numPendingSecretTasks() == 2) + assert(actor.hasSecretTaskCancellable("uid-0202", renewTaskCancellable)) + assert(actor.hasSecretTaskCancellable("uid-0303", starterTask3Cancellable)) + } +} From 87dedbc7a7472993adebee7c0a9b59dc3ce6491c Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Thu, 12 Oct 2017 17:52:49 -0700 Subject: [PATCH 38/43] Minor clean-up --- .../kubernetes/TokenRefreshServiceSuite.scala | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/TokenRefreshServiceSuite.scala b/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/TokenRefreshServiceSuite.scala index e9c5adfa708ff..f8c09218e93f9 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/TokenRefreshServiceSuite.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/TokenRefreshServiceSuite.scala @@ -19,6 +19,9 @@ package org.apache.spark.security.kubernetes import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ +import scala.concurrent.ExecutionContext +import scala.concurrent.duration.{Duration, FiniteDuration} + import akka.actor.{ActorRef, ActorSystem, Cancellable, Props, Scheduler} import akka.testkit.{TestActorRef, TestKit} import com.typesafe.config.ConfigFactory @@ -29,10 +32,9 @@ import org.mockito._ import org.mockito.Matchers.{any, same} import org.mockito.Mockito.{verify, verifyNoMoreInteractions, when} import org.scalatest.{BeforeAndAfter, FunSuiteLike} + import org.apache.spark.security.kubernetes.constants._ -import scala.concurrent.ExecutionContext -import scala.concurrent.duration.{Duration, FiniteDuration} class TokenRefreshServiceSuite extends TestKit(ActorSystem("test")) with FunSuiteLike with BeforeAndAfter { @@ -195,7 +197,7 @@ class TokenRefreshServiceSuite extends TestKit(ActorSystem("test")) assert(actor.numPendingSecretTasks() == 0) } - test("The UpdateSecretsToTrack command resets tasks set") { + test("The UpdateSecretsToTrack command resets the task set to track") { val secret1 = new SecretBuilder() .withNewMetadata() .withUid("uid-0101") @@ -218,11 +220,12 @@ class TokenRefreshServiceSuite extends TestKit(ActorSystem("test")) starterTask2Cancellable, // for secret2 renewTaskCancellable, // for secret2 starterTask3Cancellable) // for secret3 - actorRef ! UpdateSecretsToTrack(List(secret1)) - actorRef ! StartRefresh(secret2) + actorRef ! UpdateSecretsToTrack(List(secret1)) // This adds a task for secret1. + actorRef ! StartRefresh(secret2) // Adds the secret2 task. actorRef ! Renew(expireTime = nowMillis + 10000L, Map(), secret2.getMetadata, numConsecutiveErrors = 0) - actorRef ! UpdateSecretsToTrack(List(secret2, secret3)) + // This removes secret1, but not the recently added secret2. + actorRef ! UpdateSecretsToTrack(List(secret3)) verify(starterTask1Cancellable).cancel() val actor: TokenRefreshService = actorRef.underlyingActor From 4d1cb74b3bb07472b2b17030d8a9778b8d03c800 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Fri, 13 Oct 2017 13:27:34 -0700 Subject: [PATCH 39/43] Add unit tests for renew tasks --- .../kubernetes/TokenRefreshService.scala | 95 +++++--- .../security/kubernetes/RenewTaskSuite.scala | 202 ++++++++++++++++++ 2 files changed, 267 insertions(+), 30 deletions(-) create mode 100644 resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/RenewTaskSuite.scala diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index 9349ecaf6858a..e4c7d3e388ca5 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -181,12 +181,26 @@ private class UgiUtil { def loginUserFromKeytab(kerberosPrincipal: String, kerberosKeytabPath: String): Unit = UserGroupInformation.loginUserFromKeytab(kerberosPrincipal, kerberosKeytabPath) + def getLoginUser: UserGroupInformation = UserGroupInformation.getLoginUser + + def createProxyUser(user: String, realUser: UserGroupInformation): UserGroupInformation = + UserGroupInformation.createProxyUser(user, realUser) +} + +private class FileSystemUtil { + + def getFileSystem(hadoopConf: Configuration): FileSystem = FileSystem.get(hadoopConf) + + def renewToken(token: Token[_ <: TokenIdentifier], hadoopConf: Configuration): Long = + token.renew(hadoopConf) } -private class ReloginTask extends Runnable { +private class ReloginTask(ugi: UgiUtil) extends Runnable { + + def this() = this(new UgiUtil) override def run(): Unit = { - UserGroupInformation.getLoginUser.checkTGTAndReloginFromKeytab() + ugi.getLoginUser.checkTGTAndReloginFromKeytab() } } @@ -198,10 +212,13 @@ private class StarterTask(secret: Secret, private var hasError = false override def run(): Unit = { - val tokenToExpireTime = readTokensFromSecret() - logInfo(s"Read Hadoop tokens: $tokenToExpireTime") - val nextExpireTime = if (tokenToExpireTime.nonEmpty) { - tokenToExpireTime.values.min + val tokensToExpireTimes = readTokensFromSecret() + val tokenKeys = tokensToExpireTimes.keys.map( + token => token.getKind.toString + "@" + token.getService.toString).mkString(", ") + val nextExpireTime = if (tokensToExpireTimes.nonEmpty) { + val minExpireTime = tokensToExpireTimes.values.min + logInfo(s"Read Hadoop tokens: $tokenKeys with $minExpireTime") + minExpireTime } else { logWarning(s"Got an empty token list with secret ${secret.getMetadata.getUid} of" + s" ${secret.getMetadata.getSelfLink}") @@ -209,7 +226,7 @@ private class StarterTask(secret: Secret, getRetryTime } val numConsecutiveErrors = if (hasError) 1 else 0 - refreshService ! Renew(nextExpireTime, tokenToExpireTime, secret.getMetadata, + refreshService ! Renew(nextExpireTime, tokensToExpireTimes, secret.getMetadata, numConsecutiveErrors) } @@ -225,10 +242,9 @@ private class StarterTask(secret: Secret, val createTime = matcher.group(1).toLong val duration = matcher.group(2).toLong val expireTime = createTime + duration - val creds = new Credentials - creds.readTokenStorageStream(new DataInputStream(new ByteArrayInputStream( - Base64.decodeBase64(data)))) - creds.getAllTokens.asScala.toList.map { + val credentials = new Credentials + TokensSerializer.deserialize(credentials, Base64.decodeBase64(data)) + credentials.getAllTokens.asScala.toList.map { (_, expireTime) } }.toList.flatten.toMap @@ -241,7 +257,13 @@ private class RenewTask(renew: Renew, hadoopConf: Configuration, refreshService: ActorRef, kubernetesClient: KubernetesClient, - clock: Clock) extends Runnable with Logging { + clock: Clock, + ugi: UgiUtil, + fsUtil: FileSystemUtil) extends Runnable with Logging { + + def this(renew: Renew, hadoopConf: Configuration, refreshService: ActorRef, client: KubernetesClient, + clock: Clock) = this(renew, hadoopConf, refreshService, client, clock, + ugi = new UgiUtil, fsUtil = new FileSystemUtil) private var hasError = false @@ -249,14 +271,14 @@ private class RenewTask(renew: Renew, val deadline = renew.expireTime + RENEW_TASK_DEADLINE_LOOK_AHEAD_MILLIS val nowMillis = clock.nowInMillis() val newExpireTimeByToken : Map[Token[_ <: TokenIdentifier], Long] = - renew.tokenToExpireTime.map { + renew.tokensToExpireTimes.map { case (token, expireTime) => val (maybeNewToken, maybeNewExpireTime) = refresh(token, expireTime, deadline, nowMillis) (maybeNewToken, maybeNewExpireTime) } .toMap if (newExpireTimeByToken.nonEmpty) { - val newTokens = newExpireTimeByToken.keySet -- renew.tokenToExpireTime.keySet + val newTokens = newExpireTimeByToken.keySet -- renew.tokensToExpireTimes.keySet if (newTokens.nonEmpty) { writeTokensToSecret(newExpireTimeByToken, nowMillis) } @@ -309,7 +331,7 @@ private class RenewTask(renew: Renew, try { logDebug(s"Renewing token $token with current expire time $expireTime," + s" deadline $deadline, now $nowMillis") - val newExpireTime = token.renew(hadoopConf) + val newExpireTime = fsUtil.renewToken(token, hadoopConf) logDebug(s"Renewed token $token. Next expire time $newExpireTime") newExpireTime } catch { @@ -334,12 +356,12 @@ private class RenewTask(renew: Renew, realUser.toString } val credentials = new Credentials - val ugi = UserGroupInformation.createProxyUser(user, UserGroupInformation.getLoginUser) - val newToken = ugi.doAs(new PrivilegedExceptionAction[Token[_ <: TokenIdentifier]] { + val proxyUgi = ugi.createProxyUser(user, ugi.getLoginUser) + val newToken = proxyUgi.doAs(new PrivilegedExceptionAction[Token[_ <: TokenIdentifier]] { override def run() : Token[_ <: TokenIdentifier] = { - val fs = FileSystem.get(hadoopConf) - val tokens = fs.addDelegationTokens(UserGroupInformation.getLoginUser.getUserName, + val fs = fsUtil.getFileSystem(hadoopConf) + val tokens = fs.addDelegationTokens(ugi.getLoginUser.getUserName, credentials) tokens(0) } @@ -347,14 +369,11 @@ private class RenewTask(renew: Renew, newToken } - private def writeTokensToSecret(tokenToExpire: Map[Token[_ <: TokenIdentifier], Long], + private def writeTokensToSecret(tokensToExpireTimes: Map[Token[_ <: TokenIdentifier], Long], nowMillis: Long): Unit = { - val durationUntilExpire = tokenToExpire.values.min - nowMillis + val durationUntilExpire = tokensToExpireTimes.values.min - nowMillis val key = s"$SECRET_DATA_ITEM_KEY_PREFIX_HADOOP_TOKENS$nowMillis-$durationUntilExpire" - val credentials = new Credentials() - tokenToExpire.keys.foreach(token => credentials.addToken(token.getService, token)) - val serialized = serializeCredentials(credentials) - val value = Base64.encodeBase64String(serialized) + val value = TokensSerializer.serializeBase64(tokensToExpireTimes.keys) val secretMeta = renew.secretMeta val editor = kubernetesClient.secrets .inNamespace(secretMeta.getNamespace) @@ -368,11 +387,23 @@ private class RenewTask(renew: Renew, // where some newly launching executors may access the previous token. dataItemKeys.dropRight(2).foreach(editor.removeFromData) editor.done - logInfo(s"Wrote new tokens $tokenToExpire to a data item $key in secret ${secretMeta.getUid}" + - s" of ${secretMeta.getSelfLink}") + logInfo(s"Wrote new tokens $tokensToExpireTimes to a data item $key in" + + s" secret ${secretMeta.getUid} of ${secretMeta.getSelfLink}") } - private def serializeCredentials(credentials: Credentials) = { + private def getRetryTime = clock.nowInMillis() + RENEW_TASK_RETRY_TIME_MILLIS +} + +private object TokensSerializer { + + def serializeBase64(tokens: Iterable[Token[_ <: TokenIdentifier]]): String = { + val credentials = new Credentials() + tokens.foreach(token => credentials.addToken(token.getService, token)) + val serialized = serializeCredentials(credentials) + Base64.encodeBase64String(serialized) + } + + private def serializeCredentials(credentials: Credentials): Array[Byte] = { val byteStream = new ByteArrayOutputStream val dataStream = new DataOutputStream(byteStream) credentials.writeTokenStorageToStream(dataStream) @@ -380,7 +411,11 @@ private class RenewTask(renew: Renew, byteStream.toByteArray } - private def getRetryTime = clock.nowInMillis() + RENEW_TASK_RETRY_TIME_MILLIS + def deserialize(credentials: Credentials, data: Array[Byte]): Unit = { + val byteStream = new ByteArrayInputStream(data) + val dataStream = new DataInputStream(byteStream) + credentials.readTokenStorageStream(dataStream) + } } private class Clock { @@ -393,7 +428,7 @@ private case object Relogin extends Command private case class UpdateSecretsToTrack(secrets: List[Secret]) extends Command private case class StartRefresh(secret: Secret) extends Command private case class Renew(expireTime: Long, - tokenToExpireTime: Map[Token[_ <: TokenIdentifier], Long], + tokensToExpireTimes: Map[Token[_ <: TokenIdentifier], Long], secretMeta: ObjectMeta, numConsecutiveErrors: Int) extends Command private case class StopRefresh(secret: Secret) extends Command diff --git a/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/RenewTaskSuite.scala b/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/RenewTaskSuite.scala new file mode 100644 index 0000000000000..5caea2f3f31da --- /dev/null +++ b/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/RenewTaskSuite.scala @@ -0,0 +1,202 @@ +/* + * 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.security.kubernetes + +import java.security.PrivilegedExceptionAction + +import scala.collection.JavaConverters._ + +import akka.actor.ActorSystem +import akka.testkit.{TestKit, TestProbe} +import io.fabric8.kubernetes.api.model.{DoneableSecret, Secret, SecretBuilder, SecretList} +import io.fabric8.kubernetes.client.KubernetesClient +import io.fabric8.kubernetes.client.dsl.{MixedOperation, NonNamespaceOperation, Resource} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier +import org.apache.hadoop.io.Text +import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.security.token.{Token, TokenIdentifier} +import org.mockito.{Answers, Matchers, Mock, MockitoAnnotations} +import org.mockito.Matchers.{any, anyString} +import org.mockito.Mockito.{doReturn, verify, when} +import org.scalatest.{BeforeAndAfter, FunSuiteLike} + +class RenewTaskSuite extends TestKit(ActorSystem("test")) with FunSuiteLike with BeforeAndAfter { + + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var ugi: UgiUtil = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var fsUtil: FileSystemUtil = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var loginUser: UserGroupInformation = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var proxyUser: UserGroupInformation = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var hadoopConf: Configuration = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var kubernetesClient: KubernetesClient = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var clock: Clock = _ + private val tokenRefreshServiceProbe = TestProbe() + private val tokenRefreshService = tokenRefreshServiceProbe.ref + private val createTime1 = 1500100000000L // 2017/07/14-23:26:40 + private val duration1 = 86400000L // one day in millis. + private val token1ExpireTime = createTime1 + duration1 + private val maxDate1 = createTime1 + 7 * 86400000L // 7 days + private val token1 = buildHdfsToken(owner="john", renewer="refresh-server", realUser="john", + password="token1-password", service="196.0.0.1:8020", createTime1, maxDate1) + private val createTime2 = 1500200000000L // 2017/07/16-03:13:20 + private val duration2 = 86400000L // one day in millis. + private val token2ExpireTime = createTime2 + duration2 + // maxDate2 below is just over the expire time. RenewTask will get a brand new token. + private val maxDate2 = token2ExpireTime + 60 * 60 * 1000 // One hour after the expire time. + private val token2 = buildHdfsToken(owner="john", renewer="refresh-server", realUser="john", + password="token2-password", service="196.0.0.1:8020", createTime2, maxDate2) + private val secret1 = new SecretBuilder() + .withNewMetadata() + .withUid("uid-0101") + .withNamespace("namespace1") + .withName("secret1") + .endMetadata() + .withData(Map( + s"hadoop-tokens-$createTime1-$duration1" -> + TokensSerializer.serializeBase64(List(token1)) + ).asJava) + .build() + private val secret2 = new SecretBuilder() + .withNewMetadata() + .withUid("uid-0202") + .withNamespace("namespace2") + .withName("secret2") + .endMetadata() + .withData(Map( + s"hadoop-tokens-$createTime1-$duration1" -> + TokensSerializer.serializeBase64(List(token1)), + s"hadoop-tokens-$createTime2-$duration2" -> + TokensSerializer.serializeBase64(List(token2)) + ).asJava) + .build() + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var secretsOp: MixedOperation[Secret, SecretList, DoneableSecret, + Resource[Secret, DoneableSecret]] = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var secretsOpInNamespace: NonNamespaceOperation[Secret, SecretList, DoneableSecret, + Resource[Secret, DoneableSecret]] = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var secretOpWithName: Resource[Secret, DoneableSecret] = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var secretEditor: DoneableSecret = _ + + before { + MockitoAnnotations.initMocks(this) + when(ugi.getLoginUser).thenReturn(loginUser) + when(ugi.createProxyUser(anyString, Matchers.eq(loginUser))).thenReturn(proxyUser) + } + + test("ReloginTask logins on Kerberos again") { + val task = new ReloginTask(ugi) + task.run() + + verify(loginUser).checkTGTAndReloginFromKeytab() + } + + test("StarterTask reads a secret and schedules a renew task") { + val task = new StarterTask(secret1, hadoopConf, tokenRefreshService, clock) + task.run() + + val renewCommand = Renew(expireTime = token1ExpireTime, + Map(token1 -> token1ExpireTime), + secret1.getMetadata, + numConsecutiveErrors = 0) + tokenRefreshServiceProbe.expectMsg(renewCommand) + } + + test("RenewTask just renews an existing token if maxDate is far way") { + // maxDate1 of token1 is far away. So the RenewTask will only renew the existing token. + val nowMillis = token1ExpireTime - 60 * 60 * 1000 // One hour before token2 expire time. + when(clock.nowInMillis()).thenReturn(nowMillis) + val newExpireTime = nowMillis + duration1 + when(fsUtil.renewToken(token1, hadoopConf)).thenReturn(newExpireTime) + val renewCommand = Renew(expireTime = token1ExpireTime, + Map(token1 -> token1ExpireTime), + secret1.getMetadata, + numConsecutiveErrors = 0) + val task = new RenewTask(renewCommand, hadoopConf, tokenRefreshService, kubernetesClient, clock, + ugi, fsUtil) + task.run() + + val newRenewCommand = Renew(expireTime = newExpireTime, + Map(token1 -> newExpireTime), + secret1.getMetadata, + numConsecutiveErrors = 0) + tokenRefreshServiceProbe.expectMsg(newRenewCommand) // Sent a new Renew command to the service. + } + + test("RenewTask obtains a new token and write it back to the secret") { + // maxDate2 of token2 is just over the expire time. So the RenewTask will get a brand new token. + val nowMillis = token2ExpireTime - 60 * 60 * 1000 // One hour before token2 expire time. + when(clock.nowInMillis()).thenReturn(nowMillis) + val duration3 = 86400000L // one day in millis. + val maxDate3 = nowMillis + 7 * 86400000L // 7 days + val token3ExpireTime = nowMillis + duration3 + val token3 = buildHdfsToken(owner="john", renewer="refresh-server", realUser="john", + password="token3-password", service="196.0.0.1:8020", nowMillis, maxDate3) + doReturn(token3).when(proxyUser) + .doAs(any(classOf[PrivilegedExceptionAction[Token[_ <: TokenIdentifier]]])) + when(fsUtil.renewToken(token3, hadoopConf)).thenReturn(token3ExpireTime) + when(kubernetesClient.secrets()).thenReturn(secretsOp) + when(secretsOp.inNamespace("namespace2")).thenReturn(secretsOpInNamespace) + when(secretsOpInNamespace.withName("secret2")).thenReturn(secretOpWithName) + when(secretOpWithName.edit()).thenReturn(secretEditor) + when(secretEditor.getData).thenReturn(Map( + s"hadoop-tokens-$createTime1-$duration1" -> TokensSerializer.serializeBase64(List(token1)), + s"hadoop-tokens-$createTime2-$duration2" -> TokensSerializer.serializeBase64(List(token2)), + s"hadoop-tokens-$nowMillis-$duration3" -> TokensSerializer.serializeBase64(List(token3)) + ).asJava) + when(secretEditor.removeFromData(anyString())).thenReturn(secretEditor) + val renewCommand = Renew(expireTime = token2ExpireTime, + Map(token2 -> token2ExpireTime), + secret2.getMetadata, + numConsecutiveErrors = 0) + val task = new RenewTask(renewCommand, hadoopConf, tokenRefreshService, kubernetesClient, clock, + ugi, fsUtil) + task.run() + + verify(secretEditor) + .addToData(s"hadoop-tokens-$nowMillis-$duration3", // Added the new token to the secret. + TokensSerializer.serializeBase64(List(token3))) + verify(secretEditor) + .removeFromData(s"hadoop-tokens-$createTime1-$duration1") // Removed the oldest token. + val newRenewCommand = Renew(expireTime = token3ExpireTime, + Map(token3 -> token3ExpireTime), + secret2.getMetadata, + numConsecutiveErrors = 0) + tokenRefreshServiceProbe.expectMsg(newRenewCommand) // Sent a new Renew command to the service. + } + + private def buildHdfsToken(owner: String, renewer: String, realUser: String, password: String, + service: String, + issueDate: Long, + maxDate:Long): Token[_ <: TokenIdentifier] = { + val hdfsTokenIdentifier = new DelegationTokenIdentifier(new Text(owner), new Text(renewer), + new Text(realUser)) + hdfsTokenIdentifier.setIssueDate(issueDate) + hdfsTokenIdentifier.setMaxDate(maxDate) + new Token[DelegationTokenIdentifier](hdfsTokenIdentifier.getBytes, password.getBytes, + DelegationTokenIdentifier.HDFS_DELEGATION_KIND, new Text(service)) + } +} From 8997f0422697e73ae2d14bfa45666b54be823318 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Fri, 13 Oct 2017 13:52:58 -0700 Subject: [PATCH 40/43] Verify test results more --- .../security/kubernetes/RenewTaskSuite.scala | 19 ++++++++++++++++--- 1 file changed, 16 insertions(+), 3 deletions(-) diff --git a/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/RenewTaskSuite.scala b/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/RenewTaskSuite.scala index 5caea2f3f31da..88cbfe8dba1e5 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/RenewTaskSuite.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/test/scala/org/apache/spark/security/kubernetes/RenewTaskSuite.scala @@ -19,18 +19,18 @@ package org.apache.spark.security.kubernetes import java.security.PrivilegedExceptionAction import scala.collection.JavaConverters._ - import akka.actor.ActorSystem import akka.testkit.{TestKit, TestProbe} import io.fabric8.kubernetes.api.model.{DoneableSecret, Secret, SecretBuilder, SecretList} import io.fabric8.kubernetes.client.KubernetesClient import io.fabric8.kubernetes.client.dsl.{MixedOperation, NonNamespaceOperation, Resource} import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hdfs.DistributedFileSystem import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier import org.apache.hadoop.io.Text -import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.security.token.{Token, TokenIdentifier} -import org.mockito.{Answers, Matchers, Mock, MockitoAnnotations} +import org.mockito._ import org.mockito.Matchers.{any, anyString} import org.mockito.Mockito.{doReturn, verify, when} import org.scalatest.{BeforeAndAfter, FunSuiteLike} @@ -42,6 +42,8 @@ class RenewTaskSuite extends TestKit(ActorSystem("test")) with FunSuiteLike with @Mock(answer = Answers.RETURNS_SMART_NULLS) private var fsUtil: FileSystemUtil = _ @Mock(answer = Answers.RETURNS_SMART_NULLS) + private var hdfs: DistributedFileSystem = _ + @Mock(answer = Answers.RETURNS_SMART_NULLS) private var loginUser: UserGroupInformation = _ @Mock(answer = Answers.RETURNS_SMART_NULLS) private var proxyUser: UserGroupInformation = _ @@ -104,6 +106,7 @@ class RenewTaskSuite extends TestKit(ActorSystem("test")) with FunSuiteLike with before { MockitoAnnotations.initMocks(this) when(ugi.getLoginUser).thenReturn(loginUser) + when(loginUser.getUserName).thenReturn("refresh-server") when(ugi.createProxyUser(anyString, Matchers.eq(loginUser))).thenReturn(proxyUser) } @@ -186,6 +189,16 @@ class RenewTaskSuite extends TestKit(ActorSystem("test")) with FunSuiteLike with secret2.getMetadata, numConsecutiveErrors = 0) tokenRefreshServiceProbe.expectMsg(newRenewCommand) // Sent a new Renew command to the service. + + val actionCaptor: ArgumentCaptor[PrivilegedExceptionAction[Token[_ <: TokenIdentifier]]] = + ArgumentCaptor.forClass(classOf[PrivilegedExceptionAction[Token[_ <: TokenIdentifier]]]) + verify(proxyUser).doAs(actionCaptor.capture()) + val action = actionCaptor.getValue + when(fsUtil.getFileSystem(hadoopConf)) + .thenReturn(hdfs) + doReturn(Array(token3)).when(hdfs) + .addDelegationTokens(Matchers.eq("refresh-server"), any(classOf[Credentials])) + assert(action.run() == token3) } private def buildHdfsToken(owner: String, renewer: String, realUser: String, password: String, From 2ed55af9704be3b590918ec19fd1a2fbe032e592 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Wed, 18 Oct 2017 11:14:41 -0700 Subject: [PATCH 41/43] Rename the new profile to kubernetes-hdfs-extra --- pom.xml | 6 +++--- resource-managers/kubernetes/README.md | 7 ++++--- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/pom.xml b/pom.xml index 5e28d77b64b18..6b06738eea530 100644 --- a/pom.xml +++ b/pom.xml @@ -2620,9 +2620,9 @@ - - kubernetes-extra + + kubernetes-hdfs-extra diff --git a/resource-managers/kubernetes/README.md b/resource-managers/kubernetes/README.md index a83a0311c0974..3a9b30decf0b6 100644 --- a/resource-managers/kubernetes/README.md +++ b/resource-managers/kubernetes/README.md @@ -46,10 +46,11 @@ Below is a list of the submodules for this cluster manager and what they do. # Building Extra Submodules for Spark with Kubernetes There are non-core extra submodules such as token-refresh-server. To build -those, use the `kubernetes-extra` profile when invoking Maven. For example, -to build the token-refresh-server submodule: +those, use the `kubernetes-*-extra` profiles like `kubernetes-hdfs-extra` +when invoking Maven. For example, to build the token-refresh-server submodule: - build/mvn package -Pkubernetes-extra -pl resource-managers/kubernetes/token-refresh-rver -am + build/mvn package -Pkubernetes-hdfs-extra \ + -pl resource-managers/kubernetes/token-refresh-server -am Some of these submodules are helper Kubernetes services. They need not be part of the Spark distribution. The distribution build script will not include From 01be03edb25f364b4a0bc52a3578ce8538f328bb Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Wed, 18 Oct 2017 16:56:52 -0700 Subject: [PATCH 42/43] Fix style issues --- .../spark/security/kubernetes/Logging.scala | 20 ++++++++++++++-- .../security/kubernetes/SecretFinder.scala | 6 +++-- .../kubernetes/TokenRefreshServer.scala | 16 ++++++++----- .../kubernetes/TokenRefreshService.scala | 24 ++++++++++--------- 4 files changed, 45 insertions(+), 21 deletions(-) diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/Logging.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/Logging.scala index d34b592b86aa4..83feb88fc33e8 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/Logging.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/Logging.scala @@ -1,10 +1,26 @@ +/* + * 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.security.kubernetes -import org.apache.log4j.{LogManager, Logger, Priority} +import org.apache.log4j.{Logger, LogManager, Priority} trait Logging { - private var log : Logger = LogManager.getLogger(this.getClass) + private val log: Logger = LogManager.getLogger(this.getClass) protected def logDebug(msg: => String) = if (log.isDebugEnabled) log.debug(msg) diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala index e96e1829c9f6c..de7e134cadc76 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/SecretFinder.scala @@ -20,6 +20,7 @@ import java.lang import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ +import scala.concurrent.duration.Duration import scala.concurrent.ExecutionContext.Implicits.global import akka.actor.{ActorRef, Scheduler} @@ -27,9 +28,9 @@ import io.fabric8.kubernetes.api.model.{Secret, SecretList} import io.fabric8.kubernetes.client._ import io.fabric8.kubernetes.client.Watcher.Action import io.fabric8.kubernetes.client.dsl.FilterWatchListDeletable + import org.apache.spark.security.kubernetes.constants._ -import scala.concurrent.duration.Duration private trait SecretSelection { @@ -97,5 +98,6 @@ private class SecretWatcher(refreshService: ActorRef) extends Watcher[Secret] wi private object SecretFinder { def apply(refreshService: ActorRef, scheduler: Scheduler, client: KubernetesClient, - settings: Settings) = new SecretFinder(refreshService, scheduler, client, settings) + settings: Settings): SecretFinder = + new SecretFinder(refreshService, scheduler, client, settings) } diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala index 8dd55cfce953e..02731c90dea7f 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshServer.scala @@ -53,17 +53,19 @@ private class Server(injector: Injector) { private class Injector { - def newActorSystem() = ActorSystem("TokenRefreshServer") + def newActorSystem(): ActorSystem = ActorSystem("TokenRefreshServer") - def newKubernetesClient() : KubernetesClient = new DefaultKubernetesClient() + def newKubernetesClient(): KubernetesClient = new DefaultKubernetesClient() - def newSettings() = new Settings() + def newSettings(): Settings = new Settings() - def newTokenRefreshService(actorSystem: ActorSystem, client: KubernetesClient, settings: Settings) - = TokenRefreshService(actorSystem, client, settings) + def newTokenRefreshService(actorSystem: ActorSystem, client: KubernetesClient, + settings: Settings): ActorRef = + TokenRefreshService(actorSystem, client, settings) def newSecretFinder(refreshService: ActorRef, client: KubernetesClient, scheduler: Scheduler, - settings: Settings) = SecretFinder(refreshService, scheduler, client, settings) + settings: Settings): SecretFinder = + SecretFinder(refreshService, scheduler, client, settings) } private class Settings(config: Config = ConfigFactory.load) { @@ -99,7 +101,9 @@ private class CommandLine(args: List[String]) { } private def usage(): Unit = { + // scalastyle:off println println("Usage: TokenRefreshServer [--verbose | -v] [--debug | -d]") + // scalastyle:on println } } diff --git a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala index e4c7d3e388ca5..01d168675a681 100644 --- a/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala +++ b/resource-managers/kubernetes/token-refresh-server/src/main/scala/org/apache/spark/security/kubernetes/TokenRefreshService.scala @@ -21,21 +21,22 @@ import java.security.PrivilegedExceptionAction import java.util.concurrent.TimeUnit import java.util.regex.Pattern -import scala.collection.JavaConverters._ import scala.collection.mutable -import scala.concurrent.duration.Duration +import scala.collection.JavaConverters._ import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.duration.Duration + import akka.actor.{Actor, ActorRef, ActorSystem, Cancellable, Props, Scheduler} -import com.google.common.annotations.VisibleForTesting import io.fabric8.kubernetes.api.model.{ObjectMeta, Secret} import io.fabric8.kubernetes.client.KubernetesClient import org.apache.commons.codec.binary.Base64 import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier -import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.security.token.{Token, TokenIdentifier} +import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier + import org.apache.spark.security.kubernetes.constants._ @@ -156,19 +157,19 @@ private class TokenRefreshService(kubernetesClient: KubernetesClient, scheduler: private def getSecretUid(secret: ObjectMeta) = secret.getUid - @VisibleForTesting + // Exposed for testing private[kubernetes] def numExtraCancellables() = extraCancellableByClass.size - @VisibleForTesting + // Exposed for testing private[kubernetes] def hasExtraCancellable(key: Class[_], expected: Cancellable): Boolean = { val value = extraCancellableByClass.get(key) value.nonEmpty && expected == value.get } - @VisibleForTesting + // Exposed for testing private[kubernetes] def numPendingSecretTasks() = secretUidToTaskHandle.size - @VisibleForTesting + // Exposed for testing private[kubernetes] def hasSecretTaskCancellable(secretUid: String, expected: Cancellable) : Boolean = { val value = secretUidToTaskHandle.get(secretUid) @@ -261,9 +262,10 @@ private class RenewTask(renew: Renew, ugi: UgiUtil, fsUtil: FileSystemUtil) extends Runnable with Logging { - def this(renew: Renew, hadoopConf: Configuration, refreshService: ActorRef, client: KubernetesClient, - clock: Clock) = this(renew, hadoopConf, refreshService, client, clock, - ugi = new UgiUtil, fsUtil = new FileSystemUtil) + def this(renew: Renew, hadoopConf: Configuration, refreshService: ActorRef, + client: KubernetesClient, clock: Clock) = + this(renew, hadoopConf, refreshService, client, clock, ugi = new UgiUtil, + fsUtil = new FileSystemUtil) private var hasError = false From 0baaf0bbfc9169c2957c7f4e36cae66afb98b5f7 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Thu, 19 Oct 2017 11:25:54 -0700 Subject: [PATCH 43/43] Fix Hadoop 2.7 dependency issue --- resource-managers/kubernetes/token-refresh-server/pom.xml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/resource-managers/kubernetes/token-refresh-server/pom.xml b/resource-managers/kubernetes/token-refresh-server/pom.xml index ca53974285d4e..2da889ed46419 100644 --- a/resource-managers/kubernetes/token-refresh-server/pom.xml +++ b/resource-managers/kubernetes/token-refresh-server/pom.xml @@ -51,6 +51,13 @@ org.apache.hadoop hadoop-client + + + org.codehaus.jackson + jackson-mapper-asl + provided + commons-logging commons-logging