/** * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with this * work for additional information regarding copyright ownership. The ASF * licenses this file to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance with the License. * You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the * License for the specific language governing permissions and limitations under * the License. */ package org.apache.hadoop.io.file.tfile; import java.io.BufferedInputStream; import java.io.BufferedOutputStream; import java.io.FilterOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.ArrayList; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.compress.CodecPool; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.CompressionInputStream; import org.apache.hadoop.io.compress.CompressionOutputStream; import org.apache.hadoop.io.compress.Compressor; import org.apache.hadoop.io.compress.Decompressor; import org.apache.hadoop.io.compress.DefaultCodec; import org.apache.hadoop.io.compress.LzmaCodec; import org.apache.hadoop.util.ReflectionUtils; /** * Compression related stuff. */ final class Compression { static final Log LOG = LogFactory.getLog(Compression.class); /** * Prevent the instantiation of class. */ private Compression() { // nothing } static class FinishOnFlushCompressionStream extends FilterOutputStream { public FinishOnFlushCompressionStream(CompressionOutputStream cout) { super(cout); } @Override public void write(byte b[], int off, int len) throws IOException { out.write(b, off, len); } @Override public void flush() throws IOException { CompressionOutputStream cout = (CompressionOutputStream) out; cout.finish(); cout.flush(); cout.resetState(); } } /** * Compression algorithms. */ static enum Algorithm { LZO(TFile.COMPRESSION_LZO) { private transient boolean checked = false; private static final String defaultClazz = "org.apache.hadoop.io.compress.LzoCodec"; private transient CompressionCodec codec = null; @Override public synchronized boolean isSupported() { if (!checked) { checked = true; String extClazz = (conf.get(CONF_LZO_CLASS) == null ? System .getProperty(CONF_LZO_CLASS) : null); String clazz = (extClazz != null) ? extClazz : defaultClazz; try { LOG.info("Trying to load Lzo codec class: " + clazz); codec = (CompressionCodec) ReflectionUtils.newInstance(Class .forName(clazz), conf); } catch (ClassNotFoundException e) { // that is okay } } return codec != null; } @Override CompressionCodec getCodec() throws IOException { if (!isSupported()) { throw new IOException( "LZO codec class not specified. Did you forget to set property " + CONF_LZO_CLASS + "?"); } return codec; } @Override public synchronized InputStream createDecompressionStream( InputStream downStream, Decompressor decompressor, int downStreamBufferSize) throws IOException { if (!isSupported()) { throw new IOException( "LZO codec class not specified. Did you forget to set property " + CONF_LZO_CLASS + "?"); } InputStream bis1 = null; if (downStreamBufferSize > 0) { bis1 = new BufferedInputStream(downStream, downStreamBufferSize); } else { bis1 = downStream; } conf.setInt("io.compression.codec.lzo.buffersize", 64 * 1024); CompressionInputStream cis = codec.createInputStream(bis1, decompressor); BufferedInputStream bis2 = new BufferedInputStream(cis, DATA_IBUF_SIZE); return bis2; } @Override public synchronized OutputStream createCompressionStream( OutputStream downStream, Compressor compressor, int downStreamBufferSize) throws IOException { if (!isSupported()) { throw new IOException( "LZO codec class not specified. Did you forget to set property " + CONF_LZO_CLASS + "?"); } OutputStream bos1 = null; if (downStreamBufferSize > 0) { bos1 = new BufferedOutputStream(downStream, downStreamBufferSize); } else { bos1 = downStream; } conf.setInt("io.compression.codec.lzo.buffersize", 64 * 1024); CompressionOutputStream cos = codec.createOutputStream(bos1, compressor); BufferedOutputStream bos2 = new BufferedOutputStream(new FinishOnFlushCompressionStream(cos), DATA_OBUF_SIZE); return bos2; } }, LZMA(TFile.COMPRESSION_LZMA) { private transient LzmaCodec codec = null; private transient boolean checked = false; @Override public synchronized boolean isSupported() { if (!checked) { checked = true; if (LzmaCodec.isNativeLzmaLoaded(conf)) { codec = new LzmaCodec(); codec.setConf(conf); } } return codec != null; } @Override CompressionCodec getCodec() throws IOException { if (!isSupported()) { throw new IOException( "LZMA codec cannot be loaded. " + "You may want to check LD_LIBRARY_PATH."); } return codec; } @Override public synchronized InputStream createDecompressionStream( InputStream downStream, Decompressor decompressor, int downStreamBufferSize) throws IOException { if (!isSupported()) { throw new IOException( "LZMA codec cannot be loaded. " + "You may want to check LD_LIBRARY_PATH."); } InputStream bis1 = null; if (downStreamBufferSize > 0) { bis1 = new BufferedInputStream(downStream, downStreamBufferSize); } else { bis1 = downStream; } conf.setInt("io.compression.codec.lzma.buffersize&