001/*
002 * Licensed to the Apache Software Foundation (ASF) under one or more
003 * contributor license agreements. See the NOTICE file distributed with this
004 * work for additional information regarding copyright ownership. The ASF
005 * licenses this file to you under the Apache License, Version 2.0 (the
006 * "License"); you may not use this file except in compliance with the License.
007 * You may obtain a copy of the License at
008 *
009 * http://www.apache.org/licenses/LICENSE-2.0
010 *
011 * Unless required by applicable law or agreed to in writing, software
012 * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
013 * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
014 * License for the specific language governing permissions and limitations under
015 * the License.
016 */
017package org.apache.hadoop.hbase.io.crypto;
018
019import static java.lang.String.format;
020
021import java.io.IOException;
022import java.io.InputStream;
023import java.io.OutputStream;
024import java.security.Key;
025import java.security.MessageDigest;
026import java.security.NoSuchAlgorithmException;
027import java.security.spec.InvalidKeySpecException;
028import java.util.Arrays;
029import java.util.Map;
030import java.util.concurrent.ConcurrentHashMap;
031
032import javax.crypto.SecretKeyFactory;
033import javax.crypto.spec.PBEKeySpec;
034import javax.crypto.spec.SecretKeySpec;
035
036import org.apache.commons.io.IOUtils;
037import org.apache.hadoop.conf.Configuration;
038import org.apache.hadoop.hbase.HBaseConfiguration;
039import org.apache.hadoop.hbase.HConstants;
040import org.apache.hadoop.hbase.io.crypto.aes.AES;
041import org.apache.hadoop.hbase.util.Bytes;
042import org.apache.hadoop.hbase.util.Pair;
043import org.apache.hadoop.util.ReflectionUtils;
044import org.apache.yetus.audience.InterfaceAudience;
045import org.slf4j.Logger;
046import org.slf4j.LoggerFactory;
047
048/**
049 * A facade for encryption algorithms and related support.
050 */
051@InterfaceAudience.Public
052public final class Encryption {
053
054  private static final Logger LOG = LoggerFactory.getLogger(Encryption.class);
055
056
057  /**
058   * Configuration key for globally enable / disable column family encryption
059   */
060  public static final String CRYPTO_ENABLED_CONF_KEY = "hbase.crypto.enabled";
061
062  /**
063   * Default value for globally enable / disable column family encryption
064   * (set to "true" for backward compatibility)
065   */
066  public static final boolean CRYPTO_ENABLED_CONF_DEFAULT = true;
067
068  /**
069   * Configuration key for the hash algorithm used for generating key hash in encrypted HFiles.
070   * This is a MessageDigest algorithm identifier string, like "MD5", "SHA-256" or "SHA-384".
071   * (default: "MD5" for backward compatibility reasons)
072   */
073  public static final String CRYPTO_KEY_HASH_ALGORITHM_CONF_KEY = "hbase.crypto.key.hash.algorithm";
074
075  /**
076   * Default hash algorithm used for generating key hash in encrypted HFiles.
077   * (we use "MD5" for backward compatibility reasons)
078   */
079  public static final String CRYPTO_KEY_HASH_ALGORITHM_CONF_DEFAULT = "MD5";
080
081  /**
082   * Configuration key for specifying the behaviour if the configured hash algorithm
083   * differs from the one used for generating key hash in encrypted HFiles currently being read.
084   *
085   * - "false" (default): we won't fail but use the hash algorithm stored in the HFile
086   * - "true": we throw an exception (this can be useful if regulations are enforcing the usage
087   *           of certain algorithms, e.g. on FIPS compliant clusters)
088   */
089  public static final String CRYPTO_KEY_FAIL_ON_ALGORITHM_MISMATCH_CONF_KEY =
090    "hbase.crypto.key.hash.algorithm.failOnMismatch";
091
092  /**
093   * Default behaviour is not to fail if the hash algorithm configured differs from the one
094   * used in the HFile. (this is the more fail-safe approach, allowing us to read
095   * encrypted HFiles written using a different encryption key hash algorithm)
096   */
097  public static final boolean CRYPTO_KEY_FAIL_ON_ALGORITHM_MISMATCH_CONF_DEFAULT = false;
098
099
100  /**
101   * Crypto context
102   */
103  @InterfaceAudience.Public
104  public static class Context extends org.apache.hadoop.hbase.io.crypto.Context {
105
106    /** The null crypto context */
107    public static final Context NONE = new Context();
108
109    private Context() {
110      super();
111    }
112
113    private Context(Configuration conf) {
114      super(conf);
115    }
116
117    @Override
118    public Context setCipher(Cipher cipher) {
119      super.setCipher(cipher);
120      return this;
121    }
122
123    @Override
124    public Context setKey(Key key) {
125      super.setKey(key);
126      return this;
127    }
128
129    public Context setKey(byte[] key) {
130      super.setKey(new SecretKeySpec(key, getCipher().getName()));
131      return this;
132    }
133  }
134
135  public static Context newContext() {
136    return new Context();
137  }
138
139  public static Context newContext(Configuration conf) {
140    return new Context(conf);
141  }
142
143  // Prevent instantiation
144  private Encryption() {
145    super();
146  }
147
148
149  /**
150   * Returns true if the column family encryption feature is enabled globally.
151   */
152  public static boolean isEncryptionEnabled(Configuration conf) {
153    return conf.getBoolean(CRYPTO_ENABLED_CONF_KEY, CRYPTO_ENABLED_CONF_DEFAULT);
154  }
155
156  /**
157   * Get an cipher given a name
158   * @param name the cipher name
159   * @return the cipher, or null if a suitable one could not be found
160   */
161  public static Cipher getCipher(Configuration conf, String name) {
162    return getCipherProvider(conf).getCipher(name);
163  }
164
165  /**
166   * Get names of supported encryption algorithms
167   *
168   * @return Array of strings, each represents a supported encryption algorithm
169   */
170  public static String[] getSupportedCiphers() {
171    return getSupportedCiphers(HBaseConfiguration.create());
172  }
173
174  /**
175   * Get names of supported encryption algorithms
176   *
177   * @return Array of strings, each represents a supported encryption algorithm
178   */
179  public static String[] getSupportedCiphers(Configuration conf) {
180    return getCipherProvider(conf).getSupportedCiphers();
181  }
182
183  /**
184   * Returns the Hash Algorithm defined in the crypto configuration.
185   */
186  public static String getConfiguredHashAlgorithm(Configuration conf) {
187    return conf.getTrimmed(CRYPTO_KEY_HASH_ALGORITHM_CONF_KEY,
188                    CRYPTO_KEY_HASH_ALGORITHM_CONF_DEFAULT);
189  }
190
191  /**
192   * Returns the Hash Algorithm mismatch behaviour defined in the crypto configuration.
193   */
194  public static boolean failOnHashAlgorithmMismatch(Configuration conf) {
195    return conf.getBoolean(CRYPTO_KEY_FAIL_ON_ALGORITHM_MISMATCH_CONF_KEY,
196                           CRYPTO_KEY_FAIL_ON_ALGORITHM_MISMATCH_CONF_DEFAULT);
197  }
198
199  /**
200   * Returns the hash of the supplied argument, using the hash algorithm
201   * specified in the given config.
202   */
203  public static byte[] computeCryptoKeyHash(Configuration conf, byte[] arg) {
204    String algorithm = getConfiguredHashAlgorithm(conf);
205    try {
206      return hashWithAlg(algorithm, arg);
207    } catch (RuntimeException e) {
208      String message = format("Error in computeCryptoKeyHash (please check your configuration " +
209                              "parameter %s and the security provider configuration of the JVM)",
210                              CRYPTO_KEY_HASH_ALGORITHM_CONF_KEY);
211      throw new RuntimeException(message, e);
212    }
213  }
214
215  /**
216   * Return the MD5 digest of the concatenation of the supplied arguments.
217   */
218  public static byte[] hash128(String... args) {
219    return hashWithAlg("MD5", Bytes.toByteArrays(args));
220  }
221
222  /**
223   * Return the MD5 digest of the concatenation of the supplied arguments.
224   */
225  public static byte[] hash128(byte[]... args) {
226    return hashWithAlg("MD5", args);
227  }
228
229  /**
230   * Return the SHA-256 digest of the concatenation of the supplied arguments.
231   */
232  public static byte[] hash256(String... args) {
233    return hashWithAlg("SHA-256", Bytes.toByteArrays(args));
234  }
235
236  /**
237   * Return the SHA-256 digest of the concatenation of the supplied arguments.
238   */
239  public static byte[] hash256(byte[]... args) {
240    return hashWithAlg("SHA-256", args);
241  }
242
243  /**
244   * Return a 128 bit key derived from the concatenation of the supplied
245   * arguments using PBKDF2WithHmacSHA1 at 10,000 iterations.
246   *
247   */
248  public static byte[] pbkdf128(String... args) {
249    StringBuilder sb = new StringBuilder();
250    for (String s: args) {
251      sb.append(s);
252    }
253    return generateSecretKey("PBKDF2WithHmacSHA1", AES.KEY_LENGTH, sb.toString().toCharArray());
254  }
255
256  /**
257   * Return a 128 bit key derived from the concatenation of the supplied
258   * arguments using PBKDF2WithHmacSHA1 at 10,000 iterations.
259   *
260   */
261  public static byte[] pbkdf128(byte[]... args) {
262    StringBuilder sb = new StringBuilder();
263    for (byte[] b: args) {
264      sb.append(Arrays.toString(b));
265    }
266    return generateSecretKey("PBKDF2WithHmacSHA1", AES.KEY_LENGTH, sb.toString().toCharArray());
267  }
268
269  /**
270   * Return a key derived from the concatenation of the supplied arguments using
271   * PBKDF2WithHmacSHA384 key derivation algorithm at 10,000 iterations.
272   *
273   * The length of the returned key is determined based on the need of the cypher algorithm.
274   * E.g. for the default "AES"  we will need a 128 bit long key, while if the user is using
275   * a custom cipher, we might generate keys with other length.
276   *
277   * This key generation method is used currently e.g. in the HBase Shell (admin.rb) to generate a
278   * column family data encryption key, if the user provided an ENCRYPTION_KEY parameter.
279   */
280  public static byte[] generateSecretKey(Configuration conf, String cypherAlg, String... args) {
281    StringBuilder sb = new StringBuilder();
282    for (String s: args) {
283      sb.append(s);
284    }
285    int keyLengthBytes = Encryption.getCipher(conf, cypherAlg).getKeyLength();
286    return generateSecretKey("PBKDF2WithHmacSHA384", keyLengthBytes, sb.toString().toCharArray());
287  }
288
289  /**
290   * Return a key derived from the concatenation of the supplied arguments using
291   * PBKDF2WithHmacSHA384 key derivation algorithm at 10,000 iterations.
292   *
293   * The length of the returned key is determined based on the need of the cypher algorithm.
294   * E.g. for the default "AES"  we will need a 128 bit long key, while if the user is using
295   * a custom cipher, we might generate keys with other length.
296   *
297   * This key generation method is used currently e.g. in the HBase Shell (admin.rb) to generate a
298   * column family data encryption key, if the user provided an ENCRYPTION_KEY parameter.
299   */
300  public static byte[] generateSecretKey(Configuration conf, String cypherAlg, byte[]... args) {
301    StringBuilder sb = new StringBuilder();
302    for (byte[] b: args) {
303      sb.append(Arrays.toString(b));
304    }
305    int keyLength = Encryption.getCipher(conf, cypherAlg).getKeyLength();
306    return generateSecretKey("PBKDF2WithHmacSHA384", keyLength, sb.toString().toCharArray());
307  }
308
309  /**
310   * Return a key (byte array) derived from the supplied password argument using the given
311   * algorithm with a random salt at 10,000 iterations.
312   *
313   * @param algorithm the secret key generation algorithm to use
314   * @param keyLengthBytes the length of the key to be derived (in bytes, not in bits)
315   * @param password char array to use as password for the key generation algorithm
316   * @return secret key encoded as a byte array
317   */
318  private static byte[] generateSecretKey(String algorithm, int keyLengthBytes, char[] password) {
319    byte[] salt = new byte[keyLengthBytes];
320    Bytes.random(salt);
321    PBEKeySpec spec = new PBEKeySpec(password, salt, 10000, keyLengthBytes*8);
322    try {
323      return SecretKeyFactory.getInstance(algorithm).generateSecret(spec).getEncoded();
324    } catch (NoSuchAlgorithmException | InvalidKeySpecException e) {
325      throw new RuntimeException(e);
326    }
327  }
328
329  /**
330   * Encrypt a block of plaintext
331   * <p>
332   * The encryptor's state will be finalized. It should be reinitialized or
333   * returned to the pool.
334   * @param out ciphertext
335   * @param src plaintext
336   * @param offset
337   * @param length
338   * @param e
339   * @throws IOException
340    */
341  public static void encrypt(OutputStream out, byte[] src, int offset,
342      int length, Encryptor e) throws IOException {
343    OutputStream cout = e.createEncryptionStream(out);
344    try {
345      cout.write(src, offset, length);
346    } finally {
347      cout.close();
348    }
349  }
350
351  /**
352   * Encrypt a block of plaintext
353   * @param out ciphertext
354   * @param src plaintext
355   * @param offset
356   * @param length
357   * @param context
358   * @param iv
359   * @throws IOException
360    */
361  public static void encrypt(OutputStream out, byte[] src, int offset,
362      int length, Context context, byte[] iv) throws IOException {
363    Encryptor e = context.getCipher().getEncryptor();
364    e.setKey(context.getKey());
365    e.setIv(iv); // can be null
366    e.reset();
367    encrypt(out, src, offset, length, e);
368  }
369
370  /**
371   * Encrypt a stream of plaintext given an encryptor
372   * <p>
373   * The encryptor's state will be finalized. It should be reinitialized or
374   * returned to the pool.
375   * @param out ciphertext
376   * @param in plaintext
377   * @param e
378   * @throws IOException
379   */
380  public static void encrypt(OutputStream out, InputStream in, Encryptor e)
381      throws IOException {
382    OutputStream cout = e.createEncryptionStream(out);
383    try {
384      IOUtils.copy(in, cout);
385    } finally {
386      cout.close();
387    }
388  }
389
390  /**
391   * Encrypt a stream of plaintext given a context and IV
392   * @param out ciphertext
393   * @param in plaintet
394   * @param context
395   * @param iv
396   * @throws IOException
397   */
398  public static void encrypt(OutputStream out, InputStream in, Context context,
399      byte[] iv) throws IOException {
400    Encryptor e = context.getCipher().getEncryptor();
401    e.setKey(context.getKey());
402    e.setIv(iv); // can be null
403    e.reset();
404    encrypt(out, in, e);
405  }
406
407  /**
408   * Decrypt a block of ciphertext read in from a stream with the given
409   * cipher and context
410   * <p>
411   * The decryptor's state will be finalized. It should be reinitialized or
412   * returned to the pool.
413   * @param dest
414   * @param destOffset
415   * @param in
416   * @param destSize
417   * @param d
418   * @throws IOException
419   */
420  public static void decrypt(byte[] dest, int destOffset, InputStream in,
421      int destSize, Decryptor d) throws IOException {
422    InputStream cin = d.createDecryptionStream(in);
423    try {
424      IOUtils.readFully(cin, dest, destOffset, destSize);
425    } finally {
426      cin.close();
427    }
428  }
429
430  /**
431   * Decrypt a block of ciphertext from a stream given a context and IV
432   * @param dest
433   * @param destOffset
434   * @param in
435   * @param destSize
436   * @param context
437   * @param iv
438   * @throws IOException
439   */
440  public static void decrypt(byte[] dest, int destOffset, InputStream in,
441      int destSize, Context context, byte[] iv) throws IOException {
442    Decryptor d = context.getCipher().getDecryptor();
443    d.setKey(context.getKey());
444    d.setIv(iv); // can be null
445    decrypt(dest, destOffset, in, destSize, d);
446  }
447
448  /**
449   * Decrypt a stream of ciphertext given a decryptor
450   * @param out
451   * @param in
452   * @param outLen
453   * @param d
454   * @throws IOException
455   */
456  public static void decrypt(OutputStream out, InputStream in, int outLen,
457      Decryptor d) throws IOException {
458    InputStream cin = d.createDecryptionStream(in);
459    byte buf[] = new byte[8*1024];
460    long remaining = outLen;
461    try {
462      while (remaining > 0) {
463        int toRead = (int)(remaining < buf.length ? remaining : buf.length);
464        int read = cin.read(buf, 0, toRead);
465        if (read < 0) {
466          break;
467        }
468        out.write(buf, 0, read);
469        remaining -= read;
470      }
471    } finally {
472      cin.close();
473    }
474  }
475
476  /**
477   * Decrypt a stream of ciphertext given a context and IV
478   * @param out
479   * @param in
480   * @param outLen
481   * @param context
482   * @param iv
483   * @throws IOException
484   */
485  public static void decrypt(OutputStream out, InputStream in, int outLen,
486      Context context, byte[] iv) throws IOException {
487    Decryptor d = context.getCipher().getDecryptor();
488    d.setKey(context.getKey());
489    d.setIv(iv); // can be null
490    decrypt(out, in, outLen, d);
491  }
492
493  /**
494   * Resolves a key for the given subject
495   * @param subject
496   * @param conf
497   * @return a key for the given subject
498   * @throws IOException if the key is not found
499   */
500  public static Key getSecretKeyForSubject(String subject, Configuration conf)
501      throws IOException {
502    KeyProvider provider = getKeyProvider(conf);
503    if (provider != null) try {
504      Key[] keys = provider.getKeys(new String[] { subject });
505      if (keys != null && keys.length > 0) {
506        return keys[0];
507      }
508    } catch (Exception e) {
509      throw new IOException(e);
510    }
511    throw new IOException("No key found for subject '" + subject + "'");
512  }
513
514  /**
515   * Encrypts a block of plaintext with the symmetric key resolved for the given subject
516   * @param out ciphertext
517   * @param in plaintext
518   * @param conf configuration
519   * @param cipher the encryption algorithm
520   * @param iv the initialization vector, can be null
521   * @throws IOException
522   */
523  public static void encryptWithSubjectKey(OutputStream out, InputStream in,
524      String subject, Configuration conf, Cipher cipher, byte[] iv)
525      throws IOException {
526    Key key = getSecretKeyForSubject(subject, conf);
527    if (key == null) {
528      throw new IOException("No key found for subject '" + subject + "'");
529    }
530    Encryptor e = cipher.getEncryptor();
531    e.setKey(key);
532    e.setIv(iv); // can be null
533    encrypt(out, in, e);
534  }
535
536  /**
537   * Decrypts a block of ciphertext with the symmetric key resolved for the given subject
538   * @param out plaintext
539   * @param in ciphertext
540   * @param outLen the expected plaintext length
541   * @param subject the subject's key alias
542   * @param conf configuration
543   * @param cipher the encryption algorithm
544   * @param iv the initialization vector, can be null
545   * @throws IOException
546   */
547  public static void decryptWithSubjectKey(OutputStream out, InputStream in, int outLen,
548      String subject, Configuration conf, Cipher cipher, byte[] iv) throws IOException {
549    Key key = getSecretKeyForSubject(subject, conf);
550    if (key == null) {
551      throw new IOException("No key found for subject '" + subject + "'");
552    }
553    Decryptor d = cipher.getDecryptor();
554    d.setKey(key);
555    d.setIv(iv); // can be null
556    try {
557      decrypt(out, in, outLen, d);
558    } catch (IOException e) {
559      // If the current cipher algorithm fails to unwrap, try the alternate cipher algorithm, if one
560      // is configured
561      String alternateAlgorithm = conf.get(HConstants.CRYPTO_ALTERNATE_KEY_ALGORITHM_CONF_KEY);
562      if (alternateAlgorithm != null) {
563        if (LOG.isDebugEnabled()) {
564          LOG.debug("Unable to decrypt data with current cipher algorithm '"
565              + conf.get(HConstants.CRYPTO_KEY_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES)
566              + "'. Trying with the alternate cipher algorithm '" + alternateAlgorithm
567              + "' configured.");
568        }
569        Cipher alterCipher = Encryption.getCipher(conf, alternateAlgorithm);
570        if (alterCipher == null) {
571          throw new RuntimeException("Cipher '" + alternateAlgorithm + "' not available");
572        }
573        d = alterCipher.getDecryptor();
574        d.setKey(key);
575        d.setIv(iv); // can be null
576        decrypt(out, in, outLen, d);
577      } else {
578        throw new IOException(e);
579      }
580    }
581  }
582
583  private static ClassLoader getClassLoaderForClass(Class<?> c) {
584    ClassLoader cl = Thread.currentThread().getContextClassLoader();
585    if (cl == null) {
586      cl = c.getClassLoader();
587    }
588    if (cl == null) {
589      cl = ClassLoader.getSystemClassLoader();
590    }
591    if (cl == null) {
592      throw new RuntimeException("A ClassLoader to load the Cipher could not be determined");
593    }
594    return cl;
595  }
596
597  public static CipherProvider getCipherProvider(Configuration conf) {
598    String providerClassName = conf.get(HConstants.CRYPTO_CIPHERPROVIDER_CONF_KEY,
599      DefaultCipherProvider.class.getName());
600    try {
601      CipherProvider provider = (CipherProvider)
602        ReflectionUtils.newInstance(getClassLoaderForClass(CipherProvider.class)
603          .loadClass(providerClassName),
604        conf);
605      return provider;
606    } catch (Exception e) {
607      throw new RuntimeException(e);
608    }
609  }
610
611  static final Map<Pair<String,String>,KeyProvider> keyProviderCache = new ConcurrentHashMap<>();
612
613  public static KeyProvider getKeyProvider(Configuration conf) {
614    String providerClassName = conf.get(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY,
615      KeyStoreKeyProvider.class.getName());
616    String providerParameters = conf.get(HConstants.CRYPTO_KEYPROVIDER_PARAMETERS_KEY, "");
617    try {
618      Pair<String,String> providerCacheKey = new Pair<>(providerClassName,
619        providerParameters);
620      KeyProvider provider = keyProviderCache.get(providerCacheKey);
621      if (provider != null) {
622        return provider;
623      }
624      provider = (KeyProvider) ReflectionUtils.newInstance(
625        getClassLoaderForClass(KeyProvider.class).loadClass(providerClassName),
626        conf);
627      provider.init(providerParameters);
628      if (LOG.isDebugEnabled()) {
629        LOG.debug("Installed " + providerClassName + " into key provider cache");
630      }
631      keyProviderCache.put(providerCacheKey, provider);
632      return provider;
633    } catch (Exception e) {
634      throw new RuntimeException(e);
635    }
636  }
637
638  public static void incrementIv(byte[] iv) {
639    incrementIv(iv, 1);
640  }
641
642  public static void incrementIv(byte[] iv, int v) {
643    int length = iv.length;
644    boolean carry = true;
645    // TODO: Optimize for v > 1, e.g. 16, 32
646    do {
647      for (int i = 0; i < length; i++) {
648        if (carry) {
649          iv[i] = (byte) ((iv[i] + 1) & 0xFF);
650          carry = 0 == iv[i];
651        } else {
652          break;
653        }
654      }
655      v--;
656    } while (v > 0);
657  }
658
659  /**
660   * Return the hash of the concatenation of the supplied arguments, using the 
661   * hash algorithm provided.
662   */
663  public static byte[] hashWithAlg(String algorithm, byte[]... args) {
664    try {
665      MessageDigest md = MessageDigest.getInstance(algorithm);
666      for (byte[] arg: args) {
667        md.update(arg);
668      }
669      return md.digest();
670    } catch (NoSuchAlgorithmException e) {
671      throw new RuntimeException("unable to use hash algorithm: " + algorithm, e);
672    }
673  }
674
675}