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.HashMap; 023 import java.util.List; 024 import java.util.Map; 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.security.token.JobTokenIdentifier; 038 import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig; 039 import org.apache.hadoop.security.Credentials; 040 import org.apache.hadoop.security.SecurityUtil; 041 import org.apache.hadoop.security.UserGroupInformation; 042 import org.apache.hadoop.security.token.Token; 043 import org.apache.hadoop.security.token.TokenIdentifier; 044 045 046 /** 047 * This class provides user facing APIs for transferring secrets from 048 * the job client to the tasks. 049 * The secrets can be stored just before submission of jobs and read during 050 * the task execution. 051 */ 052 @InterfaceAudience.Public 053 @InterfaceStability.Evolving 054 public class TokenCache { 055 056 private static final Log LOG = LogFactory.getLog(TokenCache.class); 057 058 059 /** 060 * auxiliary method to get user's secret keys.. 061 * @param alias 062 * @return secret key from the storage 063 */ 064 public static byte[] getSecretKey(Credentials credentials, Text alias) { 065 if(credentials == null) 066 return null; 067 return credentials.getSecretKey(alias); 068 } 069 070 /** 071 * Convenience method to obtain delegation tokens from namenodes 072 * corresponding to the paths passed. 073 * @param credentials 074 * @param ps array of paths 075 * @param conf configuration 076 * @throws IOException 077 */ 078 public static void obtainTokensForNamenodes(Credentials credentials, 079 Path[] ps, Configuration conf) throws IOException { 080 if (!UserGroupInformation.isSecurityEnabled()) { 081 return; 082 } 083 obtainTokensForNamenodesInternal(credentials, ps, conf); 084 } 085 086 static void obtainTokensForNamenodesInternal(Credentials credentials, 087 Path[] ps, Configuration conf) throws IOException { 088 for(Path p: ps) { 089 FileSystem fs = FileSystem.get(p.toUri(), conf); 090 obtainTokensForNamenodesInternal(fs, credentials, conf); 091 } 092 } 093 094 /** 095 * get delegation token for a specific FS 096 * @param fs 097 * @param credentials 098 * @param p 099 * @param conf 100 * @throws IOException 101 */ 102 static void obtainTokensForNamenodesInternal(FileSystem fs, 103 Credentials credentials, Configuration conf) throws IOException { 104 String delegTokenRenewer = Master.getMasterPrincipal(conf); 105 if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { 106 throw new IOException( 107 "Can't get JobTracker Kerberos principal for use as renewer"); 108 } 109 boolean readFile = true; 110 111 String fsName = fs.getCanonicalServiceName(); 112 if (TokenCache.getDelegationToken(credentials, fsName) == null) { 113 //TODO: Need to come up with a better place to put 114 //this block of code to do with reading the file 115 if (readFile) { 116 readFile = false; 117 String binaryTokenFilename = 118 conf.get("mapreduce.job.credentials.binary"); 119 if (binaryTokenFilename != null) { 120 Credentials binary; 121 try { 122 binary = Credentials.readTokenStorageFile( 123 new Path("file:///" + binaryTokenFilename), conf); 124 } catch (IOException e) { 125 throw new RuntimeException(e); 126 } 127 credentials.addAll(binary); 128 } 129 if (TokenCache.getDelegationToken(credentials, fsName) != null) { 130 LOG.debug("DT for " + fsName + " is already present"); 131 return; 132 } 133 } 134 List<Token<?>> tokens = fs.getDelegationTokens(delegTokenRenewer); 135 if (tokens != null) { 136 for (Token<?> token : tokens) { 137 credentials.addToken(token.getService(), token); 138 LOG.info("Got dt for " + fs.getUri() + ";uri="+ fsName + 139 ";t.service="+token.getService()); 140 } 141 } 142 //Call getDelegationToken as well for now - for FS implementations 143 // which may not have implmented getDelegationTokens (hftp) 144 Token<?> token = fs.getDelegationToken(delegTokenRenewer); 145 if (token != null) { 146 Text fsNameText = new Text(fsName); 147 token.setService(fsNameText); 148 credentials.addToken(fsNameText, token); 149 LOG.info("Got dt for " + fs.getUri() + ";uri="+ fsName + 150 ";t.service="+token.getService()); 151 } 152 } 153 } 154 155 /** 156 * file name used on HDFS for generated job token 157 */ 158 @InterfaceAudience.Private 159 public static final String JOB_TOKEN_HDFS_FILE = "jobToken"; 160 161 /** 162 * conf setting for job tokens cache file name 163 */ 164 @InterfaceAudience.Private 165 public static final String JOB_TOKENS_FILENAME = "mapreduce.job.jobTokenFile"; 166 private static final Text JOB_TOKEN = new Text("ShuffleAndJobToken"); 167 168 /** 169 * 170 * @param namenode 171 * @return delegation token 172 */ 173 @SuppressWarnings("unchecked") 174 @InterfaceAudience.Private 175 public static Token<DelegationTokenIdentifier> getDelegationToken( 176 Credentials credentials, String namenode) { 177 return (Token<DelegationTokenIdentifier>) credentials.getToken(new Text( 178 namenode)); 179 } 180 181 /** 182 * load job token from a file 183 * @param conf 184 * @throws IOException 185 */ 186 @InterfaceAudience.Private 187 public static Credentials loadTokens(String jobTokenFile, JobConf conf) 188 throws IOException { 189 Path localJobTokenFile = new Path ("file:///" + jobTokenFile); 190 191 Credentials ts = Credentials.readTokenStorageFile(localJobTokenFile, conf); 192 193 if(LOG.isDebugEnabled()) { 194 LOG.debug("Task: Loaded jobTokenFile from: "+ 195 localJobTokenFile.toUri().getPath() 196 +"; num of sec keys = " + ts.numberOfSecretKeys() + 197 " Number of tokens " + ts.numberOfTokens()); 198 } 199 return ts; 200 } 201 /** 202 * store job token 203 * @param t 204 */ 205 @InterfaceAudience.Private 206 public static void setJobToken(Token<? extends TokenIdentifier> t, 207 Credentials credentials) { 208 credentials.addToken(JOB_TOKEN, t); 209 } 210 /** 211 * 212 * @return job token 213 */ 214 @SuppressWarnings("unchecked") 215 @InterfaceAudience.Private 216 public static Token<JobTokenIdentifier> getJobToken(Credentials credentials) { 217 return (Token<JobTokenIdentifier>) credentials.getToken(JOB_TOKEN); 218 } 219 }