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