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