|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | +package org.apache.spark.deploy |
| 18 | + |
| 19 | +import java.util.concurrent.{Executors, TimeUnit} |
| 20 | +import java.util.{Comparator, Arrays} |
| 21 | + |
| 22 | +import com.google.common.primitives.Longs |
| 23 | +import org.apache.hadoop.conf.Configuration |
| 24 | +import org.apache.hadoop.fs.{PathFilter, FileStatus, Path, FileSystem} |
| 25 | +import org.apache.hadoop.security.{Credentials, UserGroupInformation} |
| 26 | + |
| 27 | +import org.apache.spark.util.Utils |
| 28 | +import org.apache.spark.{Logging, SparkConf} |
| 29 | + |
| 30 | +private[spark] class ExecutorDelegationTokenUpdater( |
| 31 | + sparkConf: SparkConf, |
| 32 | + hadoopConf: Configuration) extends Logging { |
| 33 | + |
| 34 | + @volatile private var lastCredentialsFileSuffix = 0 |
| 35 | + |
| 36 | + private lazy val delegationTokenRenewer = |
| 37 | + Executors.newSingleThreadScheduledExecutor( |
| 38 | + Utils.namedThreadFactory("Delegation Token Refresh Thread")) |
| 39 | + |
| 40 | + // On the executor, this thread wakes up and picks up new tokens from HDFS, if any. |
| 41 | + private lazy val executorUpdaterRunnable = |
| 42 | + new Runnable { |
| 43 | + override def run(): Unit = Utils.logUncaughtExceptions(updateCredentialsIfRequired()) |
| 44 | + } |
| 45 | + |
| 46 | + def updateCredentialsIfRequired(): Unit = { |
| 47 | + try { |
| 48 | + sparkConf.getOption("spark.yarn.credentials.file").foreach { credentialsFile => |
| 49 | + val credentials = UserGroupInformation.getCurrentUser.getCredentials |
| 50 | + val credentialsFilePath = new Path(credentialsFile) |
| 51 | + val remoteFs = FileSystem.get(hadoopConf) |
| 52 | + SparkHadoopUtil.get.listFilesSorted( |
| 53 | + remoteFs, credentialsFilePath.getParent, credentialsFilePath.getName, ".tmp") |
| 54 | + .lastOption.foreach { credentialsStatus => |
| 55 | + val suffix = getSuffixForCredentialsPath(credentialsStatus) |
| 56 | + if (suffix > lastCredentialsFileSuffix) { |
| 57 | + logInfo("Reading new delegation tokens from " + credentialsStatus.getPath) |
| 58 | + val newCredentials = getCredentialsFromHDFSFile(remoteFs, credentialsStatus.getPath) |
| 59 | + lastCredentialsFileSuffix = suffix |
| 60 | + UserGroupInformation.getCurrentUser.addCredentials(newCredentials) |
| 61 | + val totalValidity = SparkHadoopUtil.get.getLatestTokenValidity(credentials) - |
| 62 | + credentialsStatus.getModificationTime |
| 63 | + val timeToRunRenewal = |
| 64 | + credentialsStatus.getModificationTime + (0.8 * totalValidity).toLong |
| 65 | + val timeFromNowToRenewal = timeToRunRenewal - System.currentTimeMillis() |
| 66 | + logInfo("Updated delegation tokens, will check for new tokens in " + |
| 67 | + timeFromNowToRenewal + " millis") |
| 68 | + delegationTokenRenewer.schedule( |
| 69 | + executorUpdaterRunnable, timeFromNowToRenewal, TimeUnit.MILLISECONDS) |
| 70 | + } else { |
| 71 | + // Check every hour to see if new credentials arrived. |
| 72 | + logInfo("Updated delegation tokens were expected, but the driver has not updated the " + |
| 73 | + "tokens yet, will check again in an hour.") |
| 74 | + delegationTokenRenewer.schedule(executorUpdaterRunnable, 1, TimeUnit.HOURS) |
| 75 | + } |
| 76 | + } |
| 77 | + } |
| 78 | + } catch { |
| 79 | + // Since the file may get deleted while we are reading it, catch the Exception and come |
| 80 | + // back in an hour to try again |
| 81 | + case e: Exception => |
| 82 | + logWarning("Error while trying to update credentials, will try again in 1 hour", e) |
| 83 | + delegationTokenRenewer.schedule(executorUpdaterRunnable, 1, TimeUnit.HOURS) |
| 84 | + } |
| 85 | + } |
| 86 | + |
| 87 | + private def getCredentialsFromHDFSFile( |
| 88 | + remoteFs: FileSystem, |
| 89 | + tokenPath: Path): Credentials = { |
| 90 | + val stream = remoteFs.open(tokenPath) |
| 91 | + try { |
| 92 | + val newCredentials = new Credentials() |
| 93 | + newCredentials.readFields(stream) |
| 94 | + newCredentials |
| 95 | + } finally { |
| 96 | + stream.close() |
| 97 | + } |
| 98 | + } |
| 99 | + |
| 100 | + def stop(): Unit = { |
| 101 | + delegationTokenRenewer.shutdown() |
| 102 | + } |
| 103 | + |
| 104 | + private def getSuffixForCredentialsPath(credentialsStatus: FileStatus): Int = { |
| 105 | + val fileName = credentialsStatus.getPath.getName |
| 106 | + fileName.substring(fileName.lastIndexOf("-") + 1).toInt |
| 107 | + } |
| 108 | +} |
0 commit comments