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.compress;
019
020import static org.junit.Assert.assertEquals;
021import static org.junit.Assert.assertTrue;
022
023import java.util.List;
024import java.util.Random;
025import java.util.concurrent.ThreadLocalRandom;
026import org.apache.hadoop.conf.Configuration;
027import org.apache.hadoop.fs.FSDataOutputStream;
028import org.apache.hadoop.fs.FileSystem;
029import org.apache.hadoop.fs.Path;
030import org.apache.hadoop.hbase.ExtendedCell;
031import org.apache.hadoop.hbase.HBaseTestingUtil;
032import org.apache.hadoop.hbase.HConstants;
033import org.apache.hadoop.hbase.KeyValue;
034import org.apache.hadoop.hbase.KeyValueUtil;
035import org.apache.hadoop.hbase.io.hfile.CacheConfig;
036import org.apache.hadoop.hbase.io.hfile.HFile;
037import org.apache.hadoop.hbase.io.hfile.HFileContext;
038import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
039import org.apache.hadoop.hbase.io.hfile.HFileScanner;
040import org.apache.hadoop.hbase.util.RedundantKVGenerator;
041import org.slf4j.Logger;
042import org.slf4j.LoggerFactory;
043
044public class HFileTestBase {
045
046  protected static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();
047  protected static final Logger LOG = LoggerFactory.getLogger(HFileTestBase.class);
048  protected static FileSystem FS;
049
050  public static void setUpBeforeClass() throws Exception {
051    Configuration conf = TEST_UTIL.getConfiguration();
052    // Disable block cache in this test.
053    conf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.0f);
054    FS = FileSystem.get(conf);
055  }
056
057  @SuppressWarnings("deprecation")
058  public void doTest(Configuration conf, Path path, Compression.Algorithm compression)
059    throws Exception {
060    // Create 10000 random test KVs
061    RedundantKVGenerator generator = new RedundantKVGenerator();
062    List<KeyValue> testKvs = generator.generateTestKeyValues(10000);
063
064    // Iterate through data block encoding and compression combinations
065    CacheConfig cacheConf = new CacheConfig(conf);
066    HFileContext fileContext = new HFileContextBuilder().withBlockSize(4096) // small block
067      .withCompression(compression).build();
068    // write a new test HFile
069    LOG.info("Writing with " + fileContext);
070    FSDataOutputStream out = FS.create(path);
071    HFile.Writer writer = HFile.getWriterFactory(conf, cacheConf).withOutputStream(out)
072      .withFileContext(fileContext).create();
073    try {
074      for (KeyValue kv : testKvs) {
075        writer.append(kv);
076      }
077    } finally {
078      writer.close();
079      out.close();
080    }
081
082    // read it back in
083    LOG.info("Reading with " + fileContext);
084    int i = 0;
085    HFileScanner scanner = null;
086    HFile.Reader reader = HFile.createReader(FS, path, cacheConf, true, conf);
087    try {
088      scanner = reader.getScanner(conf, false, false);
089      assertTrue("Initial seekTo failed", scanner.seekTo());
090      do {
091        ExtendedCell kv = scanner.getCell();
092        assertTrue("Read back an unexpected or invalid KV",
093          testKvs.contains(KeyValueUtil.ensureKeyValue(kv)));
094        i++;
095      } while (scanner.next());
096    } finally {
097      reader.close();
098      scanner.close();
099    }
100
101    assertEquals("Did not read back as many KVs as written", i, testKvs.size());
102
103    // Test random seeks with pread
104    Random rand = ThreadLocalRandom.current();
105    LOG.info("Random seeking with " + fileContext);
106    reader = HFile.createReader(FS, path, cacheConf, true, conf);
107    try {
108      scanner = reader.getScanner(conf, false, true);
109      assertTrue("Initial seekTo failed", scanner.seekTo());
110      for (i = 0; i < 100; i++) {
111        KeyValue kv = testKvs.get(rand.nextInt(testKvs.size()));
112        assertEquals("Unable to find KV as expected: " + kv, 0, scanner.seekTo(kv));
113      }
114    } finally {
115      scanner.close();
116      reader.close();
117    }
118  }
119
120}