3 * Licensed to the Apache Software Foundation (ASF) under one
4 * or more contributor license agreements. See the NOTICE file
5 * distributed with this work for additional information
6 * regarding copyright ownership. The ASF licenses this file
7 * to you under the Apache License, Version 2.0 (the
8 * "License"); you may not use this file except in compliance
9 * with the License. You may obtain a copy of the License at
11 * http://www.apache.org/licenses/LICENSE-2.0
13 * Unless required by applicable law or agreed to in writing, software
14 * distributed under the License is distributed on an "AS IS" BASIS,
15 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16 * See the License for the specific language governing permissions and
17 * limitations under the License.
19 package org
.apache
.hadoop
.hbase
.util
;
21 import java
.io
.ByteArrayInputStream
;
22 import java
.io
.ByteArrayOutputStream
;
23 import java
.io
.IOException
;
25 import java
.util
.concurrent
.ConcurrentHashMap
;
27 import org
.apache
.hadoop
.hbase
.HBaseInterfaceAudience
;
28 import org
.apache
.yetus
.audience
.InterfaceAudience
;
29 import org
.slf4j
.Logger
;
30 import org
.slf4j
.LoggerFactory
;
31 import org
.apache
.hadoop
.conf
.Configuration
;
32 import org
.apache
.hadoop
.hbase
.HConstants
;
33 import org
.apache
.hadoop
.hbase
.io
.crypto
.DefaultCipherProvider
;
34 import org
.apache
.hadoop
.hbase
.io
.crypto
.Encryption
;
35 import org
.apache
.hadoop
.hbase
.io
.crypto
.KeyStoreKeyProvider
;
36 import org
.apache
.hadoop
.hbase
.security
.EncryptionUtil
;
38 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience
.TOOLS
)
39 public class EncryptionTest
{
40 private static final Logger LOG
= LoggerFactory
.getLogger(EncryptionTest
.class);
42 static final Map
<String
, Boolean
> keyProviderResults
= new ConcurrentHashMap
<>();
43 static final Map
<String
, Boolean
> cipherProviderResults
= new ConcurrentHashMap
<>();
44 static final Map
<String
, Boolean
> cipherResults
= new ConcurrentHashMap
<>();
46 private EncryptionTest() {
50 * Check that the configured key provider can be loaded and initialized, or
56 public static void testKeyProvider(final Configuration conf
) throws IOException
{
57 String providerClassName
= conf
.get(HConstants
.CRYPTO_KEYPROVIDER_CONF_KEY
,
58 KeyStoreKeyProvider
.class.getName());
59 Boolean result
= keyProviderResults
.get(providerClassName
);
62 Encryption
.getKeyProvider(conf
);
63 keyProviderResults
.put(providerClassName
, true);
64 } catch (Exception e
) { // most likely a RuntimeException
65 keyProviderResults
.put(providerClassName
, false);
66 throw new IOException("Key provider " + providerClassName
+ " failed test: " +
69 } else if (result
.booleanValue() == false) {
70 throw new IOException("Key provider " + providerClassName
+ " previously failed test");
75 * Check that the configured cipher provider can be loaded and initialized, or
81 public static void testCipherProvider(final Configuration conf
) throws IOException
{
82 String providerClassName
= conf
.get(HConstants
.CRYPTO_CIPHERPROVIDER_CONF_KEY
,
83 DefaultCipherProvider
.class.getName());
84 Boolean result
= cipherProviderResults
.get(providerClassName
);
87 Encryption
.getCipherProvider(conf
);
88 cipherProviderResults
.put(providerClassName
, true);
89 } catch (Exception e
) { // most likely a RuntimeException
90 cipherProviderResults
.put(providerClassName
, false);
91 throw new IOException("Cipher provider " + providerClassName
+ " failed test: " +
94 } else if (result
.booleanValue() == false) {
95 throw new IOException("Cipher provider " + providerClassName
+ " previously failed test");
100 * Check that the specified cipher can be loaded and initialized, or throw
101 * an exception. Verifies key and cipher provider configuration as a
102 * prerequisite for cipher verification.
107 * @throws IOException
109 public static void testEncryption(final Configuration conf
, final String cipher
,
110 byte[] key
) throws IOException
{
111 if (cipher
== null) {
114 testKeyProvider(conf
);
115 testCipherProvider(conf
);
116 Boolean result
= cipherResults
.get(cipher
);
117 if (result
== null) {
119 Encryption
.Context context
= Encryption
.newContext(conf
);
120 context
.setCipher(Encryption
.getCipher(conf
, cipher
));
122 // Make a random key since one was not provided
123 context
.setKey(context
.getCipher().getRandomKey());
125 // This will be a wrapped key from schema
126 context
.setKey(EncryptionUtil
.unwrapKey(conf
,
127 conf
.get(HConstants
.CRYPTO_MASTERKEY_NAME_CONF_KEY
, "hbase"),
131 if (context
.getCipher().getIvLength() > 0) {
132 iv
= new byte[context
.getCipher().getIvLength()];
135 byte[] plaintext
= new byte[1024];
136 Bytes
.random(plaintext
);
137 ByteArrayOutputStream out
= new ByteArrayOutputStream();
138 Encryption
.encrypt(out
, new ByteArrayInputStream(plaintext
), context
, iv
);
139 byte[] ciphertext
= out
.toByteArray();
141 Encryption
.decrypt(out
, new ByteArrayInputStream(ciphertext
), plaintext
.length
,
143 byte[] test
= out
.toByteArray();
144 if (!Bytes
.equals(plaintext
, test
)) {
145 throw new IOException("Did not pass encrypt/decrypt test");
147 cipherResults
.put(cipher
, true);
148 } catch (Exception e
) {
149 cipherResults
.put(cipher
, false);
150 throw new IOException("Cipher " + cipher
+ " failed test: " + e
.getMessage(), e
);
152 } else if (result
.booleanValue() == false) {
153 throw new IOException("Cipher " + cipher
+ " previously failed test");