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 019 package org.apache.hadoop.mapreduce.security; 020 021 import java.io.IOException; 022 import java.util.HashSet; 023 import java.util.List; 024 import java.util.Set; 025 026 import org.apache.commons.logging.Log; 027 import org.apache.commons.logging.LogFactory; 028 import org.apache.hadoop.classification.InterfaceAudience; 029 import org.apache.hadoop.classification.InterfaceStability; 030 import org.apache.hadoop.conf.Configuration; 031 import org.apache.hadoop.fs.FileSystem; 032 import org.apache.hadoop.fs.Path; 033 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; 034 import org.apache.hadoop.io.Text; 035 import org.apache.hadoop.mapred.JobConf; 036 import org.apache.hadoop.mapred.Master; 037 import org.apache.hadoop.mapreduce.MRJobConfig; 038 import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier; 039 import org.apache.hadoop.security.Credentials; 040 import org.apache.hadoop.security.UserGroupInformation; 041 import org.apache.hadoop.security.token.Token; 042 import org.apache.hadoop.security.token.TokenIdentifier; 043 044 045 /** 046 * This class provides user facing APIs for transferring secrets from 047 * the job client to the tasks. 048 * The secrets can be stored just before submission of jobs and read during 049 * the task execution. 050 */ 051 @InterfaceAudience.Public 052 @InterfaceStability.Evolving 053 public class TokenCache { 054 055 private static final Log LOG = LogFactory.getLog(TokenCache.class); 056 057 058 /** 059 * auxiliary method to get user's secret keys.. 060 * @param alias 061 * @return secret key from the storage 062 */ 063 public static byte[] getSecretKey(Credentials credentials, Text alias) { 064 if(credentials == null) 065 return null; 066 return credentials.getSecretKey(alias); 067 } 068 069 /** 070 * Convenience method to obtain delegation tokens from namenodes 071 * corresponding to the paths passed. 072 * @param credentials 073 * @param ps array of paths 074 * @param conf configuration 075 * @throws IOException 076 */ 077 public static void obtainTokensForNamenodes(Credentials credentials, 078 Path[] ps, Configuration conf) throws IOException { 079 if (!UserGroupInformation.isSecurityEnabled()) { 080 return; 081 } 082 obtainTokensForNamenodesInternal(credentials, ps, conf); 083 } 084 085 /** 086 * Remove jobtoken referrals which don't make sense in the context 087 * of the task execution. 088 * 089 * @param conf 090 */ 091 public static void cleanUpTokenReferral(Configuration conf) { 092 conf.unset(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY); 093 } 094 095 static void obtainTokensForNamenodesInternal(Credentials credentials, 096 Path[] ps, Configuration conf) throws IOException { 097 Set<FileSystem> fsSet = new HashSet<FileSystem>(); 098 for(Path p: ps) { 099 fsSet.add(p.getFileSystem(conf)); 100 } 101 for (FileSystem fs : fsSet) { 102 obtainTokensForNamenodesInternal(fs, credentials, conf); 103 } 104 } 105 106 /** 107 * get delegation token for a specific FS 108 * @param fs 109 * @param credentials 110 * @param p 111 * @param conf 112 * @throws IOException 113 */ 114 @SuppressWarnings("deprecation") 115 static void obtainTokensForNamenodesInternal(FileSystem fs, 116 Credentials credentials, Configuration conf) throws IOException { 117 String delegTokenRenewer = Master.getMasterPrincipal(conf); 118 if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { 119 throw new IOException( 120 "Can't get Master Kerberos principal for use as renewer"); 121 } 122 mergeBinaryTokens(credentials, conf); 123 124 String fsName = fs.getCanonicalServiceName(); 125 if (TokenCache.getDelegationToken(credentials, fsName) == null) { 126 List<Token<?>> tokens = 127 fs.getDelegationTokens(delegTokenRenewer, credentials); 128 if (tokens != null) { 129 for (Token<?> token : tokens) { 130 credentials.addToken(token.getService(), token); 131 LOG.info("Got dt for " + fs.getUri() + ";uri="+ fsName + 132 ";t.service="+token.getService()); 133 } 134 } 135 //Call getDelegationToken as well for now - for FS implementations 136 // which may not have implmented getDelegationTokens (hftp) 137 if (tokens == null || tokens.size() == 0) { 138 Token<?> token = fs.getDelegationToken(delegTokenRenewer); 139 if (token != null) { 140 credentials.addToken(token.getService(), token); 141 LOG.info("Got dt for " + fs.getUri() + ";uri=" + fsName 142 + ";t.service=" + token.getService()); 143 } 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 new Path("file:///" + binaryTokenFilename), conf); 156 } catch (IOException e) { 157 throw new RuntimeException(e); 158 } 159 // supplement existing tokens with the tokens in the binary file 160 creds.mergeAll(binary); 161 } 162 } 163 164 /** 165 * file name used on HDFS for generated job token 166 */ 167 @InterfaceAudience.Private 168 public static final String JOB_TOKEN_HDFS_FILE = "jobToken"; 169 170 /** 171 * conf setting for job tokens cache file name 172 */ 173 @InterfaceAudience.Private 174 public static final String JOB_TOKENS_FILENAME = "mapreduce.job.jobTokenFile"; 175 private static final Text JOB_TOKEN = new Text("ShuffleAndJobToken"); 176 177 /** 178 * 179 * @param namenode 180 * @return delegation token 181 */ 182 @SuppressWarnings("unchecked") 183 @InterfaceAudience.Private 184 public static Token<DelegationTokenIdentifier> getDelegationToken( 185 Credentials credentials, String namenode) { 186 //No fs specific tokens issues by this fs. It may however issue tokens 187 // for other filesystems - which would be keyed by that filesystems name. 188 if (namenode == null) 189 return null; 190 return (Token<DelegationTokenIdentifier>) credentials.getToken(new Text( 191 namenode)); 192 } 193 194 /** 195 * load job token from a file 196 * @param conf 197 * @throws IOException 198 */ 199 @InterfaceAudience.Private 200 public static Credentials loadTokens(String jobTokenFile, JobConf conf) 201 throws IOException { 202 Path localJobTokenFile = new Path ("file:///" + jobTokenFile); 203 204 Credentials ts = Credentials.readTokenStorageFile(localJobTokenFile, conf); 205 206 if(LOG.isDebugEnabled()) { 207 LOG.debug("Task: Loaded jobTokenFile from: "+ 208 localJobTokenFile.toUri().getPath() 209 +"; num of sec keys = " + ts.numberOfSecretKeys() + 210 " Number of tokens " + ts.numberOfTokens()); 211 } 212 return ts; 213 } 214 /** 215 * store job token 216 * @param t 217 */ 218 @InterfaceAudience.Private 219 public static void setJobToken(Token<? extends TokenIdentifier> t, 220 Credentials credentials) { 221 credentials.addToken(JOB_TOKEN, t); 222 } 223 /** 224 * 225 * @return job token 226 */ 227 @SuppressWarnings("unchecked") 228 @InterfaceAudience.Private 229 public static Token<JobTokenIdentifier> getJobToken(Credentials credentials) { 230 return (Token<JobTokenIdentifier>) credentials.getToken(JOB_TOKEN); 231 } 232 }