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 edu.umd.cs.findbugs.annotations.Nullable;
021import java.io.BufferedInputStream;
022import java.io.BufferedOutputStream;
023import java.io.Closeable;
024import java.io.FilterOutputStream;
025import java.io.IOException;
026import java.io.InputStream;
027import java.io.OutputStream;
028import org.apache.hadoop.conf.Configurable;
029import org.apache.hadoop.conf.Configuration;
030import org.apache.hadoop.hbase.HBaseConfiguration;
031import org.apache.hadoop.hbase.io.HeapSize;
032import org.apache.hadoop.hbase.nio.ByteBuff;
033import org.apache.hadoop.io.compress.CompressionCodec;
034import org.apache.hadoop.io.compress.CompressionInputStream;
035import org.apache.hadoop.io.compress.CompressionOutputStream;
036import org.apache.hadoop.io.compress.Compressor;
037import org.apache.hadoop.io.compress.Decompressor;
038import org.apache.hadoop.io.compress.DoNotPool;
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  // LZO
052
053  public static final String LZO_CODEC_CLASS_KEY = "hbase.io.compress.lzo.codec";
054  public static final String LZO_CODEC_CLASS_DEFAULT = "com.hadoop.compression.lzo.LzoCodec";
055
056  // GZ
057
058  public static final String GZ_CODEC_CLASS_KEY = "hbase.io.compress.gz.codec";
059  // Our ReusableStreamGzipCodec fixes an inefficiency in Hadoop's Gzip codec, allowing us to
060  // reuse compression streams, but still requires the Hadoop native codec.
061  public static final String GZ_CODEC_CLASS_DEFAULT =
062    "org.apache.hadoop.hbase.io.compress.ReusableStreamGzipCodec";
063
064  // SNAPPY
065
066  public static final String SNAPPY_CODEC_CLASS_KEY = "hbase.io.compress.snappy.codec";
067  public static final String SNAPPY_CODEC_CLASS_DEFAULT =
068    "org.apache.hadoop.io.compress.SnappyCodec";
069
070  // LZ4
071
072  public static final String LZ4_CODEC_CLASS_KEY = "hbase.io.compress.lz4.codec";
073  public static final String LZ4_CODEC_CLASS_DEFAULT = "org.apache.hadoop.io.compress.Lz4Codec";
074
075  // ZSTD
076
077  public static final String ZSTD_CODEC_CLASS_KEY = "hbase.io.compress.zstd.codec";
078  public static final String ZSTD_CODEC_CLASS_DEFAULT =
079    "org.apache.hadoop.io.compress.ZStandardCodec";
080
081  // BZIP2
082
083  public static final String BZIP2_CODEC_CLASS_KEY = "hbase.io.compress.bzip2.codec";
084  public static final String BZIP2_CODEC_CLASS_DEFAULT = "org.apache.hadoop.io.compress.BZip2Codec";
085
086  // LZMA
087
088  /** @deprecated Deprecated in 2.5 and removed in 2.6 and up. See HBASE-28506. **/
089  @Deprecated
090  public static final String LZMA_CODEC_CLASS_KEY = "hbase.io.compress.lzma.codec";
091  /** @deprecated Deprecated in 2.5 and removed in 2.6 and up. See HBASE-28506. **/
092  @Deprecated
093  public static final String LZMA_CODEC_CLASS_DEFAULT =
094    "org.apache.hadoop.hbase.io.compress.xz.LzmaCodec";
095
096  // Brotli
097
098  public static final String BROTLI_CODEC_CLASS_KEY = "hbase.io.compress.brotli.codec";
099  public static final String BROTLI_CODEC_CLASS_DEFAULT =
100    "org.apache.hadoop.hbase.io.compress.brotli.BrotliCodec";
101
102  /**
103   * Prevent the instantiation of class.
104   */
105  private Compression() {
106    super();
107  }
108
109  static class FinishOnFlushCompressionStream extends FilterOutputStream {
110    public FinishOnFlushCompressionStream(CompressionOutputStream cout) {
111      super(cout);
112    }
113
114    @Override
115    public void write(byte b[], int off, int len) throws IOException {
116      out.write(b, off, len);
117    }
118
119    @Override
120    public void flush() throws IOException {
121      CompressionOutputStream cout = (CompressionOutputStream) out;
122      cout.finish();
123      cout.flush();
124      cout.resetState();
125    }
126  }
127
128  /**
129   * Returns the classloader to load the Codec class from.
130   */
131  private static ClassLoader getClassLoaderForCodec() {
132    ClassLoader cl = Thread.currentThread().getContextClassLoader();
133    if (cl == null) {
134      cl = Compression.class.getClassLoader();
135    }
136    if (cl == null) {
137      cl = ClassLoader.getSystemClassLoader();
138    }
139    if (cl == null) {
140      throw new RuntimeException("A ClassLoader to load the Codec could not be determined");
141    }
142    return cl;
143  }
144
145  /**
146   * Compression algorithms. The ordinal of these cannot change or else you risk breaking all
147   * existing HFiles out there. Even the ones that are not compressed! (They use the NONE algorithm)
148   */
149  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "SE_TRANSIENT_FIELD_NOT_RESTORED",
150      justification = "We are not serializing so doesn't apply (not sure why transient though)")
151  @SuppressWarnings("ImmutableEnumChecker")
152  @InterfaceAudience.Public
153  public static enum Algorithm {
154    // LZO is GPL and requires extra install to setup. See
155    // https://stackoverflow.com/questions/23441142/class-com-hadoop-compression-lzo-lzocodec-not-found-for-spark-on-cdh-5
156    LZO("lzo", LZO_CODEC_CLASS_KEY, LZO_CODEC_CLASS_DEFAULT) {
157      // Use base type to avoid compile-time dependencies.
158      private volatile transient CompressionCodec lzoCodec;
159      private final transient Object lock = new Object();
160
161      @Override
162      CompressionCodec getCodec(Configuration conf) {
163        if (lzoCodec == null) {
164          synchronized (lock) {
165            if (lzoCodec == null) {
166              lzoCodec = buildCodec(conf, this);
167            }
168          }
169        }
170        return lzoCodec;
171      }
172
173      @Override
174      public CompressionCodec reload(Configuration conf) {
175        synchronized (lock) {
176          lzoCodec = buildCodec(conf, this);
177          LOG.warn("Reloaded configuration for {}", name());
178          return lzoCodec;
179        }
180      }
181    },
182
183    GZ("gz", GZ_CODEC_CLASS_KEY, GZ_CODEC_CLASS_DEFAULT) {
184      private volatile transient CompressionCodec gzCodec;
185      private final transient Object lock = new Object();
186
187      @Override
188      CompressionCodec getCodec(Configuration conf) {
189        if (gzCodec == null) {
190          synchronized (lock) {
191            if (gzCodec == null) {
192              gzCodec = buildCodec(conf, this);
193            }
194          }
195        }
196        return gzCodec;
197      }
198
199      @Override
200      public CompressionCodec reload(Configuration conf) {
201        synchronized (lock) {
202          gzCodec = buildCodec(conf, this);
203          LOG.warn("Reloaded configuration for {}", name());
204          return gzCodec;
205        }
206      }
207    },
208
209    NONE("none", "", "") {
210      @Override
211      CompressionCodec getCodec(Configuration conf) {
212        return null;
213      }
214
215      @Override
216      public CompressionCodec reload(Configuration conf) {
217        return null;
218      }
219
220      @Override
221      public synchronized InputStream createDecompressionStream(InputStream downStream,
222        Decompressor decompressor, int downStreamBufferSize) throws IOException {
223        if (downStreamBufferSize > 0) {
224          return new BufferedInputStream(downStream, downStreamBufferSize);
225        }
226        return downStream;
227      }
228
229      @Override
230      public synchronized OutputStream createCompressionStream(OutputStream downStream,
231        Compressor compressor, int downStreamBufferSize) throws IOException {
232        if (downStreamBufferSize > 0) {
233          return new BufferedOutputStream(downStream, downStreamBufferSize);
234        }
235
236        return downStream;
237      }
238    },
239    SNAPPY("snappy", SNAPPY_CODEC_CLASS_KEY, SNAPPY_CODEC_CLASS_DEFAULT) {
240      // Use base type to avoid compile-time dependencies.
241      private volatile transient CompressionCodec snappyCodec;
242      private final transient Object lock = new Object();
243
244      @Override
245      CompressionCodec getCodec(Configuration conf) {
246        if (snappyCodec == null) {
247          synchronized (lock) {
248            if (snappyCodec == null) {
249              snappyCodec = buildCodec(conf, this);
250            }
251          }
252        }
253        return snappyCodec;
254      }
255
256      @Override
257      public CompressionCodec reload(Configuration conf) {
258        synchronized (lock) {
259          snappyCodec = buildCodec(conf, this);
260          LOG.warn("Reloaded configuration for {}", name());
261          return snappyCodec;
262        }
263      }
264    },
265    LZ4("lz4", LZ4_CODEC_CLASS_KEY, LZ4_CODEC_CLASS_DEFAULT) {
266      // Use base type to avoid compile-time dependencies.
267      private volatile transient CompressionCodec lz4Codec;
268      private final transient Object lock = new Object();
269
270      @Override
271      CompressionCodec getCodec(Configuration conf) {
272        if (lz4Codec == null) {
273          synchronized (lock) {
274            if (lz4Codec == null) {
275              lz4Codec = buildCodec(conf, this);
276            }
277          }
278        }
279        return lz4Codec;
280      }
281
282      @Override
283      public CompressionCodec reload(Configuration conf) {
284        synchronized (lock) {
285          lz4Codec = buildCodec(conf, this);
286          LOG.warn("Reloaded configuration for {}", name());
287          return lz4Codec;
288        }
289      }
290    },
291    BZIP2("bzip2", BZIP2_CODEC_CLASS_KEY, BZIP2_CODEC_CLASS_DEFAULT) {
292      // Use base type to avoid compile-time dependencies.
293      private volatile transient CompressionCodec bzipCodec;
294      private final transient Object lock = new Object();
295
296      @Override
297      CompressionCodec getCodec(Configuration conf) {
298        if (bzipCodec == null) {
299          synchronized (lock) {
300            if (bzipCodec == null) {
301              bzipCodec = buildCodec(conf, this);
302            }
303          }
304        }
305        return bzipCodec;
306      }
307
308      @Override
309      public CompressionCodec reload(Configuration conf) {
310        synchronized (lock) {
311          bzipCodec = buildCodec(conf, this);
312          LOG.warn("Reloaded configuration for {}", name());
313          return bzipCodec;
314        }
315      }
316    },
317    ZSTD("zstd", ZSTD_CODEC_CLASS_KEY, ZSTD_CODEC_CLASS_DEFAULT) {
318      // Use base type to avoid compile-time dependencies.
319      private volatile transient CompressionCodec zStandardCodec;
320      private final transient Object lock = new Object();
321
322      @Override
323      CompressionCodec getCodec(Configuration conf) {
324        if (zStandardCodec == null) {
325          synchronized (lock) {
326            if (zStandardCodec == null) {
327              zStandardCodec = buildCodec(conf, this);
328            }
329          }
330        }
331        return zStandardCodec;
332      }
333
334      @Override
335      public CompressionCodec reload(Configuration conf) {
336        synchronized (lock) {
337          zStandardCodec = buildCodec(conf, this);
338          LOG.warn("Reloaded configuration for {}", name());
339          return zStandardCodec;
340        }
341      }
342    },
343    LZMA("lzma", LZMA_CODEC_CLASS_KEY, LZMA_CODEC_CLASS_DEFAULT) {
344      // Use base type to avoid compile-time dependencies.
345      private volatile transient CompressionCodec lzmaCodec;
346      private final transient Object lock = new Object();
347
348      @Override
349      CompressionCodec getCodec(Configuration conf) {
350        if (lzmaCodec == null) {
351          synchronized (lock) {
352            if (lzmaCodec == null) {
353              lzmaCodec = buildCodec(conf, this);
354            }
355          }
356        }
357        return lzmaCodec;
358      }
359
360      @Override
361      public CompressionCodec reload(Configuration conf) {
362        synchronized (lock) {
363          lzmaCodec = buildCodec(conf, this);
364          LOG.warn("Reloaded configuration for {}", name());
365          return lzmaCodec;
366        }
367      }
368    },
369
370    BROTLI("brotli", BROTLI_CODEC_CLASS_KEY, BROTLI_CODEC_CLASS_DEFAULT) {
371      // Use base type to avoid compile-time dependencies.
372      private volatile transient CompressionCodec brotliCodec;
373      private final transient Object lock = new Object();
374
375      @Override
376      CompressionCodec getCodec(Configuration conf) {
377        if (brotliCodec == null) {
378          synchronized (lock) {
379            if (brotliCodec == null) {
380              brotliCodec = buildCodec(conf, this);
381            }
382          }
383        }
384        return brotliCodec;
385      }
386
387      @Override
388      public CompressionCodec reload(Configuration conf) {
389        synchronized (lock) {
390          brotliCodec = buildCodec(conf, this);
391          LOG.warn("Reloaded configuration for {}", name());
392          return brotliCodec;
393        }
394      }
395    };
396
397    private final Configuration conf;
398    private final String compressName;
399    private final String confKey;
400    private final String confDefault;
401    /** data input buffer size to absorb small reads from application. */
402    private static final int DATA_IBUF_SIZE = 1 * 1024;
403    /** data output buffer size to absorb small writes from application. */
404    private static final int DATA_OBUF_SIZE = 4 * 1024;
405
406    Algorithm(String name, String confKey, String confDefault) {
407      this.conf = HBaseConfiguration.create();
408      this.conf.setBoolean("io.native.lib.available", true);
409      this.compressName = name;
410      this.confKey = confKey;
411      this.confDefault = confDefault;
412    }
413
414    abstract CompressionCodec getCodec(Configuration conf);
415
416    /**
417     * Reload configuration for the given algorithm.
418     * <p>
419     * NOTE: Experts only. This can only be done safely during process startup, before the
420     * algorithm's codecs are in use. If the codec implementation is changed, the new implementation
421     * may not be fully compatible with what was loaded at static initialization time, leading to
422     * potential data corruption. Mostly used by unit tests.
423     * @param conf configuration
424     */
425    public abstract CompressionCodec reload(Configuration conf);
426
427    public InputStream createDecompressionStream(InputStream downStream, Decompressor decompressor,
428      int downStreamBufferSize) throws IOException {
429      CompressionCodec codec = getCodec(conf);
430      // Set the internal buffer size to read from down stream.
431      if (downStreamBufferSize > 0) {
432        ((Configurable) codec).getConf().setInt("io.file.buffer.size", downStreamBufferSize);
433      }
434      CompressionInputStream cis = codec.createInputStream(downStream, decompressor);
435      BufferedInputStream bis2 = new BufferedInputStream(cis, DATA_IBUF_SIZE);
436      return bis2;
437
438    }
439
440    public OutputStream createCompressionStream(OutputStream downStream, Compressor compressor,
441      int downStreamBufferSize) throws IOException {
442      OutputStream bos1 = null;
443      if (downStreamBufferSize > 0) {
444        bos1 = new BufferedOutputStream(downStream, downStreamBufferSize);
445      } else {
446        bos1 = downStream;
447      }
448      CompressionOutputStream cos = createPlainCompressionStream(bos1, compressor);
449      BufferedOutputStream bos2 =
450        new BufferedOutputStream(new FinishOnFlushCompressionStream(cos), DATA_OBUF_SIZE);
451      return bos2;
452    }
453
454    /**
455     * Creates a compression stream without any additional wrapping into buffering streams.
456     */
457    public CompressionOutputStream createPlainCompressionStream(OutputStream downStream,
458      Compressor compressor) throws IOException {
459      CompressionCodec codec = getCodec(conf);
460      ((Configurable) codec).getConf().setInt("io.file.buffer.size", 32 * 1024);
461      return codec.createOutputStream(downStream, compressor);
462    }
463
464    public Compressor getCompressor() {
465      CompressionCodec codec = getCodec(conf);
466      if (codec != null) {
467        Compressor compressor = CodecPool.getCompressor(codec);
468        if (LOG.isTraceEnabled()) LOG.trace("Retrieved compressor " + compressor + " from pool.");
469        if (compressor != null) {
470          if (compressor.finished()) {
471            // Somebody returns the compressor to CodecPool but is still using it.
472            LOG.warn("Compressor obtained from CodecPool is already finished()");
473          }
474          compressor.reset();
475        }
476        return compressor;
477      }
478      return null;
479    }
480
481    public void returnCompressor(Compressor compressor) {
482      if (compressor != null) {
483        if (LOG.isTraceEnabled()) LOG.trace("Returning compressor " + compressor + " to pool.");
484        CodecPool.returnCompressor(compressor);
485      }
486    }
487
488    public Decompressor getDecompressor() {
489      CompressionCodec codec = getCodec(conf);
490      if (codec != null) {
491        Decompressor decompressor = CodecPool.getDecompressor(codec);
492        if (LOG.isTraceEnabled())
493          LOG.trace("Retrieved decompressor " + decompressor + " from pool.");
494        if (decompressor != null) {
495          if (decompressor.finished()) {
496            // Somebody returns the decompressor to CodecPool but is still using it.
497            LOG.warn("Decompressor {} obtained from CodecPool is already finished", decompressor);
498          }
499          decompressor.reset();
500        }
501        return decompressor;
502      }
503
504      return null;
505    }
506
507    public void returnDecompressor(Decompressor decompressor) {
508      if (decompressor != null) {
509        if (LOG.isTraceEnabled()) LOG.trace("Returning decompressor " + decompressor + " to pool.");
510        CodecPool.returnDecompressor(decompressor);
511        if (decompressor.getClass().isAnnotationPresent(DoNotPool.class)) {
512          if (LOG.isTraceEnabled()) LOG.trace("Ending decompressor " + decompressor);
513          decompressor.end();
514        }
515      }
516    }
517
518    /**
519     * Signals if this codec theoretically supports decompression on {@link ByteBuff}s. This can be
520     * faster than using a DecompressionStream. If this method returns true, you can call
521     * {@link #getByteBuffDecompressor()} to obtain a {@link ByteBuffDecompressor}. You must then
522     * also call {@link ByteBuffDecompressor#canDecompress(ByteBuff, ByteBuff)} before attempting
523     * decompression, to verify if that decompressor is capable of handling your particular input
524     * and output buffers.
525     */
526    public boolean supportsByteBuffDecompression() {
527      CompressionCodec codec = getCodec(conf);
528      return codec instanceof ByteBuffDecompressionCodec;
529    }
530
531    /**
532     * Be sure to call {@link #supportsByteBuffDecompression()} before calling this method.
533     * @throws IllegalStateException if the codec does not support block decompression
534     */
535    public ByteBuffDecompressor getByteBuffDecompressor() {
536      CompressionCodec codec = getCodec(conf);
537      if (codec instanceof ByteBuffDecompressionCodec) {
538        ByteBuffDecompressor decompressor =
539          CodecPool.getByteBuffDecompressor((ByteBuffDecompressionCodec) codec);
540        if (LOG.isTraceEnabled()) {
541          LOG.trace("Retrieved decompressor {} from pool.", decompressor);
542        }
543        return decompressor;
544      } else {
545        throw new IllegalStateException("Codec " + codec + " does not support block decompression");
546      }
547    }
548
549    public void returnByteBuffDecompressor(ByteBuffDecompressor decompressor) {
550      if (decompressor != null) {
551        if (LOG.isTraceEnabled()) {
552          LOG.trace("Returning decompressor {} to pool.", decompressor);
553        }
554        CodecPool.returnByteBuffDecompressor(decompressor);
555      }
556    }
557
558    /**
559     * Get an object that holds settings used by ByteBuffDecompressor. It's expensive to pull these
560     * from a Configuration object every time we decompress a block, so pull them here when, for
561     * example, opening an HFile, and reuse the returned HFileDecompressionContext as much as
562     * possible. The concrete class of this object will be one that is specific to the codec
563     * implementation in use. You don't need to inspect it yourself, just pass it along to
564     * {@link ByteBuffDecompressor#reinit(HFileDecompressionContext)}.
565     */
566    @Nullable
567    public HFileDecompressionContext
568      getHFileDecompressionContextForConfiguration(Configuration conf) {
569      if (supportsByteBuffDecompression()) {
570        return ((ByteBuffDecompressionCodec) getCodec(conf))
571          .getDecompressionContextFromConfiguration(conf);
572      } else {
573        return null;
574      }
575    }
576
577    public String getName() {
578      return compressName;
579    }
580  }
581
582  /**
583   * See {@link Algorithm#getHFileDecompressionContextForConfiguration(Configuration)}.
584   */
585  public static abstract class HFileDecompressionContext implements Closeable, HeapSize {
586  }
587
588  public static Algorithm getCompressionAlgorithmByName(String compressName) {
589    Algorithm[] algos = Algorithm.class.getEnumConstants();
590
591    for (Algorithm a : algos) {
592      if (a.getName().equals(compressName)) {
593        return a;
594      }
595    }
596
597    throw new IllegalArgumentException("Unsupported compression algorithm name: " + compressName);
598  }
599
600  /**
601   * Get names of supported compression algorithms.
602   * @return Array of strings, each represents a supported compression algorithm. Currently, the
603   *         following compression algorithms are supported.
604   */
605  public static String[] getSupportedAlgorithms() {
606    Algorithm[] algos = Algorithm.class.getEnumConstants();
607
608    String[] ret = new String[algos.length];
609    int i = 0;
610    for (Algorithm a : algos) {
611      ret[i++] = a.getName();
612    }
613
614    return ret;
615  }
616
617  /**
618   * Load a codec implementation for an algorithm using the supplied configuration.
619   * @param conf the configuration to use
620   * @param algo the algorithm to implement
621   */
622  private static CompressionCodec buildCodec(final Configuration conf, final Algorithm algo) {
623    try {
624      String codecClassName = conf.get(algo.confKey, algo.confDefault);
625      if (codecClassName == null) {
626        throw new RuntimeException("No codec configured for " + algo.confKey);
627      }
628      Class<?> codecClass = getClassLoaderForCodec().loadClass(codecClassName);
629      // The class is from hadoop so we use hadoop's ReflectionUtils to create it
630      CompressionCodec codec =
631        (CompressionCodec) ReflectionUtils.newInstance(codecClass, new Configuration(conf));
632      LOG.info("Loaded codec {} for compression algorithm {}", codec.getClass().getCanonicalName(),
633        algo.name());
634      return codec;
635    } catch (ClassNotFoundException e) {
636      throw new RuntimeException(e);
637    }
638  }
639
640  public static void main(String[] args) throws Exception {
641    Configuration conf = HBaseConfiguration.create();
642    java.util.Map<String, CompressionCodec> implMap = new java.util.HashMap<>();
643    for (Algorithm algo : Algorithm.class.getEnumConstants()) {
644      try {
645        implMap.put(algo.name(), algo.getCodec(conf));
646      } catch (Exception e) {
647        // Ignore failures to load codec native implementations while building the report.
648        // We are to report what is configured.
649      }
650    }
651    for (Algorithm algo : Algorithm.class.getEnumConstants()) {
652      System.out.println(algo.name() + ":");
653      System.out.println("    name: " + algo.getName());
654      System.out.println("    confKey: " + algo.confKey);
655      System.out.println("    confDefault: " + algo.confDefault);
656      CompressionCodec codec = implMap.get(algo.name());
657      System.out.println(
658        "    implClass: " + (codec != null ? codec.getClass().getCanonicalName() : "<none>"));
659    }
660  }
661
662}