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.io.compress; 019 020import java.io.BufferedInputStream; 021import java.io.BufferedOutputStream; 022import java.io.FilterOutputStream; 023import java.io.IOException; 024import java.io.InputStream; 025import java.io.OutputStream; 026import org.apache.hadoop.conf.Configurable; 027import org.apache.hadoop.conf.Configuration; 028import org.apache.hadoop.hbase.io.util.BlockIOUtils; 029import org.apache.hadoop.hbase.nio.ByteBuff; 030import org.apache.hadoop.io.compress.CodecPool; 031import org.apache.hadoop.io.compress.CompressionCodec; 032import org.apache.hadoop.io.compress.CompressionInputStream; 033import org.apache.hadoop.io.compress.CompressionOutputStream; 034import org.apache.hadoop.io.compress.Compressor; 035import org.apache.hadoop.io.compress.Decompressor; 036import org.apache.hadoop.io.compress.DefaultCodec; 037import org.apache.hadoop.io.compress.DoNotPool; 038import org.apache.hadoop.io.compress.GzipCodec; 039import org.apache.hadoop.util.ReflectionUtils; 040import org.apache.yetus.audience.InterfaceAudience; 041import org.slf4j.Logger; 042import org.slf4j.LoggerFactory; 043 044/** 045 * Compression related stuff. Copied from hadoop-3315 tfile. 046 */ 047@InterfaceAudience.Private 048public final class Compression { 049 private static final Logger LOG = LoggerFactory.getLogger(Compression.class); 050 051 /** 052 * Prevent the instantiation of class. 053 */ 054 private Compression() { 055 super(); 056 } 057 058 static class FinishOnFlushCompressionStream extends FilterOutputStream { 059 public FinishOnFlushCompressionStream(CompressionOutputStream cout) { 060 super(cout); 061 } 062 063 @Override 064 public void write(byte b[], int off, int len) throws IOException { 065 out.write(b, off, len); 066 } 067 068 @Override 069 public void flush() throws IOException { 070 CompressionOutputStream cout = (CompressionOutputStream) out; 071 cout.finish(); 072 cout.flush(); 073 cout.resetState(); 074 } 075 } 076 077 /** 078 * Returns the classloader to load the Codec class from. 079 */ 080 private static ClassLoader getClassLoaderForCodec() { 081 ClassLoader cl = Thread.currentThread().getContextClassLoader(); 082 if (cl == null) { 083 cl = Compression.class.getClassLoader(); 084 } 085 if (cl == null) { 086 cl = ClassLoader.getSystemClassLoader(); 087 } 088 if (cl == null) { 089 throw new RuntimeException("A ClassLoader to load the Codec could not be determined"); 090 } 091 return cl; 092 } 093 094 /** 095 * Compression algorithms. The ordinal of these cannot change or else you risk breaking all 096 * existing HFiles out there. Even the ones that are not compressed! (They use the NONE algorithm) 097 */ 098 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "SE_TRANSIENT_FIELD_NOT_RESTORED", 099 justification = "We are not serializing so doesn't apply (not sure why transient though)") 100 @InterfaceAudience.Public 101 public static enum Algorithm { 102 // LZO is GPL and requires extra install to setup. See 103 // https://stackoverflow.com/questions/23441142/class-com-hadoop-compression-lzo-lzocodec-not-found-for-spark-on-cdh-5 104 LZO("lzo") { 105 // Use base type to avoid compile-time dependencies. 106 private volatile transient CompressionCodec lzoCodec; 107 private final transient Object lock = new Object(); 108 109 @Override 110 CompressionCodec getCodec(Configuration conf) { 111 if (lzoCodec == null) { 112 synchronized (lock) { 113 if (lzoCodec == null) { 114 lzoCodec = buildCodec(conf); 115 } 116 } 117 } 118 return lzoCodec; 119 } 120 121 private CompressionCodec buildCodec(Configuration conf) { 122 try { 123 Class<?> externalCodec = 124 getClassLoaderForCodec().loadClass("com.hadoop.compression.lzo.LzoCodec"); 125 return (CompressionCodec) ReflectionUtils.newInstance(externalCodec, 126 new Configuration(conf)); 127 } catch (ClassNotFoundException e) { 128 throw new RuntimeException(e); 129 } 130 } 131 }, 132 GZ("gz") { 133 private volatile transient GzipCodec codec; 134 private final transient Object lock = new Object(); 135 136 @Override 137 DefaultCodec getCodec(Configuration conf) { 138 if (codec == null) { 139 synchronized (lock) { 140 if (codec == null) { 141 codec = buildCodec(conf); 142 } 143 } 144 } 145 146 return codec; 147 } 148 149 private GzipCodec buildCodec(Configuration conf) { 150 GzipCodec gzcodec = new ReusableStreamGzipCodec(); 151 gzcodec.setConf(new Configuration(conf)); 152 return gzcodec; 153 } 154 }, 155 156 NONE("none") { 157 @Override 158 DefaultCodec getCodec(Configuration conf) { 159 return null; 160 } 161 162 @Override 163 public synchronized InputStream createDecompressionStream(InputStream downStream, 164 Decompressor decompressor, int downStreamBufferSize) throws IOException { 165 if (downStreamBufferSize > 0) { 166 return new BufferedInputStream(downStream, downStreamBufferSize); 167 } 168 return downStream; 169 } 170 171 @Override 172 public synchronized OutputStream createCompressionStream(OutputStream downStream, 173 Compressor compressor, int downStreamBufferSize) throws IOException { 174 if (downStreamBufferSize > 0) { 175 return new BufferedOutputStream(downStream, downStreamBufferSize); 176 } 177 178 return downStream; 179 } 180 }, 181 SNAPPY("snappy") { 182 // Use base type to avoid compile-time dependencies. 183 private volatile transient CompressionCodec snappyCodec; 184 private final transient Object lock = new Object(); 185 186 @Override 187 CompressionCodec getCodec(Configuration conf) { 188 if (snappyCodec == null) { 189 synchronized (lock) { 190 if (snappyCodec == null) { 191 snappyCodec = buildCodec(conf); 192 } 193 } 194 } 195 return snappyCodec; 196 } 197 198 private CompressionCodec buildCodec(Configuration conf) { 199 try { 200 Class<?> externalCodec = 201 getClassLoaderForCodec().loadClass("org.apache.hadoop.io.compress.SnappyCodec"); 202 return (CompressionCodec) ReflectionUtils.newInstance(externalCodec, conf); 203 } catch (ClassNotFoundException e) { 204 throw new RuntimeException(e); 205 } 206 } 207 }, 208 LZ4("lz4") { 209 // Use base type to avoid compile-time dependencies. 210 private volatile transient CompressionCodec lz4Codec; 211 private final transient Object lock = new Object(); 212 213 @Override 214 CompressionCodec getCodec(Configuration conf) { 215 if (lz4Codec == null) { 216 synchronized (lock) { 217 if (lz4Codec == null) { 218 lz4Codec = buildCodec(conf); 219 } 220 } 221 } 222 return lz4Codec; 223 } 224 225 private CompressionCodec buildCodec(Configuration conf) { 226 try { 227 Class<?> externalCodec = 228 getClassLoaderForCodec().loadClass("org.apache.hadoop.io.compress.Lz4Codec"); 229 return (CompressionCodec) ReflectionUtils.newInstance(externalCodec, conf); 230 } catch (ClassNotFoundException e) { 231 throw new RuntimeException(e); 232 } 233 } 234 }, 235 BZIP2("bzip2") { 236 // Use base type to avoid compile-time dependencies. 237 private volatile transient CompressionCodec bzipCodec; 238 private final transient Object lock = new Object(); 239 240 @Override 241 CompressionCodec getCodec(Configuration conf) { 242 if (bzipCodec == null) { 243 synchronized (lock) { 244 if (bzipCodec == null) { 245 bzipCodec = buildCodec(conf); 246 } 247 } 248 } 249 return bzipCodec; 250 } 251 252 private CompressionCodec buildCodec(Configuration conf) { 253 try { 254 Class<?> externalCodec = 255 getClassLoaderForCodec().loadClass("org.apache.hadoop.io.compress.BZip2Codec"); 256 return (CompressionCodec) ReflectionUtils.newInstance(externalCodec, conf); 257 } catch (ClassNotFoundException e) { 258 throw new RuntimeException(e); 259 } 260 } 261 }, 262 ZSTD("zstd") { 263 // Use base type to avoid compile-time dependencies. 264 private volatile transient CompressionCodec zStandardCodec; 265 private final transient Object lock = new Object(); 266 267 @Override 268 CompressionCodec getCodec(Configuration conf) { 269 if (zStandardCodec == null) { 270 synchronized (lock) { 271 if (zStandardCodec == null) { 272 zStandardCodec = buildCodec(conf); 273 } 274 } 275 } 276 return zStandardCodec; 277 } 278 279 private CompressionCodec buildCodec(Configuration conf) { 280 try { 281 Class<?> externalCodec = 282 getClassLoaderForCodec().loadClass("org.apache.hadoop.io.compress.ZStandardCodec"); 283 return (CompressionCodec) ReflectionUtils.newInstance(externalCodec, conf); 284 } catch (ClassNotFoundException e) { 285 throw new RuntimeException(e); 286 } 287 } 288 }; 289 290 private final Configuration conf; 291 private final String compressName; 292 /** data input buffer size to absorb small reads from application. */ 293 private static final int DATA_IBUF_SIZE = 1 * 1024; 294 /** data output buffer size to absorb small writes from application. */ 295 private static final int DATA_OBUF_SIZE = 4 * 1024; 296 297 Algorithm(String name) { 298 this.conf = new Configuration(); 299 this.conf.setBoolean("io.native.lib.available", true); 300 this.compressName = name; 301 } 302 303 abstract CompressionCodec getCodec(Configuration conf); 304 305 public InputStream createDecompressionStream(InputStream downStream, Decompressor decompressor, 306 int downStreamBufferSize) throws IOException { 307 CompressionCodec codec = getCodec(conf); 308 // Set the internal buffer size to read from down stream. 309 if (downStreamBufferSize > 0) { 310 ((Configurable) codec).getConf().setInt("io.file.buffer.size", downStreamBufferSize); 311 } 312 CompressionInputStream cis = codec.createInputStream(downStream, decompressor); 313 BufferedInputStream bis2 = new BufferedInputStream(cis, DATA_IBUF_SIZE); 314 return bis2; 315 316 } 317 318 public OutputStream createCompressionStream(OutputStream downStream, Compressor compressor, 319 int downStreamBufferSize) throws IOException { 320 OutputStream bos1 = null; 321 if (downStreamBufferSize > 0) { 322 bos1 = new BufferedOutputStream(downStream, downStreamBufferSize); 323 } else { 324 bos1 = downStream; 325 } 326 CompressionOutputStream cos = createPlainCompressionStream(bos1, compressor); 327 BufferedOutputStream bos2 = 328 new BufferedOutputStream(new FinishOnFlushCompressionStream(cos), DATA_OBUF_SIZE); 329 return bos2; 330 } 331 332 /** 333 * Creates a compression stream without any additional wrapping into buffering streams. 334 */ 335 public CompressionOutputStream createPlainCompressionStream(OutputStream downStream, 336 Compressor compressor) throws IOException { 337 CompressionCodec codec = getCodec(conf); 338 ((Configurable) codec).getConf().setInt("io.file.buffer.size", 32 * 1024); 339 return codec.createOutputStream(downStream, compressor); 340 } 341 342 public Compressor getCompressor() { 343 CompressionCodec codec = getCodec(conf); 344 if (codec != null) { 345 Compressor compressor = CodecPool.getCompressor(codec); 346 if (LOG.isTraceEnabled()) LOG.trace("Retrieved compressor " + compressor + " from pool."); 347 if (compressor != null) { 348 if (compressor.finished()) { 349 // Somebody returns the compressor to CodecPool but is still using it. 350 LOG.warn("Compressor obtained from CodecPool is already finished()"); 351 } 352 compressor.reset(); 353 } 354 return compressor; 355 } 356 return null; 357 } 358 359 public void returnCompressor(Compressor compressor) { 360 if (compressor != null) { 361 if (LOG.isTraceEnabled()) LOG.trace("Returning compressor " + compressor + " to pool."); 362 CodecPool.returnCompressor(compressor); 363 } 364 } 365 366 public Decompressor getDecompressor() { 367 CompressionCodec codec = getCodec(conf); 368 if (codec != null) { 369 Decompressor decompressor = CodecPool.getDecompressor(codec); 370 if (LOG.isTraceEnabled()) 371 LOG.trace("Retrieved decompressor " + decompressor + " from pool."); 372 if (decompressor != null) { 373 if (decompressor.finished()) { 374 // Somebody returns the decompressor to CodecPool but is still using it. 375 LOG.warn("Deompressor obtained from CodecPool is already finished()"); 376 } 377 decompressor.reset(); 378 } 379 return decompressor; 380 } 381 382 return null; 383 } 384 385 public void returnDecompressor(Decompressor decompressor) { 386 if (decompressor != null) { 387 if (LOG.isTraceEnabled()) LOG.trace("Returning decompressor " + decompressor + " to pool."); 388 CodecPool.returnDecompressor(decompressor); 389 if (decompressor.getClass().isAnnotationPresent(DoNotPool.class)) { 390 if (LOG.isTraceEnabled()) LOG.trace("Ending decompressor " + decompressor); 391 decompressor.end(); 392 } 393 } 394 } 395 396 public String getName() { 397 return compressName; 398 } 399 } 400 401 public static Algorithm getCompressionAlgorithmByName(String compressName) { 402 Algorithm[] algos = Algorithm.class.getEnumConstants(); 403 404 for (Algorithm a : algos) { 405 if (a.getName().equals(compressName)) { 406 return a; 407 } 408 } 409 410 throw new IllegalArgumentException("Unsupported compression algorithm name: " + compressName); 411 } 412 413 /** 414 * Get names of supported compression algorithms. 415 * @return Array of strings, each represents a supported compression algorithm. Currently, the 416 * following compression algorithms are supported. 417 */ 418 public static String[] getSupportedAlgorithms() { 419 Algorithm[] algos = Algorithm.class.getEnumConstants(); 420 421 String[] ret = new String[algos.length]; 422 int i = 0; 423 for (Algorithm a : algos) { 424 ret[i++] = a.getName(); 425 } 426 427 return ret; 428 } 429 430 /** 431 * Decompresses data from the given stream using the configured compression algorithm. It will 432 * throw an exception if the dest buffer does not have enough space to hold the decompressed data. 433 * @param dest the output buffer 434 * @param bufferedBoundedStream a stream to read compressed data from, bounded to the exact amount 435 * of compressed data 436 * @param uncompressedSize uncompressed data size, header not included 437 * @param compressAlgo compression algorithm used 438 * @throws IOException if any IO error happen 439 */ 440 public static void decompress(ByteBuff dest, InputStream bufferedBoundedStream, 441 int uncompressedSize, Compression.Algorithm compressAlgo) throws IOException { 442 if (dest.remaining() < uncompressedSize) { 443 throw new IllegalArgumentException("Output buffer does not have enough space to hold " 444 + uncompressedSize + " decompressed bytes, available: " + dest.remaining()); 445 } 446 447 Decompressor decompressor = null; 448 try { 449 decompressor = compressAlgo.getDecompressor(); 450 try (InputStream is = 451 compressAlgo.createDecompressionStream(bufferedBoundedStream, decompressor, 0)) { 452 BlockIOUtils.readFullyWithHeapBuffer(is, dest, uncompressedSize); 453 } 454 } finally { 455 if (decompressor != null) { 456 compressAlgo.returnDecompressor(decompressor); 457 } 458 } 459 } 460}