Kaynağa Gözat

HADOOP-17891. Exclude snappy-java and lz4-java from relocation in shaded hadoop client libraries (#3385)

Liang-Chi Hsieh 3 yıl önce
ebeveyn
işleme
b9715c2931

+ 9 - 2
dev-support/bin/hadoop.sh

@@ -512,7 +512,7 @@ function shadedclient_initialize
   maven_add_install shadedclient
 }
 
-## @description build client facing shaded artifacts and test them
+## @description build client facing shaded and non-shaded artifacts and test them
 ## @audience private
 ## @stability evolving
 ## @param repostatus
@@ -545,13 +545,20 @@ function shadedclient_rebuild
     return 0
   fi
 
-  big_console_header "Checking client artifacts on ${repostatus}"
+  big_console_header "Checking client artifacts on ${repostatus} with shaded clients"
 
   echo_and_redirect "${logfile}" \
     "${MAVEN}" "${MAVEN_ARGS[@]}" verify -fae --batch-mode -am \
       "${modules[@]}" \
       -Dtest=NoUnitTests -Dmaven.javadoc.skip=true -Dcheckstyle.skip=true -Dspotbugs.skip=true
 
+  big_console_header "Checking client artifacts on ${repostatus} with non-shaded clients"
+
+  echo_and_redirect "${logfile}" \
+    "${MAVEN}" "${MAVEN_ARGS[@]}" verify -fae --batch-mode -am \
+      "${modules[@]}" \
+      -Pnoshade -Dtest=NoUnitTests -Dmaven.javadoc.skip=true -Dcheckstyle.skip=true -Dspotbugs.skip=true
+
   count=$("${GREP}" -c '\[ERROR\]' "${logfile}")
   if [[ ${count} -gt 0 ]]; then
     add_vote_table -1 shadedclient "${repostatus} has errors when building and testing our client artifacts."

+ 17 - 0
hadoop-client-modules/hadoop-client-api/pom.xml

@@ -67,6 +67,13 @@
         </exclusion>
       </exclusions>
     </dependency>
+    <!-- snappy-java is native library and cannot be relocated. So we explicitly exclude it
+         from shaded jar to prevent possible conflict. Make it as transitive dependency to
+         make the downstream pull it. -->
+    <dependency>
+      <groupId>org.xerial.snappy</groupId>
+      <artifactId>snappy-java</artifactId>
+    </dependency>
   </dependencies>
   <profiles>
     <profile>
@@ -109,6 +116,10 @@
                     <includes>
                       <include>org.apache.hadoop:*</include>
                     </includes>
+                    <excludes>
+                      <!-- Leave snappy that includes native methods which cannot be relocated. -->
+                      <exclude>org.xerial.snappy:*</exclude>
+                    </excludes>
                   </artifactSet>
                   <filters>
                     <!-- We get these package level classes from various yarn api jars -->
@@ -149,6 +160,9 @@
                         <exclude>org/xml/sax/**/*</exclude>
                         <exclude>org/bouncycastle/*</exclude>
                         <exclude>org/bouncycastle/**/*</exclude>
+                        <!-- Exclude snappy-java -->
+                        <exclude>org/xerial/snappy/*</exclude>
+                        <exclude>org/xerial/snappy/**/*</exclude>
                       </excludes>
                     </relocation>
                     <relocation>
@@ -225,6 +239,9 @@
                         <!-- Exclude config keys for Hadoop that look like package names -->
                         <exclude>net/topology/*</exclude>
                         <exclude>net/topology/**/*</exclude>
+                        <!-- Exclude lz4-java -->
+                        <exclude>net/jpountz/*</exclude>
+                        <exclude>net/jpountz/**/*</exclude>
                       </excludes>
                     </relocation>
                     <!-- okio declares a top level package instead of nested -->

+ 2 - 0
hadoop-client-modules/hadoop-client-check-invariants/pom.xml

@@ -92,6 +92,8 @@
                     <exclude>com.google.code.findbugs:jsr305</exclude>
                     <!-- Leave bouncycastle unshaded because it's signed with a special Oracle certificate so it can be a custom JCE security provider -->
                     <exclude>org.bouncycastle:*</exclude>
+                    <!-- Leave snappy that includes native methods which cannot be relocated. -->
+                    <exclude>org.xerial.snappy:*</exclude>
                   </excludes>
                 </banTransitiveDependencies>
                 <banDuplicateClasses>

+ 2 - 0
hadoop-client-modules/hadoop-client-check-invariants/src/test/resources/ensure-jars-have-correct-contents.sh

@@ -67,6 +67,8 @@ allowed_expr+="|^krb5_udp-template.conf$"
 # Jetty uses this style sheet for directory listings. TODO ensure our
 # internal use of jetty disallows directory listings and remove this.
 allowed_expr+="|^jetty-dir.css$"
