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.util;
019
020import java.io.IOException;
021import java.util.Locale;
022import org.apache.commons.lang3.StringUtils;
023import org.apache.hadoop.conf.Configuration;
024import org.apache.hadoop.fs.FileSystem;
025import org.apache.hadoop.fs.Path;
026import org.apache.hadoop.hbase.Cell;
027import org.apache.hadoop.hbase.CellComparator;
028import org.apache.hadoop.hbase.CellUtil;
029import org.apache.hadoop.hbase.DoNotRetryIOException;
030import org.apache.hadoop.hbase.HBaseConfiguration;
031import org.apache.hadoop.hbase.HBaseInterfaceAudience;
032import org.apache.hadoop.hbase.io.compress.Compression;
033import org.apache.hadoop.hbase.io.hfile.CacheConfig;
034import org.apache.hadoop.hbase.io.hfile.HFile;
035import org.apache.hadoop.hbase.io.hfile.HFileContext;
036import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
037import org.apache.hadoop.hbase.io.hfile.HFileScanner;
038import org.apache.hadoop.hbase.io.hfile.HFileWriterImpl;
039import org.apache.hadoop.io.compress.Compressor;
040import org.apache.yetus.audience.InterfaceAudience;
041import org.apache.yetus.audience.InterfaceStability;
042import org.slf4j.Logger;
043import org.slf4j.LoggerFactory;
044
045/**
046 * Compression validation test. Checks compression is working. Be sure to run on every node in your
047 * cluster.
048 */
049@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
050@InterfaceStability.Evolving
051public class CompressionTest {
052  private static final Logger LOG = LoggerFactory.getLogger(CompressionTest.class);
053
054  public static boolean testCompression(String codec) {
055    codec = codec.toLowerCase(Locale.ROOT);
056
057    Compression.Algorithm a;
058
059    try {
060      a = Compression.getCompressionAlgorithmByName(codec);
061    } catch (IllegalArgumentException e) {
062      LOG.warn("Codec type: " + codec + " is not known");
063      return false;
064    }
065
066    try {
067      testCompression(a);
068      return true;
069    } catch (IOException ignored) {
070      LOG.warn("Can't instantiate codec: " + codec, ignored);
071      return false;
072    }
073  }
074
075  private final static Boolean[] compressionTestResults =
076    new Boolean[Compression.Algorithm.values().length];
077  static {
078    for (int i = 0; i < compressionTestResults.length; ++i) {
079      compressionTestResults[i] = null;
080    }
081  }
082
083  public static void testCompression(Compression.Algorithm algo) throws IOException {
084    if (compressionTestResults[algo.ordinal()] != null) {
085      if (compressionTestResults[algo.ordinal()]) {
086        return; // already passed test, dont do it again.
087      } else {
088        // failed.
089        throw new DoNotRetryIOException(
090          "Compression algorithm '" + algo.getName() + "'" + " previously failed test.");
091      }
092    }
093
094    try {
095      Compressor c = algo.getCompressor();
096      algo.returnCompressor(c);
097      compressionTestResults[algo.ordinal()] = true; // passes
098    } catch (Throwable t) {
099      compressionTestResults[algo.ordinal()] = false; // failure
100      throw new DoNotRetryIOException(t);
101    }
102  }
103
104  protected static Path path = new Path(".hfile-comp-test");
105
106  public static void usage() {
107
108    System.err.println("Usage: CompressionTest <path> "
109      + StringUtils.join(Compression.Algorithm.values(), "|").toLowerCase(Locale.ROOT) + "\n"
110      + "For example:\n" + "  hbase " + CompressionTest.class + " file:///tmp/testfile gz\n");
111    System.exit(1);
112  }
113
114  public static void doSmokeTest(FileSystem fs, Path path, String codec) throws Exception {
115    Configuration conf = HBaseConfiguration.create();
116    HFileContext context =
117      new HFileContextBuilder().withCompression(HFileWriterImpl.compressionByName(codec)).build();
118    HFile.Writer writer =
119      HFile.getWriterFactoryNoCache(conf).withPath(fs, path).withFileContext(context).create();
120    // Write any-old Cell...
121    final byte[] rowKey = Bytes.toBytes("compressiontestkey");
122    Cell c = CellUtil.createCell(rowKey, Bytes.toBytes("compressiontestval"));
123    writer.append(c);
124    writer.appendFileInfo(Bytes.toBytes("compressioninfokey"), Bytes.toBytes("compressioninfoval"));
125    writer.close();
126    Cell cc = null;
127    HFile.Reader reader = HFile.createReader(fs, path, CacheConfig.DISABLED, true, conf);
128    try {
129      HFileScanner scanner = reader.getScanner(conf, false, true);
130      scanner.seekTo(); // position to the start of file
131      // Scanner does not do Cells yet. Do below for now till fixed.
132      cc = scanner.getCell();
133      if (CellComparator.getInstance().compareRows(c, cc) != 0) {
134        throw new Exception("Read back incorrect result: " + c.toString() + " vs " + cc.toString());
135      }
136    } finally {
137      reader.close();
138    }
139  }
140
141  public static void main(String[] args) throws Exception {
142    if (args.length != 2) {
143      usage();
144      System.exit(1);
145    }
146
147    Configuration conf = new Configuration();
148    Path path = new Path(args[0]);
149    FileSystem fs = path.getFileSystem(conf);
150    if (fs.exists(path)) {
151      System.err.println("The specified path exists, aborting!");
152      System.exit(1);
153    }
154
155    try {
156      doSmokeTest(fs, path, args[1]);
157    } finally {
158      fs.delete(path, false);
159    }
160    System.out.println("SUCCESS");
161  }
162}