View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.util;
20  
21  import java.io.IOException;
22  
23  import org.apache.commons.lang.StringUtils;
24  import org.apache.commons.logging.Log;
25  import org.apache.commons.logging.LogFactory;
26  import org.apache.hadoop.hbase.classification.InterfaceAudience;
27  import org.apache.hadoop.hbase.classification.InterfaceStability;
28  import org.apache.hadoop.conf.Configuration;
29  import org.apache.hadoop.fs.FileSystem;
30  import org.apache.hadoop.fs.Path;
31  import org.apache.hadoop.hbase.Cell;
32  import org.apache.hadoop.hbase.CellComparator;
33  import org.apache.hadoop.hbase.CellUtil;
34  import org.apache.hadoop.hbase.DoNotRetryIOException;
35  import org.apache.hadoop.hbase.HBaseConfiguration;
36  import org.apache.hadoop.hbase.HBaseInterfaceAudience;
37  import org.apache.hadoop.hbase.io.compress.Compression;
38  import org.apache.hadoop.hbase.io.hfile.AbstractHFileWriter;
39  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
40  import org.apache.hadoop.hbase.io.hfile.HFile;
41  import org.apache.hadoop.hbase.io.hfile.HFileContext;
42  import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
43  import org.apache.hadoop.hbase.io.hfile.HFileScanner;
44  import org.apache.hadoop.io.compress.Compressor;
45  
46  /**
47   * Compression validation test.  Checks compression is working.  Be sure to run
48   * on every node in your cluster.
49   */
50  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
51  @InterfaceStability.Evolving
52  public class CompressionTest {
53    private static final Log LOG = LogFactory.getLog(CompressionTest.class);
54  
55    public static boolean testCompression(String codec) {
56      codec = codec.toLowerCase();
57  
58      Compression.Algorithm a;
59  
60      try {
61        a = Compression.getCompressionAlgorithmByName(codec);
62      } catch (IllegalArgumentException e) {
63        LOG.warn("Codec type: " + codec + " is not known");
64        return false;
65      }
66  
67      try {
68        testCompression(a);
69        return true;
70      } catch (IOException ignored) {
71        LOG.warn("Can't instantiate codec: " + codec, ignored);
72        return false;
73      }
74    }
75  
76    private final static Boolean[] compressionTestResults
77        = new Boolean[Compression.Algorithm.values().length];
78    static {
79      for (int i = 0 ; i < compressionTestResults.length ; ++i) {
80        compressionTestResults[i] = null;
81      }
82    }
83  
84    public static void testCompression(Compression.Algorithm algo)
85        throws IOException {
86      if (compressionTestResults[algo.ordinal()] != null) {
87        if (compressionTestResults[algo.ordinal()]) {
88          return ; // already passed test, dont do it again.
89        } else {
90          // failed.
91          throw new DoNotRetryIOException("Compression algorithm '" + algo.getName() + "'" +
92          " previously failed test.");
93        }
94      }
95  
96      try {
97        Compressor c = algo.getCompressor();
98        algo.returnCompressor(c);
99        compressionTestResults[algo.ordinal()] = true; // passes
100     } catch (Throwable t) {
101       compressionTestResults[algo.ordinal()] = false; // failure
102       throw new DoNotRetryIOException(t);
103     }
104   }
105 
106   protected static Path path = new Path(".hfile-comp-test");
107 
108   public static void usage() {
109 
110     System.err.println(
111       "Usage: CompressionTest <path> " +
112       StringUtils.join( Compression.Algorithm.values(), "|").toLowerCase() +
113       "\n" +
114       "For example:\n" +
115       "  hbase " + CompressionTest.class + " file:///tmp/testfile gz\n");
116     System.exit(1);
117   }
118 
119   public static void doSmokeTest(FileSystem fs, Path path, String codec)
120   throws Exception {
121     Configuration conf = HBaseConfiguration.create();
122     HFileContext context = new HFileContextBuilder()
123                            .withCompression(AbstractHFileWriter.compressionByName(codec)).build();
124     HFile.Writer writer = HFile.getWriterFactoryNoCache(conf)
125         .withPath(fs, path)
126         .withFileContext(context)
127         .create();
128     // Write any-old Cell...
129     final byte [] rowKey = Bytes.toBytes("compressiontestkey");
130     Cell c = CellUtil.createCell(rowKey, Bytes.toBytes("compressiontestval"));
131     writer.append(c);
132     writer.appendFileInfo(Bytes.toBytes("compressioninfokey"), Bytes.toBytes("compressioninfoval"));
133     writer.close();
134     Cell cc = null;
135     HFile.Reader reader = HFile.createReader(fs, path, new CacheConfig(conf), conf);
136     try {
137       reader.loadFileInfo();
138       HFileScanner scanner = reader.getScanner(false, true);
139       scanner.seekTo(); // position to the start of file
140       // Scanner does not do Cells yet. Do below for now till fixed.
141       cc = scanner.getKeyValue();
142       if (CellComparator.compareRows(c, cc) != 0) {
143         throw new Exception("Read back incorrect result: " + c.toString() + " vs " + cc.toString());
144       }
145     } finally {
146       reader.close();
147     }
148   }
149 
150   public static void main(String[] args) throws Exception {
151     if (args.length != 2) {
152       usage();
153       System.exit(1);
154     }
155 
156     Configuration conf = new Configuration();
157     Path path = new Path(args[0]);
158     FileSystem fs = path.getFileSystem(conf);
159     try {
160       doSmokeTest(fs, path, args[1]);
161     } finally {
162       fs.delete(path, false);
163     }
164     System.out.println("SUCCESS");
165   }
166 }