You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by GitBox <gi...@apache.org> on 2020/07/21 10:48:00 UTC

[GitHub] [hadoop] steveloughran commented on a change in pull request #2159: HADOOP-17124. Support LZO Codec using aircompressor

steveloughran commented on a change in pull request #2159:
URL: https://github.com/apache/hadoop/pull/2159#discussion_r457977064



##########
File path: hadoop-common-project/hadoop-common/src/main/java/com/hadoop/compression/lzo/LzoCodec.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import org.apache.hadoop.io.compress.CompressionOutputStream;
+import org.apache.hadoop.io.compress.Compressor;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+public class LzoCodec extends org.apache.hadoop.io.compress.LzoCodec {
+    private static final Log LOG = LogFactory.getLog(LzoCodec.class);
+
+    static final String gplLzoCodec = LzoCodec.class.getName();
+    static final String hadoopLzoCodec = org.apache.hadoop.io.compress.LzoCodec.class.getName();
+    static boolean warned = false;
+
+    static {
+        LOG.info("Bridging " + gplLzoCodec + " to " + hadoopLzoCodec + ".");

Review comment:
       with a move to slf4J you can do the string construction on demand with {} placemarkers

##########
File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/LzopCodec.java
##########
@@ -0,0 +1,108 @@
+/*
+ * 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 io.airlift.compress.lzo.LzoCodec;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.IOException;
+
+public class LzopCodec extends io.airlift.compress.lzo.LzopCodec
+        implements Configurable, CompressionCodec {
+    @Override
+    public Class<? extends Compressor> getCompressorType()
+    {
+        return LzopCodec.HadoopLzopCompressor.class;
+    }
+
+    /**
+     * No Hadoop code seems to actually use the compressor, so just return a dummy one so the createOutputStream method

Review comment:
       does anyone know about downstream uses?

##########
File path: hadoop-common-project/hadoop-common/src/main/java/com/hadoop/compression/lzo/LzoCodec.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import org.apache.hadoop.io.compress.CompressionOutputStream;
+import org.apache.hadoop.io.compress.Compressor;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+public class LzoCodec extends org.apache.hadoop.io.compress.LzoCodec {
+    private static final Log LOG = LogFactory.getLog(LzoCodec.class);
+
+    static final String gplLzoCodec = LzoCodec.class.getName();

Review comment:
       private scope

##########
File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/LzopCodec.java
##########
@@ -0,0 +1,108 @@
+/*
+ * 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 io.airlift.compress.lzo.LzoCodec;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.IOException;
+
+public class LzopCodec extends io.airlift.compress.lzo.LzopCodec
+        implements Configurable, CompressionCodec {
+    @Override
+    public Class<? extends Compressor> getCompressorType()
+    {
+        return LzopCodec.HadoopLzopCompressor.class;
+    }
+
+    /**
+     * No Hadoop code seems to actually use the compressor, so just return a dummy one so the createOutputStream method
+     * with a compressor can function.  This interface can be implemented if needed.
+     */
+    @DoNotPool
+    static class HadoopLzopCompressor
+            implements Compressor

Review comment:
       K&R / java {  placement please. 

##########
File path: hadoop-project/pom.xml
##########
@@ -1727,6 +1728,11 @@
         <artifactId>jna</artifactId>
         <version>${jna.version}</version>
       </dependency>
+      <dependency>

Review comment:
       does this add any transient dependencies?

##########
File path: hadoop-common-project/hadoop-common/src/main/java/com/hadoop/compression/lzo/LzoCodec.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import org.apache.hadoop.io.compress.CompressionOutputStream;
+import org.apache.hadoop.io.compress.Compressor;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+public class LzoCodec extends org.apache.hadoop.io.compress.LzoCodec {
+    private static final Log LOG = LogFactory.getLog(LzoCodec.class);
+
+    static final String gplLzoCodec = LzoCodec.class.getName();
+    static final String hadoopLzoCodec = org.apache.hadoop.io.compress.LzoCodec.class.getName();
+    static boolean warned = false;
+
+    static {
+        LOG.info("Bridging " + gplLzoCodec + " to " + hadoopLzoCodec + ".");
+    }
+
+    @Override
+    public CompressionOutputStream createOutputStream(OutputStream out,
+                                                      Compressor compressor) throws IOException {
+        if (!warned) {

Review comment:
       there's a risk of >1 warning in a multithread env, but not something I'm worried about. AtomicBoolean would be the purist way to do it.

##########
File path: hadoop-common-project/hadoop-common/src/main/java/com/hadoop/compression/lzo/LzoCodec.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import org.apache.hadoop.io.compress.CompressionOutputStream;

Review comment:
       imports: separate blocks for non-static imports of, in order
   
   java.* and javax.*
   
   non-org.apache
   
   org.apache*
   
   static imports
   
   existing code to be left alone to keep cherrypicking simple

##########
File path: hadoop-common-project/hadoop-common/src/main/java/com/hadoop/compression/lzo/LzoCodec.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import org.apache.hadoop.io.compress.CompressionOutputStream;
+import org.apache.hadoop.io.compress.Compressor;
+
+import org.apache.commons.logging.Log;

Review comment:
       Use SLF4J

##########
File path: hadoop-common-project/hadoop-common/src/main/java/com/hadoop/compression/lzo/LzopCodec.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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;

Review comment:
       com.hadoop?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org