View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.io.hfile;
19  
20  import java.io.DataInputStream;
21  import java.io.DataOutputStream;
22  import java.io.IOException;
23  import java.security.SecureRandom;
24  import java.util.List;
25  import java.util.UUID;
26  
27  import org.apache.commons.logging.Log;
28  import org.apache.commons.logging.LogFactory;
29  import org.apache.hadoop.conf.Configuration;
30  import org.apache.hadoop.fs.FSDataInputStream;
31  import org.apache.hadoop.fs.FSDataOutputStream;
32  import org.apache.hadoop.fs.FileSystem;
33  import org.apache.hadoop.fs.Path;
34  import org.apache.hadoop.hbase.Cell;
35  import org.apache.hadoop.hbase.HBaseTestingUtility;
36  import org.apache.hadoop.hbase.HConstants;
37  import org.apache.hadoop.hbase.KeyValue;
38  import org.apache.hadoop.hbase.KeyValueUtil;
39  import org.apache.hadoop.hbase.testclassification.SmallTests;
40  import org.apache.hadoop.hbase.io.compress.Compression;
41  import org.apache.hadoop.hbase.io.crypto.Cipher;
42  import org.apache.hadoop.hbase.io.crypto.Encryption;
43  import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
44  import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
45  import org.apache.hadoop.hbase.util.Bytes;
46  import org.apache.hadoop.hbase.util.test.RedundantKVGenerator;
47  import org.junit.BeforeClass;
48  import org.junit.Test;
49  import org.junit.experimental.categories.Category;
50  
51  import static org.junit.Assert.*;
52  
53  @Category(SmallTests.class)
54  public class TestHFileEncryption {
55    private static final Log LOG = LogFactory.getLog(TestHFileEncryption.class);
56    private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
57    private static final SecureRandom RNG = new SecureRandom();
58  
59    private static FileSystem fs;
60    private static Encryption.Context cryptoContext;
61  
62    @BeforeClass
63    public static void setUp() throws Exception {
64      Configuration conf = TEST_UTIL.getConfiguration();
65      conf.set(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY, KeyProviderForTesting.class.getName());
66      conf.set(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, "hbase");
67      conf.setInt("hfile.format.version", 3);
68  
69      fs = FileSystem.get(conf);
70  
71      cryptoContext = Encryption.newContext(conf);
72      Cipher aes = Encryption.getCipher(conf, "AES");
73      assertNotNull(aes);
74      cryptoContext.setCipher(aes);
75      byte[] key = new byte[aes.getKeyLength()];
76      RNG.nextBytes(key);
77      cryptoContext.setKey(key);
78    }
79  
80    private int writeBlock(FSDataOutputStream os, HFileContext fileContext, int size)
81        throws IOException {
82      HFileBlock.Writer hbw = new HFileBlock.Writer(null, fileContext);
83      DataOutputStream dos = hbw.startWriting(BlockType.DATA);
84      for (int j = 0; j < size; j++) {
85        dos.writeInt(j);
86      }
87      hbw.writeHeaderAndData(os);
88      LOG.info("Wrote a block at " + os.getPos() + " with" +
89          " onDiskSizeWithHeader=" + hbw.getOnDiskSizeWithHeader() +
90          " uncompressedSizeWithoutHeader=" + hbw.getOnDiskSizeWithoutHeader() +
91          " uncompressedSizeWithoutHeader=" + hbw.getUncompressedSizeWithoutHeader());
92      return hbw.getOnDiskSizeWithHeader();
93    }
94  
95    private long readAndVerifyBlock(long pos, HFileContext ctx, HFileBlock.FSReaderImpl hbr, int size)
96        throws IOException {
97      HFileBlock b = hbr.readBlockData(pos, -1, -1, false);
98      assertEquals(0, HFile.getChecksumFailuresCount());
99      b.sanityCheck();
100     assertFalse(b.isUnpacked());
101     b = b.unpack(ctx, hbr);
102     LOG.info("Read a block at " + pos + " with" +
103         " onDiskSizeWithHeader=" + b.getOnDiskSizeWithHeader() +
104         " uncompressedSizeWithoutHeader=" + b.getOnDiskSizeWithoutHeader() +
105         " uncompressedSizeWithoutHeader=" + b.getUncompressedSizeWithoutHeader());
106     DataInputStream dis = b.getByteStream();
107     for (int i = 0; i < size; i++) {
108       int read = dis.readInt();
109       if (read != i) {
110         fail("Block data corrupt at element " + i);
111       }
112     }
113     return b.getOnDiskSizeWithHeader();
114   }
115 
116   @Test(timeout=20000)
117   public void testDataBlockEncryption() throws IOException {
118     final int blocks = 10;
119     final int[] blockSizes = new int[blocks];
120     for (int i = 0; i < blocks; i++) {
121       blockSizes[i] = (1024 + RNG.nextInt(1024 * 63)) / Bytes.SIZEOF_INT;
122     }
123     for (Compression.Algorithm compression : TestHFileBlock.COMPRESSION_ALGORITHMS) {
124       Path path = new Path(TEST_UTIL.getDataTestDir(), "block_v3_" + compression + "_AES");
125       LOG.info("testDataBlockEncryption: encryption=AES compression=" + compression);
126       long totalSize = 0;
127       HFileContext fileContext = new HFileContextBuilder()
128         .withCompression(compression)
129         .withEncryptionContext(cryptoContext)
130         .build();
131       FSDataOutputStream os = fs.create(path);
132       try {
133         for (int i = 0; i < blocks; i++) {
134           totalSize += writeBlock(os, fileContext, blockSizes[i]);
135         }
136       } finally {
137         os.close();
138       }
139       FSDataInputStream is = fs.open(path);
140       try {
141         HFileBlock.FSReaderImpl hbr = new HFileBlock.FSReaderImpl(is, totalSize, fileContext);
142         long pos = 0;
143         for (int i = 0; i < blocks; i++) {
144           pos += readAndVerifyBlock(pos, fileContext, hbr, blockSizes[i]);
145         }
146       } finally {
147         is.close();
148       }
149     }
150   }
151 
152   @Test(timeout=20000)
153   public void testHFileEncryptionMetadata() throws Exception {
154     Configuration conf = TEST_UTIL.getConfiguration();
155     CacheConfig cacheConf = new CacheConfig(conf);
156 
157     HFileContext fileContext = new HFileContextBuilder()
158     .withEncryptionContext(cryptoContext)
159     .build();
160 
161     // write a simple encrypted hfile
162     Path path = new Path(TEST_UTIL.getDataTestDir(), "cryptometa.hfile");
163     FSDataOutputStream out = fs.create(path);
164     HFile.Writer writer = HFile.getWriterFactory(conf, cacheConf)
165       .withOutputStream(out)
166       .withFileContext(fileContext)
167       .create();
168     KeyValue kv = new KeyValue("foo".getBytes(), "f1".getBytes(), null, "value".getBytes());
169     writer.append(kv);
170     writer.close();
171     out.close();
172 
173     // read it back in and validate correct crypto metadata
174     HFile.Reader reader = HFile.createReader(fs, path, cacheConf, conf);
175     reader.loadFileInfo();
176     FixedFileTrailer trailer = reader.getTrailer();
177     assertNotNull(trailer.getEncryptionKey());
178     Encryption.Context readerContext = reader.getFileContext().getEncryptionContext();
179     assertEquals(readerContext.getCipher().getName(), cryptoContext.getCipher().getName());
180     assertTrue(Bytes.equals(readerContext.getKeyBytes(),
181       cryptoContext.getKeyBytes()));
182   }
183 
184   @Test(timeout=6000000)
185   public void testHFileEncryption() throws Exception {
186     // Create 1000 random test KVs
187     RedundantKVGenerator generator = new RedundantKVGenerator();
188     List<KeyValue> testKvs = generator.generateTestKeyValues(1000);
189 
190     // Iterate through data block encoding and compression combinations
191     Configuration conf = TEST_UTIL.getConfiguration();
192     CacheConfig cacheConf = new CacheConfig(conf);
193     for (DataBlockEncoding encoding: DataBlockEncoding.values()) {
194       for (Compression.Algorithm compression: TestHFileBlock.COMPRESSION_ALGORITHMS) {
195         HFileContext fileContext = new HFileContextBuilder()
196           .withBlockSize(4096) // small blocks
197           .withEncryptionContext(cryptoContext)
198           .withCompression(compression)
199           .withDataBlockEncoding(encoding)
200           .build();
201         // write a new test HFile
202         LOG.info("Writing with " + fileContext);
203         Path path = new Path(TEST_UTIL.getDataTestDir(), UUID.randomUUID().toString() + ".hfile");
204         FSDataOutputStream out = fs.create(path);
205         HFile.Writer writer = HFile.getWriterFactory(conf, cacheConf)
206           .withOutputStream(out)
207           .withFileContext(fileContext)
208           .create();
209         for (KeyValue kv: testKvs) {
210           writer.append(kv);
211         }
212         writer.close();
213         out.close();
214 
215         // read it back in
216         LOG.info("Reading with " + fileContext);
217         HFile.Reader reader = HFile.createReader(fs, path, cacheConf, conf);
218         reader.loadFileInfo();
219         FixedFileTrailer trailer = reader.getTrailer();
220         assertNotNull(trailer.getEncryptionKey());
221         HFileScanner scanner = reader.getScanner(false, false);
222         assertTrue("Initial seekTo failed", scanner.seekTo());
223         int i = 0;
224         do {
225           Cell kv = scanner.getKeyValue();
226           assertTrue("Read back an unexpected or invalid KV",
227               testKvs.contains(KeyValueUtil.ensureKeyValue(kv)));
228           i++;
229         } while (scanner.next());
230         reader.close();
231 
232         assertEquals("Did not read back as many KVs as written", i, testKvs.size());
233 
234         // Test random seeks with pread
235         LOG.info("Random seeking with " + fileContext);
236         reader = HFile.createReader(fs, path, cacheConf, conf);
237         scanner = reader.getScanner(false, true);
238         assertTrue("Initial seekTo failed", scanner.seekTo());
239         for (i = 0; i < 100; i++) {
240           KeyValue kv = testKvs.get(RNG.nextInt(testKvs.size()));
241           assertEquals("Unable to find KV as expected: " + kv, scanner.seekTo(kv), 0);
242         }
243         reader.close();
244       }
245     }
246   }
247 
248 }