+# Snappy java is native library. We cannot relocate it to under org/apache/hadoop.
+allowed_expr+="|^org/xerial/"
 
 allowed_expr+=")"
 declare -i bad_artifacts=0

+ 2 - 0
hadoop-client-modules/hadoop-client-check-test-invariants/pom.xml

@@ -100,6 +100,8 @@
                     <exclude>com.google.code.findbugs:jsr305</exclude>
                     <!-- Leave bouncycastle unshaded because it's signed with a special Oracle certificate so it can be a custom JCE security provider -->
                     <exclude>org.bouncycastle:*</exclude>
+                    <!-- Leave snappy that includes native methods which cannot be relocated. -->
+                    <exclude>org.xerial.snappy:*</exclude>
                   </excludes>
                 </banTransitiveDependencies>
                 <banDuplicateClasses>

+ 5 - 0
hadoop-client-modules/hadoop-client-integration-tests/pom.xml

@@ -52,6 +52,11 @@
       <artifactId>junit</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.lz4</groupId>
+      <artifactId>lz4-java</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
   <profiles>
     <profile>

+ 144 - 0
hadoop-client-modules/hadoop-client-integration-tests/src/test/java/org/apache/hadoop/example/ITUseHadoopCodecs.java

@@ -0,0 +1,144 @@
+/*
+ * 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.example;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import java.io.*;
+import java.util.Arrays;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.RandomDatum;
+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.zlib.ZlibFactory;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Ensure that we can perform codec operations given the API and runtime jars
+ * by performing some simple smoke tests.
+ */
+public class ITUseHadoopCodecs {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ITUseHadoopCodecs.class);
+
+  private Configuration haddopConf = new Configuration();
+  private int dataCount = 100;
+  private int dataSeed = new Random().nextInt();
+
+  @Test
+  public void testGzipCodec() throws IOException {
+    ZlibFactory.setNativeZlibLoaded(false);
+    assertFalse(ZlibFactory.isNativeZlibLoaded(haddopConf));
+    codecTest(haddopConf, dataSeed, 0, "org.apache.hadoop.io.compress.GzipCodec");
+    codecTest(haddopConf, dataSeed, dataCount, "org.apache.hadoop.io.compress.GzipCodec");
+  }
+
+  @Test
+  public void testSnappyCodec() throws IOException {
+    codecTest(haddopConf, dataSeed, 0, "org.apache.hadoop.io.compress.SnappyCodec");
+    codecTest(haddopConf, dataSeed, dataCount, "org.apache.hadoop.io.compress.SnappyCodec");
+  }
+
+  @Test
+  public void testLz4Codec() {
+    Arrays.asList(false, true).forEach(config -> {
+      haddopConf.setBoolean(
+          CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZ4_USELZ4HC_KEY,
+          config);
+      try {
+        codecTest(haddopConf, dataSeed, 0, "org.apache.hadoop.io.compress.Lz4Codec");
+        codecTest(haddopConf, dataSeed, dataCount, "org.apache.hadoop.io.compress.Lz4Codec");
+      } catch (IOException e) {
+        throw new RuntimeException("failed when running codecTest", e);
+      }
+    });
+  }
+
+  private void codecTest(Configuration conf, int seed, int count, String codecClass)
+      throws IOException {
+
+    // Create the codec
+    CompressionCodec codec = null;
+    try {
+      codec = (CompressionCodec)
+              ReflectionUtils.newInstance(conf.getClassByName(codecClass), conf);
+    } catch (ClassNotFoundException cnfe) {
+      throw new IOException("Illegal codec!");
+    }
+    LOG.info("Created a Codec object of type: " + codecClass);
+
+    // Generate data
+    DataOutputBuffer data = new DataOutputBuffer();
+    RandomDatum.Generator generator = new RandomDatum.Generator(seed);
+    for(int i = 0; i < count; ++i) {
+      generator.next();
+      RandomDatum key = generator.getKey();
+      RandomDatum value = generator.getValue();
+
+      key.write(data);
+      value.write(data);
+    }
+    LOG.info("Generated " + count + " records");
+
+    // Compress data
+    DataOutputBuffer compressedDataBuffer = new DataOutputBuffer();
+    try (CompressionOutputStream deflateFilter =
+        codec.createOutputStream(compressedDataBuffer);
+        DataOutputStream deflateOut =
+            new DataOutputStream(new BufferedOutputStream(deflateFilter))) {
+      deflateOut.write(data.getData(), 0, data.getLength());
+      deflateOut.flush();
+      deflateFilter.finish();
+    }
+
+    // De-compress data
+    DataInputBuffer deCompressedDataBuffer = new DataInputBuffer();
+    deCompressedDataBuffer.reset(compressedDataBuffer.getData(), 0,
+            compressedDataBuffer.getLength());
+    DataInputBuffer originalData = new DataInputBuffer();
+    originalData.reset(data.getData(), 0, data.getLength());
+    try (CompressionInputStream inflateFilter =
+        codec.createInputStream(deCompressedDataBuffer);
+        DataInputStream originalIn =
+            new DataInputStream(new BufferedInputStream(originalData))) {
+
+      // Check
+      int expected;
+      do {
+        expected = originalIn.read();
+        assertEquals("Inflated stream read by byte does not match",
+                expected, inflateFilter.read());
+      } while (expected != -1);
+    }
+
+    LOG.info("SUCCESS! Completed checking " + count + " records");
+  }
+}

