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.aes; 019 020import java.io.IOException; 021import java.io.InputStream; 022import java.io.OutputStream; 023import java.security.GeneralSecurityException; 024import java.security.Key; 025import java.security.SecureRandom; 026import java.util.Properties; 027import javax.crypto.spec.SecretKeySpec; 028import org.apache.commons.crypto.cipher.CryptoCipherFactory; 029import org.apache.hadoop.conf.Configuration; 030import org.apache.hadoop.hbase.io.crypto.Cipher; 031import org.apache.hadoop.hbase.io.crypto.CipherProvider; 032import org.apache.hadoop.hbase.io.crypto.Context; 033import org.apache.hadoop.hbase.io.crypto.Decryptor; 034import org.apache.hadoop.hbase.io.crypto.Encryptor; 035import org.apache.yetus.audience.InterfaceAudience; 036import org.apache.yetus.audience.InterfaceStability; 037import org.slf4j.Logger; 038import org.slf4j.LoggerFactory; 039 040import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; 041 042@InterfaceAudience.Private 043@InterfaceStability.Evolving 044public class CommonsCryptoAES extends Cipher { 045 046 private static final Logger LOG = LoggerFactory.getLogger(CommonsCryptoAES.class); 047 048 public static final String CIPHER_MODE_KEY = "hbase.crypto.commons.mode"; 049 public static final String CIPHER_CLASSES_KEY = "hbase.crypto.commons.cipher.classes"; 050 public static final String CIPHER_JCE_PROVIDER_KEY = "hbase.crypto.commons.cipher.jce.provider"; 051 052 private final String cipherMode; 053 private Properties props; 054 private final String rngAlgorithm; 055 private SecureRandom rng; 056 057 public CommonsCryptoAES(CipherProvider provider) { 058 super(provider); 059 // The mode for Commons Crypto Ciphers 060 cipherMode = provider.getConf().get(CIPHER_MODE_KEY, "AES/CTR/NoPadding"); 061 // Reads Commons Crypto properties from HBase conf 062 props = readCryptoProps(provider.getConf()); 063 // RNG algorithm 064 rngAlgorithm = provider.getConf().get(RNG_ALGORITHM_KEY, "SHA1PRNG"); 065 // RNG provider, null if default 066 String rngProvider = provider.getConf().get(RNG_PROVIDER_KEY); 067 try { 068 if (rngProvider != null) { 069 rng = SecureRandom.getInstance(rngAlgorithm, rngProvider); 070 } else { 071 rng = SecureRandom.getInstance(rngAlgorithm); 072 } 073 } catch (GeneralSecurityException e) { 074 LOG.warn("Could not instantiate specified RNG, falling back to default", e); 075 rng = new SecureRandom(); 076 } 077 } 078 079 private static Properties readCryptoProps(Configuration conf) { 080 Properties props = new Properties(); 081 082 props.setProperty(CryptoCipherFactory.CLASSES_KEY, conf.get(CIPHER_CLASSES_KEY, "")); 083 props.setProperty(CryptoCipherFactory.JCE_PROVIDER_KEY, conf.get(CIPHER_JCE_PROVIDER_KEY, "")); 084 085 return props; 086 } 087 088 @Override 089 public String getName() { 090 return "AES"; 091 } 092 093 @Override 094 public int getKeyLength() { 095 return KEY_LENGTH; 096 } 097 098 @Override 099 public int getIvLength() { 100 return IV_LENGTH; 101 } 102 103 @Override 104 public Key getRandomKey() { 105 byte[] keyBytes = new byte[getKeyLength()]; 106 rng.nextBytes(keyBytes); 107 return new SecretKeySpec(keyBytes, getName()); 108 } 109 110 @Override 111 public Encryptor getEncryptor() { 112 return new CommonsCryptoAESEncryptor(cipherMode, props, rng); 113 } 114 115 @Override 116 public Decryptor getDecryptor() { 117 return new CommonsCryptoAESDecryptor(cipherMode, props); 118 } 119 120 @Override 121 public OutputStream createEncryptionStream(OutputStream out, Context context, 122 byte[] iv) throws IOException { 123 Preconditions.checkNotNull(context); 124 Preconditions.checkState(context.getKey() != null, "Context does not have a key"); 125 Preconditions.checkNotNull(iv); 126 Encryptor e = getEncryptor(); 127 e.setKey(context.getKey()); 128 e.setIv(iv); 129 return e.createEncryptionStream(out); 130 } 131 132 @Override 133 public OutputStream createEncryptionStream(OutputStream out, 134 Encryptor encryptor) throws 135 IOException { 136 return encryptor.createEncryptionStream(out); 137 } 138 139 @Override 140 public InputStream createDecryptionStream(InputStream in, Context context, 141 byte[] iv) throws IOException { 142 Preconditions.checkNotNull(context); 143 Preconditions.checkState(context.getKey() != null, "Context does not have a key"); 144 Preconditions.checkNotNull(iv); 145 Decryptor d = getDecryptor(); 146 d.setKey(context.getKey()); 147 d.setIv(iv); 148 return d.createDecryptionStream(in); 149 } 150 151 @Override 152 public InputStream createDecryptionStream(InputStream in, 153 Decryptor decryptor) throws 154 IOException { 155 Preconditions.checkNotNull(decryptor); 156 return decryptor.createDecryptionStream(in); 157 } 158 159 SecureRandom getRNG() { 160 return rng; 161 } 162}