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.hfile;
019
020import static org.junit.jupiter.api.Assertions.assertEquals;
021import static org.junit.jupiter.api.Assertions.assertNotNull;
022import static org.junit.jupiter.api.Assertions.assertTrue;
023import static org.mockito.ArgumentMatchers.any;
024import static org.mockito.ArgumentMatchers.anyBoolean;
025import static org.mockito.Mockito.mock;
026import static org.mockito.Mockito.when;
027
028import java.io.IOException;
029import java.util.concurrent.atomic.AtomicInteger;
030import org.apache.hadoop.conf.Configuration;
031import org.apache.hadoop.fs.FSDataOutputStream;
032import org.apache.hadoop.fs.FileSystem;
033import org.apache.hadoop.fs.Path;
034import org.apache.hadoop.hbase.Cell;
035import org.apache.hadoop.hbase.HBaseTestingUtil;
036import org.apache.hadoop.hbase.KeyValue;
037import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
038import org.apache.hadoop.hbase.testclassification.IOTests;
039import org.apache.hadoop.hbase.testclassification.SmallTests;
040import org.apache.hadoop.hbase.util.Bytes;
041import org.junit.jupiter.api.Tag;
042import org.junit.jupiter.api.Test;
043
044import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
045
046/**
047 * Test
048 */
049@Tag(IOTests.TAG)
050@Tag(SmallTests.TAG)
051public class TestHFileReaderImpl {
052
053  private final static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();
054
055  static KeyValue toKV(String row) {
056    return new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"), Bytes.toBytes("qualifier"),
057      Bytes.toBytes("value"));
058  }
059
060  static String toRowStr(Cell c) {
061    return Bytes.toString(c.getRowArray(), c.getRowOffset(), c.getRowLength());
062  }
063
064  Path makeNewFile() throws IOException {
065    Path ncTFile = new Path(TEST_UTIL.getDataTestDir(), "basic.hfile");
066    FSDataOutputStream fout = TEST_UTIL.getTestFileSystem().create(ncTFile);
067    int blocksize = toKV("a").getLength() * 3;
068    HFileContext context =
069      new HFileContextBuilder().withBlockSize(blocksize).withIncludesTags(true).build();
070    Configuration conf = TEST_UTIL.getConfiguration();
071    HFile.Writer writer =
072      HFile.getWriterFactoryNoCache(conf).withOutputStream(fout).withFileContext(context).create();
073    // 4 bytes * 3 * 2 for each key/value +
074    // 3 for keys, 15 for values = 42 (woot)
075    writer.append(toKV("c"));
076    writer.append(toKV("e"));
077    writer.append(toKV("g"));
078    // block transition
079    writer.append(toKV("i"));
080    writer.append(toKV("k"));
081    writer.close();
082    fout.close();
083    return ncTFile;
084  }
085
086  /**
087   * Test that we only count block size once per block while scanning
088   */
089  @Test
090  public void testRecordBlockSize() throws IOException {
091    Path p = makeNewFile();
092    FileSystem fs = TEST_UTIL.getTestFileSystem();
093    Configuration conf = TEST_UTIL.getConfiguration();
094    HFile.Reader reader = HFile.createReader(fs, p, CacheConfig.DISABLED, true, conf);
095
096    try (HFileReaderImpl.HFileScannerImpl scanner =
097      (HFileReaderImpl.HFileScannerImpl) reader.getScanner(conf, true, true, false)) {
098      scanner.seekTo();
099
100      scanner.recordBlockSize(
101        size -> assertTrue(size > 0, "expected non-zero block size on first request"));
102      scanner.recordBlockSize(
103        size -> assertEquals(0, (int) size, "expected zero block size on second request"));
104
105      AtomicInteger blocks = new AtomicInteger(0);
106      while (scanner.next()) {
107        scanner.recordBlockSize(size -> {
108          blocks.incrementAndGet();
109          // there's only 2 cells in the second block
110          assertTrue(size < toKV("a").getLength() * 3,
111            "expected remaining block to be less than block size");
112        });
113      }
114
115      assertEquals(1, blocks.get(), "expected only one remaining block but got " + blocks.get());
116    }
117  }
118
119  @Test
120  public void testReadWorksWhenCacheCorrupt() throws Exception {
121    BlockCache mockedCache = mock(BlockCache.class);
122    when(mockedCache.getBlock(any(), anyBoolean(), anyBoolean(), anyBoolean(), any()))
123      .thenThrow(new RuntimeException("Injected error"));
124    Path p = makeNewFile();
125    FileSystem fs = TEST_UTIL.getTestFileSystem();
126    Configuration conf = TEST_UTIL.getConfiguration();
127    HFile.Reader reader = HFile.createReader(fs, p, new CacheConfig(conf, mockedCache), true, conf);
128    long offset = 0;
129    while (offset < reader.getTrailer().getLoadOnOpenDataOffset()) {
130      HFileBlock block = reader.readBlock(offset, -1, false, true, false, true, null, null, false);
131      assertNotNull(block);
132      offset += block.getOnDiskSizeWithHeader();
133    }
134  }
135
136  @Test
137  public void testSeekBefore() throws Exception {
138    Path p = makeNewFile();
139    FileSystem fs = TEST_UTIL.getTestFileSystem();
140    Configuration conf = TEST_UTIL.getConfiguration();
141    int[] bucketSizes = { 512, 2048, 4096, 64 * 1024, 128 * 1024 };
142    BucketCache bucketcache =
143      new BucketCache("offheap", 128 * 1024 * 1024, 64 * 1024, bucketSizes, 5, 64 * 100, null);
144
145    HFile.Reader reader = HFile.createReader(fs, p, new CacheConfig(conf, bucketcache), true, conf);
146
147    // warm cache
148    HFileScanner scanner = reader.getScanner(conf, true, true);
149    scanner.seekTo(toKV("i"));
150    assertEquals("i", toRowStr(scanner.getCell()));
151    scanner.close();
152
153    while (bucketcache.getBlockCount() <= 0) {
154      Thread.sleep(10);
155    }
156
157    // reopen again.
158    scanner = reader.getScanner(conf, true, true);
159    scanner.seekTo(toKV("i"));
160    assertEquals("i", toRowStr(scanner.getCell()));
161    scanner.seekBefore(toKV("i"));
162    assertEquals("g", toRowStr(scanner.getCell()));
163    scanner.close();
164
165    for (CachedBlock cachedBlock : Lists.newArrayList(bucketcache)) {
166      BlockCacheKey cacheKey =
167        new BlockCacheKey(cachedBlock.getFilename(), cachedBlock.getOffset());
168      int refCount = bucketcache.getRpcRefCount(cacheKey);
169      assertEquals(0, refCount);
170    }
171
172    // case 2
173    scanner = reader.getScanner(conf, true, true);
174    scanner.seekTo(toKV("i"));
175    assertEquals("i", toRowStr(scanner.getCell()));
176    scanner.seekBefore(toKV("c"));
177    scanner.close();
178    for (CachedBlock cachedBlock : Lists.newArrayList(bucketcache)) {
179      BlockCacheKey cacheKey =
180        new BlockCacheKey(cachedBlock.getFilename(), cachedBlock.getOffset());
181      int refCount = bucketcache.getRpcRefCount(cacheKey);
182      assertEquals(0, refCount);
183    }
184
185    reader.close();
186
187    // clear bucketcache
188    for (CachedBlock cachedBlock : Lists.newArrayList(bucketcache)) {
189      BlockCacheKey cacheKey =
190        new BlockCacheKey(cachedBlock.getFilename(), cachedBlock.getOffset());
191      bucketcache.evictBlock(cacheKey);
192    }
193    bucketcache.shutdown();
194
195    deleteTestDir(fs);
196  }
197
198  protected void deleteTestDir(FileSystem fs) throws IOException {
199    Path dataTestDir = TEST_UTIL.getDataTestDir();
200    if (fs.exists(dataTestDir)) {
201      fs.delete(dataTestDir, true);
202    }
203  }
204
205}