001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018
019package org.apache.hadoop.mapreduce.security;
020
021import java.io.IOException;
022import java.util.HashSet;
023import java.util.Set;
024
025import org.apache.commons.logging.Log;
026import org.apache.commons.logging.LogFactory;
027import org.apache.hadoop.classification.InterfaceAudience;
028import org.apache.hadoop.classification.InterfaceStability;
029import org.apache.hadoop.conf.Configuration;
030import org.apache.hadoop.fs.FileSystem;
031import org.apache.hadoop.fs.Path;
032import org.apache.hadoop.io.Text;
033import org.apache.hadoop.mapred.JobConf;
034import org.apache.hadoop.mapred.Master;
035import org.apache.hadoop.mapreduce.MRJobConfig;
036import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
037import org.apache.hadoop.security.Credentials;
038import org.apache.hadoop.security.UserGroupInformation;
039import org.apache.hadoop.security.token.Token;
040import org.apache.hadoop.security.token.TokenIdentifier;
041
042
043/**
044 * This class provides user facing APIs for transferring secrets from
045 * the job client to the tasks.
046 * The secrets can be stored just before submission of jobs and read during
047 * the task execution.  
048 */
049@InterfaceAudience.Public
050@InterfaceStability.Evolving
051public class TokenCache {
052  
053  private static final Log LOG = LogFactory.getLog(TokenCache.class);
054
055  
056  /**
057   * auxiliary method to get user's secret keys..
058   * @param alias
059   * @return secret key from the storage
060   */
061  public static byte[] getSecretKey(Credentials credentials, Text alias) {
062    if(credentials == null)
063      return null;
064    return credentials.getSecretKey(alias);
065  }
066  
067  /**
068   * Convenience method to obtain delegation tokens from namenodes 
069   * corresponding to the paths passed.
070   * @param credentials
071   * @param ps array of paths
072   * @param conf configuration
073   * @throws IOException
074   */
075  public static void obtainTokensForNamenodes(Credentials credentials,
076      Path[] ps, Configuration conf) throws IOException {
077    if (!UserGroupInformation.isSecurityEnabled()) {
078      return;
079    }
080    obtainTokensForNamenodesInternal(credentials, ps, conf);
081  }
082
083  /**
084   * Remove jobtoken referrals which don't make sense in the context
085   * of the task execution.
086   *
087   * @param conf
088   */
089  public static void cleanUpTokenReferral(Configuration conf) {
090    conf.unset(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY);
091  }
092
093  static void obtainTokensForNamenodesInternal(Credentials credentials,
094      Path[] ps, Configuration conf) throws IOException {
095    Set<FileSystem> fsSet = new HashSet<FileSystem>();
096    for(Path p: ps) {
097      fsSet.add(p.getFileSystem(conf));
098    }
099    for (FileSystem fs : fsSet) {
100      obtainTokensForNamenodesInternal(fs, credentials, conf);
101    }
102  }
103
104  static boolean isTokenRenewalExcluded(FileSystem fs, Configuration conf) {
105    String [] nns =
106        conf.getStrings(MRJobConfig.JOB_NAMENODES_TOKEN_RENEWAL_EXCLUDE);
107    if (nns != null) {
108      String host = fs.getUri().getHost();
109      for(int i=0; i< nns.length; i++) {
110        if (nns[i].equals(host)) {
111          return true;
112        }
113      }
114    }
115    return false;
116  }
117
118  /**
119   * get delegation token for a specific FS
120   * @param fs
121   * @param credentials
122   * @param conf
123   * @throws IOException
124   */
125  static void obtainTokensForNamenodesInternal(FileSystem fs, 
126      Credentials credentials, Configuration conf) throws IOException {
127    // RM skips renewing token with empty renewer
128    String delegTokenRenewer = "";
129    if (!isTokenRenewalExcluded(fs, conf)) {
130      delegTokenRenewer = Master.getMasterPrincipal(conf);
131      if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) {
132        throw new IOException(
133            "Can't get Master Kerberos principal for use as renewer");
134      }
135    }
136
137    mergeBinaryTokens(credentials, conf);
138
139    final Token<?> tokens[] = fs.addDelegationTokens(delegTokenRenewer,
140                                                     credentials);
141    if (tokens != null) {
142      for (Token<?> token : tokens) {
143        LOG.info("Got dt for " + fs.getUri() + "; "+token);
144      }
145    }
146  }
147
148  private static void mergeBinaryTokens(Credentials creds, Configuration conf) {
149    String binaryTokenFilename =
150        conf.get(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY);
151    if (binaryTokenFilename != null) {
152      Credentials binary;
153      try {
154        binary = Credentials.readTokenStorageFile(
155            FileSystem.getLocal(conf).makeQualified(
156                new Path(binaryTokenFilename)),
157            conf);
158      } catch (IOException e) {
159        throw new RuntimeException(e);
160      }
161      // supplement existing tokens with the tokens in the binary file
162      creds.mergeAll(binary);
163    }
164  }
165  
166  /**
167   * file name used on HDFS for generated job token
168   */
169  @InterfaceAudience.Private
170  public static final String JOB_TOKEN_HDFS_FILE = "jobToken";
171
172  /**
173   * conf setting for job tokens cache file name
174   */
175  @InterfaceAudience.Private
176  public static final String JOB_TOKENS_FILENAME = "mapreduce.job.jobTokenFile";
177  private static final Text JOB_TOKEN = new Text("JobToken");
178  private static final Text SHUFFLE_TOKEN = new Text("MapReduceShuffleToken");
179  private static final Text ENC_SPILL_KEY = new Text("MapReduceEncryptedSpillKey");
180  
181  /**
182   * load job token from a file
183   * @deprecated Use {@link Credentials#readTokenStorageFile} instead,
184   * this method is included for compatibility against Hadoop-1.
185   * @param conf
186   * @throws IOException
187   */
188  @InterfaceAudience.Private
189  @Deprecated
190  public static Credentials loadTokens(String jobTokenFile, JobConf conf)
191  throws IOException {
192    Path localJobTokenFile = new Path ("file:///" + jobTokenFile);
193
194    Credentials ts = Credentials.readTokenStorageFile(localJobTokenFile, conf);
195
196    if(LOG.isDebugEnabled()) {
197      LOG.debug("Task: Loaded jobTokenFile from: "+
198          localJobTokenFile.toUri().getPath() 
199          +"; num of sec keys  = " + ts.numberOfSecretKeys() +
200          " Number of tokens " +  ts.numberOfTokens());
201    }
202    return ts;
203  }
204  
205  /**
206   * load job token from a file
207   * @deprecated Use {@link Credentials#readTokenStorageFile} instead,
208   * this method is included for compatibility against Hadoop-1.
209   * @param conf
210   * @throws IOException
211   */
212  @InterfaceAudience.Private
213  @Deprecated
214  public static Credentials loadTokens(String jobTokenFile, Configuration conf)
215      throws IOException {
216    return loadTokens(jobTokenFile, new JobConf(conf));
217  }
218  
219  /**
220   * store job token
221   * @param t
222   */
223  @InterfaceAudience.Private
224  public static void setJobToken(Token<? extends TokenIdentifier> t, 
225      Credentials credentials) {
226    credentials.addToken(JOB_TOKEN, t);
227  }
228  /**
229   * 
230   * @return job token
231   */
232  @SuppressWarnings("unchecked")
233  @InterfaceAudience.Private
234  public static Token<JobTokenIdentifier> getJobToken(Credentials credentials) {
235    return (Token<JobTokenIdentifier>) credentials.getToken(JOB_TOKEN);
236  }
237
238  @InterfaceAudience.Private
239  public static void setShuffleSecretKey(byte[] key, Credentials credentials) {
240    credentials.addSecretKey(SHUFFLE_TOKEN, key);
241  }
242
243  @InterfaceAudience.Private
244  public static byte[] getShuffleSecretKey(Credentials credentials) {
245    return getSecretKey(credentials, SHUFFLE_TOKEN);
246  }
247
248  @InterfaceAudience.Private
249  public static void setEncryptedSpillKey(byte[] key, Credentials credentials) {
250    credentials.addSecretKey(ENC_SPILL_KEY, key);
251  }
252
253  @InterfaceAudience.Private
254  public static byte[] getEncryptedSpillKey(Credentials credentials) {
255    return getSecretKey(credentials, ENC_SPILL_KEY);
256  }
257  /**
258   * @deprecated Use {@link Credentials#getToken(org.apache.hadoop.io.Text)}
259   * instead, this method is included for compatibility against Hadoop-1
260   * @param namenode
261   * @return delegation token
262   */
263  @InterfaceAudience.Private
264  @Deprecated
265  public static
266      Token<?> getDelegationToken(
267          Credentials credentials, String namenode) {
268    return (Token<?>) credentials.getToken(new Text(
269      namenode));
270  }
271}