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