View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one or more
3    * contributor license agreements. See the NOTICE file distributed with this
4    * work for additional information regarding copyright ownership. The ASF
5    * licenses this file to you under the Apache License, Version 2.0 (the
6    * "License"); you may not use this file except in compliance with the License.
7    * You may obtain a copy of the License at
8    *
9    * http://www.apache.org/licenses/LICENSE-2.0
10   *
11   * Unless required by applicable law or agreed to in writing, software
12   * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
13   * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
14   * License for the specific language governing permissions and limitations
15   * under the License.
16   */
17  package org.apache.hadoop.hbase.io.hfile;
18  
19  import java.util.ArrayList;
20  import java.util.List;
21  
22  import org.apache.hadoop.conf.Configuration;
23  import org.apache.hadoop.fs.FileSystem;
24  import org.apache.hadoop.fs.Path;
25  import org.apache.hadoop.hbase.CellUtil;
26  import org.apache.hadoop.hbase.HBaseTestingUtility;
27  import org.apache.hadoop.hbase.testclassification.SmallTests;
28  import org.apache.hadoop.hbase.util.Bytes;
29  import org.junit.Test;
30  import org.junit.experimental.categories.Category;
31  
32  /**
33   * Test a case when an inline index chunk is converted to a root one. This reproduces the bug in
34   * HBASE-6871. We write a carefully selected number of relatively large keys so that we accumulate
35   * a leaf index chunk that only goes over the configured index chunk size after adding the last
36   * key/value. The bug is in that when we close the file, we convert that inline (leaf-level) chunk
37   * into a root chunk, but then look at the size of that root chunk, find that it is greater than
38   * the configured chunk size, and split it into a number of intermediate index blocks that should
39   * really be leaf-level blocks. If more keys were added, we would flush the leaf-level block, add
40   * another entry to the root-level block, and that would prevent us from upgrading the leaf-level
41   * chunk to the root chunk, thus not triggering the bug.
42   */
43  @Category(SmallTests.class)
44  public class TestHFileInlineToRootChunkConversion {
45    private final HBaseTestingUtility testUtil = new HBaseTestingUtility();
46    private final Configuration conf = testUtil.getConfiguration();
47  
48    @Test
49    public void testWriteHFile() throws Exception {
50      Path hfPath = new Path(testUtil.getDataTestDir(),
51          TestHFileInlineToRootChunkConversion.class.getSimpleName() + ".hfile");
52      int maxChunkSize = 1024;
53      FileSystem fs = FileSystem.get(conf);
54      CacheConfig cacheConf = new CacheConfig(conf);
55      conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, maxChunkSize);
56      HFileContext context = new HFileContextBuilder().withBlockSize(16).build();
57      HFileWriterV2 hfw =
58          (HFileWriterV2) new HFileWriterV2.WriterFactoryV2(conf, cacheConf)
59              .withFileContext(context)
60              .withPath(fs, hfPath).create();
61      List<byte[]> keys = new ArrayList<byte[]>();
62      StringBuilder sb = new StringBuilder();
63  
64      for (int i = 0; i < 4; ++i) {
65        sb.append("key" + String.format("%05d", i));
66        sb.append("_");
67        for (int j = 0; j < 100; ++j) {
68          sb.append('0' + j);
69        }
70        String keyStr = sb.toString();
71        sb.setLength(0);
72  
73        byte[] k = Bytes.toBytes(keyStr);
74        keys.add(k);
75        byte[] v = Bytes.toBytes("value" + i);
76        hfw.append(CellUtil.createCell(k, v));
77      }
78      hfw.close();
79  
80      HFileReaderV2 reader = (HFileReaderV2) HFile.createReader(fs, hfPath, cacheConf, conf);
81      // Scanner doesn't do Cells yet.  Fix.
82      HFileScanner scanner = reader.getScanner(true, true);
83      for (int i = 0; i < keys.size(); ++i) {
84        scanner.seekTo(CellUtil.createCell(keys.get(i)));
85      }
86      reader.close();
87    }
88  }