Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 4 additions & 0 deletions hadoop-common-project/hadoop-common/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -362,6 +362,10 @@
<artifactId>wildfly-openssl-java</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>io.airlift</groupId>
<artifactId>aircompressor</artifactId>
</dependency>
</dependencies>

<build>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
/*
* 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 com.hadoop.compression.lzo;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

hmm, why we need this bridging class in Hadoop repo while the class is from hadoop-lzo library?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Historically, Hadoop included LZO until it was removed in HADOOP-4874 due to GPL licensing concern. Then the GPL LZO codec was maintained as a separate project in https://github.com/twitter/hadoop-lzo with new codec class com.hadoop.compression.lzo.LzoCodec. In Hadoop's sequence file, the first couple bytes of file includes the class of the compression codec used when writing, and hadoop uses this information to pick up the right codec to read the data. As a result, we need to bridge it in order to enable Hadoop to read the old LZO compressed data.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@steveloughran this might answer why Not sure why the com.hadoop classes are there at all..

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK. In that case the classes should have the reason explained, but tag the new classes as Deprecated


import java.io.IOException;
import java.io.OutputStream;
import java.util.concurrent.atomic.AtomicBoolean;

import org.apache.hadoop.io.compress.CompressionOutputStream;
import org.apache.hadoop.io.compress.Compressor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* This class creates lzo compressors/decompressors that are bridged
* to `org.apache.hadoop.io.compress.LzoCodec` from `com.hadoop.compression.lzo.LzoCodec`
*/
public class LzoCodec extends org.apache.hadoop.io.compress.LzoCodec {
private static final Logger LOG = LoggerFactory.getLogger(LzoCodec.class.getName());
private static final String gplLzoCodec = LzoCodec.class.getName();
private static final String hadoopLzoCodec = org.apache.hadoop.io.compress.LzoCodec.class.getName();
private static AtomicBoolean warned = new AtomicBoolean(false);

static {
LOG.info("Bridging " + gplLzoCodec + " to " + hadoopLzoCodec + ".");
}

@Override
public CompressionOutputStream createOutputStream(OutputStream out,
Compressor compressor) throws IOException {
if (warned.compareAndSet(false, true)) {
LOG.warn("{} is deprecated. You should use {} instead to generate LZO compressed data.",
gplLzoCodec, hadoopLzoCodec);
}
return super.createOutputStream(out, compressor);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
/*
* 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 com.hadoop.compression.lzo;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

com.hadoop?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this is to offer a bridge for those who are using hadoop-lzo library.


import java.io.IOException;
import java.io.OutputStream;
import java.util.concurrent.atomic.AtomicBoolean;

import org.apache.hadoop.io.compress.CompressionOutputStream;
import org.apache.hadoop.io.compress.Compressor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* This class creates lzop compressors/decompressors that are bridged
* to `org.apache.hadoop.io.compress.LzopCodec` from `com.hadoop.compression.lzo.LzopCodec`
*/
public class LzopCodec extends org.apache.hadoop.io.compress.LzopCodec {
private static final Logger LOG = LoggerFactory.getLogger(LzopCodec.class.getName());
private static final String gplLzopCodec = LzopCodec.class.getName();
private static final String hadoopLzopCodec = org.apache.hadoop.io.compress.LzopCodec.class.getName();
private static AtomicBoolean warned = new AtomicBoolean(false);

static {
LOG.info("Bridging " + gplLzopCodec + " to " + hadoopLzopCodec + ".");
}

@Override
public CompressionOutputStream createOutputStream(OutputStream out,
Compressor compressor) throws IOException {
if (warned.compareAndSet(false, true)) {
LOG.warn("{} is deprecated. You should use {} instead to generate LZOP compressed data.",
gplLzopCodec, hadoopLzopCodec);
}
return super.createOutputStream(out, compressor);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -142,7 +142,7 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {

/** Default value for IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY */
public static final int IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_DEFAULT =
64*1024;
256 * 1024;

/** Internal buffer size for Snappy compressor/decompressors */
public static final String IO_COMPRESSION_CODEC_SNAPPY_BUFFERSIZE_KEY =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,4 +65,14 @@ private CodecConstants() {
* Default extension for {@link org.apache.hadoop.io.compress.ZStandardCodec}.
*/
public static final String ZSTANDARD_CODEC_EXTENSION = ".zst";

/**
* Default extension for {@link org.apache.hadoop.io.compress.LzoCodec}.
*/
public static final String LZO_CODEC_EXTENSION = ".lzo_deflate";

/**
* Default extension for {@link org.apache.hadoop.io.compress.LzopCodec}.
*/
public static final String LZOP_CODEC_EXTENSION = ".lzo";
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,186 @@
/*
* 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.compress;

import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;

import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.io.compress.lzo.LzoCompressor;
import org.apache.hadoop.io.compress.lzo.LzoDecompressor;

/**
* This class creates lzo compressors/decompressors.
*/
public class LzoCodec implements Configurable, CompressionCodec {
Configuration conf;

/**
* Set the configuration to be used by this object.
*
* @param conf the configuration object.
*/
@Override
public void setConf(Configuration conf) {
this.conf = conf;
}

/**
* Return the configuration used by this object.
*
* @return the configuration object used by this objec.
*/
@Override
public Configuration getConf() {
return conf;
}

/**
* Create a {@link CompressionOutputStream} that will write to the given
* {@link OutputStream}.
*
* @param out the location for the final output stream
* @return a stream the user can write uncompressed data to have it compressed
* @throws IOException
*/
@Override
public CompressionOutputStream createOutputStream(OutputStream out)
throws IOException {
return CompressionCodec.Util.
createOutputStreamWithCodecPool(this, conf, out);
}

/**
* Create a {@link CompressionOutputStream} that will write to the given
* {@link OutputStream} with the given {@link Compressor}.
*
* @param out the location for the final output stream
* @param compressor compressor to use
* @return a stream the user can write uncompressed data to have it compressed
* @throws IOException
*/
@Override
public CompressionOutputStream createOutputStream(OutputStream out,
Compressor compressor) throws IOException {
int bufferSize = conf.getInt(
CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY,
CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_DEFAULT);
io.airlift.compress.lzo.LzoCodec codec = new io.airlift.compress.lzo.LzoCodec();
Configuration lzoConf = new Configuration(this.conf);
lzoConf.setInt(CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY, bufferSize);
codec.setConf(lzoConf);
return codec.createOutputStream(out);
}

/**
* Get the type of {@link Compressor} needed by this {@link CompressionCodec}.
*
* @return the type of compressor needed by this codec.
*/
@Override
public Class<? extends Compressor> getCompressorType() {
return LzoCompressor.class;
}

/**
* Create a new {@link Compressor} for use by this {@link CompressionCodec}.
*
* @return a new compressor for use by this codec
*/
@Override
public Compressor createCompressor() {
int bufferSize = conf.getInt(
CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY,
CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_DEFAULT);
return new LzoCompressor(bufferSize);
}

/**
* Create a {@link CompressionInputStream} that will read from the given
* input stream.
*
* @param in the stream to read compressed bytes from
* @return a stream to read uncompressed bytes from
* @throws IOException
*/
@Override
public CompressionInputStream createInputStream(InputStream in)
throws IOException {
return CompressionCodec.Util.
createInputStreamWithCodecPool(this, conf, in);
}

/**
* Create a {@link CompressionInputStream} that will read from the given
* {@link InputStream} with the given {@link Decompressor}.
*
* @param in the stream to read compressed bytes from
* @param decompressor decompressor to use
* @return a stream to read uncompressed bytes from
* @throws IOException
*/
@Override
public CompressionInputStream createInputStream(InputStream in,
Decompressor decompressor) throws IOException {
int bufferSize = conf.getInt(
CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY,
CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_DEFAULT);
io.airlift.compress.lzo.LzoCodec codec = new io.airlift.compress.lzo.LzoCodec();
Configuration lzoConf = new Configuration(this.conf);
lzoConf.setInt(CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY, bufferSize);
codec.setConf(lzoConf);
return codec.createInputStream(in);
}

/**
* Get the type of {@link Decompressor} needed by this {@link CompressionCodec}.
*
* @return the type of decompressor needed by this codec.
*/
@Override
public Class<? extends Decompressor> getDecompressorType() {
return LzoDecompressor.class;
}

/**
* Create a new {@link Decompressor} for use by this {@link CompressionCodec}.
*
* @return a new decompressor for use by this codec
*/
@Override
public Decompressor createDecompressor() {
int bufferSize = conf.getInt(
CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY,
CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_DEFAULT);
return new LzoDecompressor(bufferSize);
}

/**
* Get the default filename extension for this kind of compression.
*
* @return <code>.lzo_deflate</code>.
*/
@Override
public String getDefaultExtension() {
return CodecConstants.LZO_CODEC_EXTENSION;
}
}
Loading