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