+ 14 - 0
hadoop-client-modules/hadoop-client-minicluster/pom.xml

@@ -40,6 +40,12 @@
       <artifactId>hadoop-client-api</artifactId>
       <scope>runtime</scope>
     </dependency>
+    <!-- This is the api's compile dependency, but we don't want it to be compile dependency here too. -->
+    <dependency>
+      <groupId>org.xerial.snappy</groupId>
+      <artifactId>snappy-java</artifactId>
+      <scope>runtime</scope>
+    </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-client-runtime</artifactId>
@@ -683,6 +689,8 @@
                       <!-- We need a filter that matches just those things that are included in the above artiacts -->
                       <!-- Leave bouncycastle unshaded because it's signed with a special Oracle certificate so it can be a custom JCE security provider -->
                       <exclude>org.bouncycastle:*</exclude>
+                      <!-- Leave snappy that includes native methods which cannot be relocated. -->
+                      <exclude>org.xerial.snappy:*</exclude>
                     </excludes>
                   </artifactSet>
                   <filters>
@@ -886,6 +894,9 @@
                         <exclude>org/xml/sax/**/*</exclude>
                         <exclude>org/bouncycastle/*</exclude>
                         <exclude>org/bouncycastle/**/*</exclude>
+                        <!-- Exclude snappy-java -->
+                        <exclude>org/xerial/snappy/*</exclude>
+                        <exclude>org/xerial/snappy/**/*</exclude>
                       </excludes>
                     </relocation>
                     <relocation>
@@ -1004,6 +1015,9 @@
                         <!-- Exclude config keys for Hadoop that look like package names -->
                         <exclude>net/topology/*</exclude>
                         <exclude>net/topology/**/*</exclude>
+                        <!-- Exclude lz4-java -->
+                        <exclude>net/jpountz/*</exclude>
+                        <exclude>net/jpountz/**/*</exclude>
                       </excludes>
                     </relocation>
                     <!-- okio declares a top level package instead of nested -->

+ 14 - 0
hadoop-client-modules/hadoop-client-runtime/pom.xml

@@ -60,6 +60,12 @@
       <artifactId>hadoop-client-api</artifactId>
       <scope>runtime</scope>
     </dependency>
+    <!-- This is the api's compile dependency, but we don't want it to be compile dependency here too. -->
+    <dependency>
+      <groupId>org.xerial.snappy</groupId>
+      <artifactId>snappy-java</artifactId>
+      <scope>runtime</scope>
+    </dependency>
     <!-- This comes from our parent pom. If we don't expressly change it here to get included,
          downstream will get warnings at compile time. -->
     <dependency>
@@ -163,6 +169,8 @@
                       <exclude>org.ow2.asm:*</exclude>
                       <!-- Leave bouncycastle unshaded because it's signed with a special Oracle certificate so it can be a custom JCE security provider -->
                       <exclude>org.bouncycastle:*</exclude>
+                      <!-- Leave snappy that includes native methods which cannot be relocated. -->
+                      <exclude>org.xerial.snappy:*</exclude>
                     </excludes>
                   </artifactSet>
                   <filters>
@@ -269,6 +277,9 @@
                         <exclude>org/xml/sax/**/*</exclude>
                         <exclude>org/bouncycastle/*</exclude>
                         <exclude>org/bouncycastle/**/*</exclude>
+                        <!-- Exclude snappy-java -->
+                        <exclude>org/xerial/snappy/*</exclude>
+                        <exclude>org/xerial/snappy/**/*</exclude>
                       </excludes>
                     </relocation>
                     <relocation>
@@ -359,6 +370,9 @@
                         <!-- Exclude config keys for Hadoop that look like package names -->
                         <exclude>net/topology/*</exclude>
                         <exclude>net/topology/**/*</exclude>
+                        <!-- Exclude lz4-java -->
+                        <exclude>net/jpountz/*</exclude>
+                        <exclude>net/jpountz/**/*</exclude>
                       </excludes>
                     </relocation>
                     <!-- okio declares a top level package instead of nested -->