001/** 002 * 003 * Licensed to the Apache Software Foundation (ASF) under one 004 * or more contributor license agreements. See the NOTICE file 005 * distributed with this work for additional information 006 * regarding copyright ownership. The ASF licenses this file 007 * to you under the Apache License, Version 2.0 (the 008 * "License"); you may not use this file except in compliance 009 * with the License. You may obtain a copy of the License at 010 * 011 * http://www.apache.org/licenses/LICENSE-2.0 012 * 013 * Unless required by applicable law or agreed to in writing, software 014 * distributed under the License is distributed on an "AS IS" BASIS, 015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 016 * See the License for the specific language governing permissions and 017 * limitations under the License. 018 */ 019package org.apache.hadoop.hbase.util; 020 021import java.io.IOException; 022import java.util.Locale; 023 024import org.apache.commons.lang3.StringUtils; 025import org.apache.hadoop.hbase.CellComparator; 026import org.apache.yetus.audience.InterfaceAudience; 027import org.apache.yetus.audience.InterfaceStability; 028import org.slf4j.Logger; 029import org.slf4j.LoggerFactory; 030import org.apache.hadoop.conf.Configuration; 031import org.apache.hadoop.fs.FileSystem; 032import org.apache.hadoop.fs.Path; 033import org.apache.hadoop.hbase.Cell; 034import org.apache.hadoop.hbase.CellComparatorImpl; 035import org.apache.hadoop.hbase.CellUtil; 036import org.apache.hadoop.hbase.DoNotRetryIOException; 037import org.apache.hadoop.hbase.HBaseConfiguration; 038import org.apache.hadoop.hbase.HBaseInterfaceAudience; 039import org.apache.hadoop.hbase.io.compress.Compression; 040import org.apache.hadoop.hbase.io.hfile.HFileWriterImpl; 041import org.apache.hadoop.hbase.io.hfile.CacheConfig; 042import org.apache.hadoop.hbase.io.hfile.HFile; 043import org.apache.hadoop.hbase.io.hfile.HFileContext; 044import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; 045import org.apache.hadoop.hbase.io.hfile.HFileScanner; 046import org.apache.hadoop.io.compress.Compressor; 047 048/** 049 * Compression validation test. Checks compression is working. Be sure to run 050 * on every node in your 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) 087 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("Compression algorithm '" + algo.getName() + "'" + 094 " 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( 113 "Usage: CompressionTest <path> " + 114 StringUtils.join( Compression.Algorithm.values(), "|").toLowerCase(Locale.ROOT) + 115 "\n" + 116 "For example:\n" + 117 " hbase " + CompressionTest.class + " file:///tmp/testfile gz\n"); 118 System.exit(1); 119 } 120 121 public static void doSmokeTest(FileSystem fs, Path path, String codec) 122 throws Exception { 123 Configuration conf = HBaseConfiguration.create(); 124 HFileContext context = new HFileContextBuilder() 125 .withCompression(HFileWriterImpl.compressionByName(codec)).build(); 126 HFile.Writer writer = HFile.getWriterFactoryNoCache(conf) 127 .withPath(fs, path) 128 .withFileContext(context) 129 .create(); 130 // Write any-old Cell... 131 final byte [] rowKey = Bytes.toBytes("compressiontestkey"); 132 Cell c = CellUtil.createCell(rowKey, Bytes.toBytes("compressiontestval")); 133 writer.append(c); 134 writer.appendFileInfo(Bytes.toBytes("compressioninfokey"), Bytes.toBytes("compressioninfoval")); 135 writer.close(); 136 Cell cc = null; 137 HFile.Reader reader = HFile.createReader(fs, path, CacheConfig.DISABLED, true, conf); 138 try { 139 reader.loadFileInfo(); 140 HFileScanner scanner = reader.getScanner(false, true); 141 scanner.seekTo(); // position to the start of file 142 // Scanner does not do Cells yet. Do below for now till fixed. 143 cc = scanner.getCell(); 144 if (CellComparator.getInstance().compareRows(c, cc) != 0) { 145 throw new Exception("Read back incorrect result: " + c.toString() + " vs " + cc.toString()); 146 } 147 } finally { 148 reader.close(); 149 } 150 } 151 152 public static void main(String[] args) throws Exception { 153 if (args.length != 2) { 154 usage(); 155 System.exit(1); 156 } 157 158 Configuration conf = new Configuration(); 159 Path path = new Path(args[0]); 160 FileSystem fs = path.getFileSystem(conf); 161 if (fs.exists(path)) { 162 System.err.println("The specified path exists, aborting!"); 163 System.exit(1); 164 } 165 166 try { 167 doSmokeTest(fs, path, args[1]); 168 } finally { 169 fs.delete(path, false); 170 } 171 System.out.println("SUCCESS"); 172 } 173}