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