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.security;
020
021import java.io.BufferedInputStream;
022import java.io.DataInput;
023import java.io.DataInputStream;
024import java.io.DataOutput;
025import java.io.DataOutputStream;
026import java.io.File;
027import java.io.FileInputStream;
028import java.io.IOException;
029import java.nio.charset.StandardCharsets;
030import java.util.Arrays;
031import java.util.Collection;
032import java.util.HashMap;
033import java.util.List;
034import java.util.Map;
035import java.util.Map.Entry;
036
037import org.apache.commons.logging.Log;
038import org.apache.commons.logging.LogFactory;
039import org.apache.hadoop.classification.InterfaceAudience;
040import org.apache.hadoop.classification.InterfaceStability;
041import org.apache.hadoop.conf.Configuration;
042import org.apache.hadoop.fs.FSDataInputStream;
043import org.apache.hadoop.fs.FSDataOutputStream;
044import org.apache.hadoop.fs.Path;
045import org.apache.hadoop.io.IOUtils;
046import org.apache.hadoop.io.Text;
047import org.apache.hadoop.io.Writable;
048import org.apache.hadoop.io.WritableUtils;
049import org.apache.hadoop.security.token.Token;
050import org.apache.hadoop.security.token.TokenIdentifier;
051
052/**
053 * A class that provides the facilities of reading and writing
054 * secret keys and Tokens.
055 */
056@InterfaceAudience.Public
057@InterfaceStability.Evolving
058public class Credentials implements Writable {
059  private static final Log LOG = LogFactory.getLog(Credentials.class);
060
061  private  Map<Text, byte[]> secretKeysMap = new HashMap<Text, byte[]>();
062  private  Map<Text, Token<? extends TokenIdentifier>> tokenMap = 
063    new HashMap<Text, Token<? extends TokenIdentifier>>(); 
064
065  /**
066   * Create an empty credentials instance
067   */
068  public Credentials() {
069  }
070  
071  /**
072   * Create a copy of the given credentials
073   * @param credentials to copy
074   */
075  public Credentials(Credentials credentials) {
076    this.addAll(credentials);
077  }
078  
079  /**
080   * Returns the Token object for the alias
081   * @param alias the alias for the Token
082   * @return token for this alias
083   */
084  public Token<? extends TokenIdentifier> getToken(Text alias) {
085    return tokenMap.get(alias);
086  }
087  
088  /**
089   * Add a token in the storage (in memory)
090   * @param alias the alias for the key
091   * @param t the token object
092   */
093  public void addToken(Text alias, Token<? extends TokenIdentifier> t) {
094    if (t != null) {
095      tokenMap.put(alias, t);
096    } else {
097      LOG.warn("Null token ignored for " + alias);
098    }
099  }
100  
101  /**
102   * Return all the tokens in the in-memory map
103   */
104  public Collection<Token<? extends TokenIdentifier>> getAllTokens() {
105    return tokenMap.values();
106  }
107  
108  /**
109   * @return number of Tokens in the in-memory map
110   */
111  public int numberOfTokens() {
112    return tokenMap.size();
113  }
114
115  /**
116   * Returns the key bytes for the alias
117   * @param alias the alias for the key
118   * @return key for this alias
119   */
120  public byte[] getSecretKey(Text alias) {
121    return secretKeysMap.get(alias);
122  }
123  
124  /**
125   * @return number of keys in the in-memory map
126   */
127  public int numberOfSecretKeys() {
128    return secretKeysMap.size();
129  }
130  
131  /**
132   * Set the key for an alias
133   * @param alias the alias for the key
134   * @param key the key bytes
135   */
136  public void addSecretKey(Text alias, byte[] key) {
137    secretKeysMap.put(alias, key);
138  }
139
140  /**
141   * Remove the key for a given alias.
142   * @param alias the alias for the key
143   */
144  public void removeSecretKey(Text alias) {
145    secretKeysMap.remove(alias);
146  }
147
148  /**
149   * Return all the secret key entries in the in-memory map
150   */
151  public List<Text> getAllSecretKeys() {
152    List<Text> list = new java.util.ArrayList<Text>();
153    list.addAll(secretKeysMap.keySet());
154
155    return list;
156  }
157
158  /**
159   * Convenience method for reading a token storage file, and loading the Tokens
160   * therein in the passed UGI
161   * @param filename
162   * @param conf
163   * @throws IOException
164   */
165  public static Credentials readTokenStorageFile(Path filename, Configuration conf)
166  throws IOException {
167    FSDataInputStream in = null;
168    Credentials credentials = new Credentials();
169    try {
170      in = filename.getFileSystem(conf).open(filename);
171      credentials.readTokenStorageStream(in);
172      in.close();
173      return credentials;
174    } catch(IOException ioe) {
175      throw new IOException("Exception reading " + filename, ioe);
176    } finally {
177      IOUtils.cleanup(LOG, in);
178    }
179  }
180
181  /**
182   * Convenience method for reading a token storage file, and loading the Tokens
183   * therein in the passed UGI
184   * @param filename
185   * @param conf
186   * @throws IOException
187   */
188  public static Credentials readTokenStorageFile(File filename, Configuration conf)
189      throws IOException {
190    DataInputStream in = null;
191    Credentials credentials = new Credentials();
192    try {
193      in = new DataInputStream(new BufferedInputStream(
194          new FileInputStream(filename)));
195      credentials.readTokenStorageStream(in);
196      return credentials;
197    } catch(IOException ioe) {
198      throw new IOException("Exception reading " + filename, ioe);
199    } finally {
200      IOUtils.cleanup(LOG, in);
201    }
202  }
203  
204  /**
205   * Convenience method for reading a token storage file directly from a 
206   * datainputstream
207   */
208  public void readTokenStorageStream(DataInputStream in) throws IOException {
209    byte[] magic = new byte[TOKEN_STORAGE_MAGIC.length];
210    in.readFully(magic);
211    if (!Arrays.equals(magic, TOKEN_STORAGE_MAGIC)) {
212      throw new IOException("Bad header found in token storage.");
213    }
214    byte version = in.readByte();
215    if (version != TOKEN_STORAGE_VERSION) {
216      throw new IOException("Unknown version " + version + 
217                            " in token storage.");
218    }
219    readFields(in);
220  }
221  
222  private static final byte[] TOKEN_STORAGE_MAGIC =
223      "HDTS".getBytes(StandardCharsets.UTF_8);
224  private static final byte TOKEN_STORAGE_VERSION = 0;
225  
226  public void writeTokenStorageToStream(DataOutputStream os)
227    throws IOException {
228    os.write(TOKEN_STORAGE_MAGIC);
229    os.write(TOKEN_STORAGE_VERSION);
230    write(os);
231  }
232
233  public void writeTokenStorageFile(Path filename, 
234                                    Configuration conf) throws IOException {
235    FSDataOutputStream os = filename.getFileSystem(conf).create(filename);
236    writeTokenStorageToStream(os);
237    os.close();
238  }
239
240  /**
241   * Stores all the keys to DataOutput
242   * @param out
243   * @throws IOException
244   */
245  @Override
246  public void write(DataOutput out) throws IOException {
247    // write out tokens first
248    WritableUtils.writeVInt(out, tokenMap.size());
249    for(Map.Entry<Text, 
250        Token<? extends TokenIdentifier>> e: tokenMap.entrySet()) {
251      e.getKey().write(out);
252      e.getValue().write(out);
253    }
254    
255    // now write out secret keys
256    WritableUtils.writeVInt(out, secretKeysMap.size());
257    for(Map.Entry<Text, byte[]> e : secretKeysMap.entrySet()) {
258      e.getKey().write(out);
259      WritableUtils.writeVInt(out, e.getValue().length);
260      out.write(e.getValue());
261    }
262  }
263  
264  /**
265   * Loads all the keys
266   * @param in
267   * @throws IOException
268   */
269  @Override
270  public void readFields(DataInput in) throws IOException {
271    secretKeysMap.clear();
272    tokenMap.clear();
273    
274    int size = WritableUtils.readVInt(in);
275    for(int i=0; i<size; i++) {
276      Text alias = new Text();
277      alias.readFields(in);
278      Token<? extends TokenIdentifier> t = new Token<TokenIdentifier>();
279      t.readFields(in);
280      tokenMap.put(alias, t);
281    }
282    
283    size = WritableUtils.readVInt(in);
284    for(int i=0; i<size; i++) {
285      Text alias = new Text();
286      alias.readFields(in);
287      int len = WritableUtils.readVInt(in);
288      byte[] value = new byte[len];
289      in.readFully(value);
290      secretKeysMap.put(alias, value);
291    }
292  }
293 
294  /**
295   * Copy all of the credentials from one credential object into another.
296   * Existing secrets and tokens are overwritten.
297   * @param other the credentials to copy
298   */
299  public void addAll(Credentials other) {
300    addAll(other, true);
301  }
302
303  /**
304   * Copy all of the credentials from one credential object into another.
305   * Existing secrets and tokens are not overwritten.
306   * @param other the credentials to copy
307   */
308  public void mergeAll(Credentials other) {
309    addAll(other, false);
310  }
311
312  private void addAll(Credentials other, boolean overwrite) {
313    for(Map.Entry<Text, byte[]> secret: other.secretKeysMap.entrySet()) {
314      Text key = secret.getKey();
315      if (!secretKeysMap.containsKey(key) || overwrite) {
316        secretKeysMap.put(key, secret.getValue());
317      }
318    }
319    for(Map.Entry<Text, Token<?>> token: other.tokenMap.entrySet()){
320      Text key = token.getKey();
321      if (!tokenMap.containsKey(key) || overwrite) {
322        tokenMap.put(key, token.getValue());
323      }
324    }
325  }
326}