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.security;
019
020import java.io.ByteArrayInputStream;
021import java.io.ByteArrayOutputStream;
022import java.io.IOException;
023import java.security.Key;
024import java.security.KeyException;
025import java.util.Properties;
026import javax.crypto.spec.SecretKeySpec;
027import org.apache.commons.crypto.cipher.CryptoCipherFactory;
028import org.apache.hadoop.conf.Configuration;
029import org.apache.hadoop.hbase.HConstants;
030import org.apache.hadoop.hbase.io.crypto.Cipher;
031import org.apache.hadoop.hbase.io.crypto.Encryption;
032import org.apache.hadoop.hbase.io.crypto.aes.CryptoAES;
033import org.apache.hadoop.hbase.util.Bytes;
034import org.apache.yetus.audience.InterfaceAudience;
035import org.apache.yetus.audience.InterfaceStability;
036import org.slf4j.Logger;
037import org.slf4j.LoggerFactory;
038
039import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
040
041import org.apache.hadoop.hbase.shaded.protobuf.generated.EncryptionProtos;
042import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos;
043
044/**
045 * Some static utility methods for encryption uses in hbase-client.
046 */
047@InterfaceAudience.Private
048@InterfaceStability.Evolving
049public final class EncryptionUtil {
050  static private final Logger LOG = LoggerFactory.getLogger(EncryptionUtil.class);
051
052  /**
053   * Private constructor to keep this class from being instantiated.
054   */
055  private EncryptionUtil() {
056  }
057
058  /**
059   * Protect a key by encrypting it with the secret key of the given subject. The configuration must
060   * be set up correctly for key alias resolution.
061   * @param conf      configuration
062   * @param key       the raw key bytes
063   * @param algorithm the algorithm to use with this key material
064   * @return the encrypted key bytes
065   */
066  public static byte[] wrapKey(Configuration conf, byte[] key, String algorithm)
067    throws IOException {
068    return wrapKey(conf,
069      conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, User.getCurrent().getShortName()),
070      new SecretKeySpec(key, algorithm));
071  }
072
073  /**
074   * Protect a key by encrypting it with the secret key of the given subject. The configuration must
075   * be set up correctly for key alias resolution.
076   * @param conf    configuration
077   * @param subject subject key alias
078   * @param key     the key
079   * @return the encrypted key bytes
080   */
081  public static byte[] wrapKey(Configuration conf, String subject, Key key) throws IOException {
082    return wrapKey(conf, subject, key, null);
083  }
084
085  /**
086   * Protect a key by encrypting it with the secret key of the given subject or kek. The
087   * configuration must be set up correctly for key alias resolution. Only one of the
088   * {@code subject} or {@code kek} needs to be specified and the other one can be {@code null}.
089   * @param conf    configuration
090   * @param subject subject key alias
091   * @param key     the key
092   * @param kek     the key encryption key
093   * @return the encrypted key bytes
094   */
095  public static byte[] wrapKey(Configuration conf, String subject, Key key, Key kek)
096    throws IOException {
097    // Wrap the key with the configured encryption algorithm.
098    String algorithm = conf.get(HConstants.CRYPTO_KEY_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES);
099    Cipher cipher = Encryption.getCipher(conf, algorithm);
100    if (cipher == null) {
101      throw new RuntimeException("Cipher '" + algorithm + "' not available");
102    }
103    EncryptionProtos.WrappedKey.Builder builder = EncryptionProtos.WrappedKey.newBuilder();
104    builder.setAlgorithm(key.getAlgorithm());
105    byte[] iv = null;
106    if (cipher.getIvLength() > 0) {
107      iv = new byte[cipher.getIvLength()];
108      Bytes.secureRandom(iv);
109      builder.setIv(UnsafeByteOperations.unsafeWrap(iv));
110    }
111    byte[] keyBytes = key.getEncoded();
112    builder.setLength(keyBytes.length);
113    builder.setHashAlgorithm(Encryption.getConfiguredHashAlgorithm(conf));
114    builder
115      .setHash(UnsafeByteOperations.unsafeWrap(Encryption.computeCryptoKeyHash(conf, keyBytes)));
116    ByteArrayOutputStream out = new ByteArrayOutputStream();
117    if (kek != null) {
118      Encryption.encryptWithGivenKey(kek, out, new ByteArrayInputStream(keyBytes), cipher, iv);
119    } else {
120      Encryption.encryptWithSubjectKey(out, new ByteArrayInputStream(keyBytes), subject, conf,
121        cipher, iv);
122    }
123    builder.setData(UnsafeByteOperations.unsafeWrap(out.toByteArray()));
124    // Build and return the protobuf message
125    out.reset();
126    builder.build().writeDelimitedTo(out);
127    return out.toByteArray();
128  }
129
130  /**
131   * Unwrap a key by decrypting it with the secret key of the given subject. The configuration must
132   * be set up correctly for key alias resolution.
133   * @param conf    configuration
134   * @param subject subject key alias
135   * @param value   the encrypted key bytes
136   * @return the raw key bytes
137   */
138  public static Key unwrapKey(Configuration conf, String subject, byte[] value)
139    throws IOException, KeyException {
140    return unwrapKey(conf, subject, value, null);
141  }
142
143  /**
144   * Unwrap a key by decrypting it with the secret key of the given subject. The configuration must
145   * be set up correctly for key alias resolution. Only one of the {@code subject} or {@code kek}
146   * needs to be specified and the other one can be {@code null}.
147   * @param conf    configuration
148   * @param subject subject key alias
149   * @param value   the encrypted key bytes
150   * @param kek     the key encryption key
151   * @return the raw key bytes
152   */
153  public static Key unwrapKey(Configuration conf, String subject, byte[] value, Key kek)
154    throws IOException, KeyException {
155    EncryptionProtos.WrappedKey wrappedKey =
156      EncryptionProtos.WrappedKey.parser().parseDelimitedFrom(new ByteArrayInputStream(value));
157    String algorithm = conf.get(HConstants.CRYPTO_KEY_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES);
158    Cipher cipher = Encryption.getCipher(conf, algorithm);
159    if (cipher == null) {
160      throw new RuntimeException("Cipher '" + algorithm + "' not available");
161    }
162    return getUnwrapKey(conf, subject, wrappedKey, cipher, kek);
163  }
164
165  private static Key getUnwrapKey(Configuration conf, String subject,
166    EncryptionProtos.WrappedKey wrappedKey, Cipher cipher, Key kek)
167    throws IOException, KeyException {
168    String configuredHashAlgorithm = Encryption.getConfiguredHashAlgorithm(conf);
169    String wrappedHashAlgorithm = wrappedKey.getHashAlgorithm().trim();
170    if (!configuredHashAlgorithm.equalsIgnoreCase(wrappedHashAlgorithm)) {
171      String msg = String.format("Unexpected encryption key hash algorithm: %s (expecting: %s)",
172        wrappedHashAlgorithm, configuredHashAlgorithm);
173      if (Encryption.failOnHashAlgorithmMismatch(conf)) {
174        throw new KeyException(msg);
175      }
176      LOG.debug(msg);
177    }
178    ByteArrayOutputStream out = new ByteArrayOutputStream();
179    byte[] iv = wrappedKey.hasIv() ? wrappedKey.getIv().toByteArray() : null;
180    if (kek != null) {
181      Encryption.decryptWithGivenKey(kek, out, wrappedKey.getData().newInput(),
182        wrappedKey.getLength(), cipher, iv);
183    } else {
184      Encryption.decryptWithSubjectKey(out, wrappedKey.getData().newInput(), wrappedKey.getLength(),
185        subject, conf, cipher, iv);
186    }
187    byte[] keyBytes = out.toByteArray();
188    if (wrappedKey.hasHash()) {
189      if (
190        !Bytes.equals(wrappedKey.getHash().toByteArray(),
191          Encryption.hashWithAlg(wrappedHashAlgorithm, keyBytes))
192      ) {
193        throw new KeyException("Key was not successfully unwrapped");
194      }
195    }
196    return new SecretKeySpec(keyBytes, wrappedKey.getAlgorithm());
197  }
198
199  /**
200   * Unwrap a wal key by decrypting it with the secret key of the given subject. The configuration
201   * must be set up correctly for key alias resolution.
202   * @param conf    configuration
203   * @param subject subject key alias
204   * @param value   the encrypted key bytes
205   * @return the raw key bytes
206   * @throws IOException  if key is not found for the subject, or if some I/O error occurs
207   * @throws KeyException if fail to unwrap the key
208   */
209  public static Key unwrapWALKey(Configuration conf, String subject, byte[] value)
210    throws IOException, KeyException {
211    EncryptionProtos.WrappedKey wrappedKey =
212      EncryptionProtos.WrappedKey.parser().parseDelimitedFrom(new ByteArrayInputStream(value));
213    String algorithm = conf.get(HConstants.CRYPTO_WAL_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES);
214    Cipher cipher = Encryption.getCipher(conf, algorithm);
215    if (cipher == null) {
216      throw new RuntimeException("Cipher '" + algorithm + "' not available");
217    }
218    return getUnwrapKey(conf, subject, wrappedKey, cipher, null);
219  }
220
221  /**
222   * Helper for {@link #unwrapKey(Configuration, String, byte[])} which automatically uses the
223   * configured master and alternative keys, rather than having to specify a key type to unwrap
224   * with. The configuration must be set up correctly for key alias resolution.
225   * @param conf     the current configuration
226   * @param keyBytes the key encrypted by master (or alternative) to unwrap
227   * @return the key bytes, decrypted
228   * @throws IOException if the key cannot be unwrapped
229   */
230  public static Key unwrapKey(Configuration conf, byte[] keyBytes) throws IOException {
231    Key key;
232    String masterKeyName =
233      conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, User.getCurrent().getShortName());
234    try {
235      // First try the master key
236      key = unwrapKey(conf, masterKeyName, keyBytes);
237    } catch (KeyException e) {
238      // If the current master key fails to unwrap, try the alternate, if
239      // one is configured
240      if (LOG.isDebugEnabled()) {
241        LOG.debug("Unable to unwrap key with current master key '" + masterKeyName + "'");
242      }
243      String alternateKeyName = conf.get(HConstants.CRYPTO_MASTERKEY_ALTERNATE_NAME_CONF_KEY);
244      if (alternateKeyName != null) {
245        try {
246          key = unwrapKey(conf, alternateKeyName, keyBytes);
247        } catch (KeyException ex) {
248          throw new IOException(ex);
249        }
250      } else {
251        throw new IOException(e);
252      }
253    }
254    return key;
255  }
256
257  /**
258   * Helper to create an instance of CryptoAES.
259   * @param conf             The current configuration.
260   * @param cryptoCipherMeta The metadata for create CryptoAES.
261   * @return The instance of CryptoAES.
262   * @throws IOException if create CryptoAES failed
263   */
264  public static CryptoAES createCryptoAES(RPCProtos.CryptoCipherMeta cryptoCipherMeta,
265    Configuration conf) throws IOException {
266    Properties properties = new Properties();
267    // the property for cipher class
268    properties.setProperty(CryptoCipherFactory.CLASSES_KEY,
269      conf.get("hbase.rpc.crypto.encryption.aes.cipher.class",
270        "org.apache.commons.crypto.cipher.JceCipher"));
271    // create SaslAES for client
272    return new CryptoAES(cryptoCipherMeta.getTransformation(), properties,
273      cryptoCipherMeta.getInKey().toByteArray(), cryptoCipherMeta.getOutKey().toByteArray(),
274      cryptoCipherMeta.getInIv().toByteArray(), cryptoCipherMeta.getOutIv().toByteArray());
275  }
276}