1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.io.crypto.aes;
19
20 import com.google.common.base.Preconditions;
21 import java.io.IOException;
22 import java.io.InputStream;
23 import java.io.OutputStream;
24 import java.security.GeneralSecurityException;
25 import java.security.Key;
26 import java.security.SecureRandom;
27 import javax.crypto.spec.SecretKeySpec;
28 import org.apache.commons.logging.Log;
29 import org.apache.commons.logging.LogFactory;
30 import org.apache.hadoop.hbase.classification.InterfaceAudience;
31 import org.apache.hadoop.hbase.classification.InterfaceStability;
32 import org.apache.hadoop.hbase.io.crypto.Cipher;
33 import org.apache.hadoop.hbase.io.crypto.CipherProvider;
34 import org.apache.hadoop.hbase.io.crypto.Context;
35 import org.apache.hadoop.hbase.io.crypto.Decryptor;
36 import org.apache.hadoop.hbase.io.crypto.Encryptor;
37
38
39
40
41
42
43
44 @InterfaceAudience.Private
45 @InterfaceStability.Evolving
46 public class AES extends Cipher {
47
48 private static final Log LOG = LogFactory.getLog(AES.class);
49
50 public static final int KEY_LENGTH = 16;
51 public static final int KEY_LENGTH_BITS = KEY_LENGTH * 8;
52 public static final int BLOCK_SIZE = 16;
53 public static final int IV_LENGTH = 16;
54
55 public static final String CIPHER_MODE_KEY = "hbase.crypto.algorithm.aes.mode";
56 public static final String CIPHER_PROVIDER_KEY = "hbase.crypto.algorithm.aes.provider";
57 public static final String RNG_ALGORITHM_KEY = "hbase.crypto.algorithm.rng";
58 public static final String RNG_PROVIDER_KEY = "hbase.crypto.algorithm.rng.provider";
59
60 private final String rngAlgorithm;
61 private final String cipherMode;
62 private final String cipherProvider;
63 private SecureRandom rng;
64
65 public AES(CipherProvider provider) {
66 super(provider);
67
68 cipherMode = provider.getConf().get(CIPHER_MODE_KEY, "AES/CTR/NoPadding");
69
70 cipherProvider = provider.getConf().get(CIPHER_PROVIDER_KEY);
71
72 rngAlgorithm = provider.getConf().get(RNG_ALGORITHM_KEY, "SHA1PRNG");
73
74 String rngProvider = provider.getConf().get(RNG_PROVIDER_KEY);
75 try {
76 if (rngProvider != null) {
77 rng = SecureRandom.getInstance(rngAlgorithm, rngProvider);
78 } else {
79 rng = SecureRandom.getInstance(rngAlgorithm);
80 }
81 } catch (GeneralSecurityException e) {
82 LOG.warn("Could not instantiate specified RNG, falling back to default", e);
83 rng = new SecureRandom();
84 }
85 }
86
87 @Override
88 public String getName() {
89 return "AES";
90 }
91
92 @Override
93 public int getKeyLength() {
94 return KEY_LENGTH;
95 }
96
97 @Override
98 public int getIvLength() {
99 return IV_LENGTH;
100 }
101
102 @Override
103 public Key getRandomKey() {
104 byte[] keyBytes = new byte[getKeyLength()];
105 rng.nextBytes(keyBytes);
106 return new SecretKeySpec(keyBytes, getName());
107 }
108
109 @Override
110 public Encryptor getEncryptor() {
111 return new AESEncryptor(getJCECipherInstance(), rng);
112 }
113
114 @Override
115 public Decryptor getDecryptor() {
116 return new AESDecryptor(getJCECipherInstance());
117 }
118
119 @Override
120 public OutputStream createEncryptionStream(OutputStream out, Context context, byte[] iv)
121 throws IOException {
122 Preconditions.checkNotNull(context);
123 Preconditions.checkState(context.getKey() != null, "Context does not have a key");
124 Preconditions.checkNotNull(iv);
125 Encryptor e = getEncryptor();
126 e.setKey(context.getKey());
127 e.setIv(iv);
128 return e.createEncryptionStream(out);
129 }
130
131 @Override
132 public OutputStream createEncryptionStream(OutputStream out, Encryptor e) throws IOException {
133 Preconditions.checkNotNull(e);
134 return e.createEncryptionStream(out);
135 }
136
137 @Override
138 public InputStream createDecryptionStream(InputStream in, Context context, byte[] iv)
139 throws IOException {
140 Preconditions.checkNotNull(context);
141 Preconditions.checkState(context.getKey() != null, "Context does not have a key");
142 Preconditions.checkNotNull(iv);
143 Decryptor d = getDecryptor();
144 d.setKey(context.getKey());
145 d.setIv(iv);
146 return d.createDecryptionStream(in);
147 }
148
149 @Override
150 public InputStream createDecryptionStream(InputStream in, Decryptor d) throws IOException {
151 Preconditions.checkNotNull(d);
152 return d.createDecryptionStream(in);
153 }
154
155 SecureRandom getRNG() {
156 return rng;
157 }
158
159 private javax.crypto.Cipher getJCECipherInstance() {
160 try {
161 if (cipherProvider != null) {
162 return javax.crypto.Cipher.getInstance(cipherMode, cipherProvider);
163 }
164 return javax.crypto.Cipher.getInstance(cipherMode);
165 } catch (GeneralSecurityException e) {
166 throw new RuntimeException(e);
167 }
168 }
169
170 }