Bladeren bron

HADOOP-18029: Update CompressionCodecFactory to handle uppercase file extensions (#3739)

Co-authored-by: Desmond Sisson <sissonde@amazon.com>
Desmond Sisson 3 jaren geleden
bovenliggende
commit
df4197592f

+ 74 - 66
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionCodecFactory.java

@@ -40,7 +40,7 @@ public class CompressionCodecFactory {
       LoggerFactory.getLogger(CompressionCodecFactory.class.getName());
   
   private static final ServiceLoader<CompressionCodec> CODEC_PROVIDERS =
-    ServiceLoader.load(CompressionCodec.class);
+          ServiceLoader.load(CompressionCodec.class);
 
   /**
    * A map from the reversed filename suffixes to the codecs.
@@ -49,15 +49,15 @@ public class CompressionCodecFactory {
    */
   private SortedMap<String, CompressionCodec> codecs = null;
 
-    /**
-     * A map from the reversed filename suffixes to the codecs.
-     * This is probably overkill, because the maps should be small, but it
-     * automatically supports finding the longest matching suffix.
-     */
-    private Map<String, CompressionCodec> codecsByName = null;
+  /**
+   * A map from the reversed filename suffixes to the codecs.
+   * This is probably overkill, because the maps should be small, but it
+   * automatically supports finding the longest matching suffix.
+   */
+  private Map<String, CompressionCodec> codecsByName = null;
 
   /**
-   * A map from class names to the codecs
+   * A map from class names to the codecs.
    */
   private HashMap<String, CompressionCodec> codecsByClassName = null;
 
@@ -80,8 +80,8 @@ public class CompressionCodecFactory {
   @Override
   public String toString() {
     StringBuilder buf = new StringBuilder();
-    Iterator<Map.Entry<String, CompressionCodec>> itr = 
-      codecs.entrySet().iterator();
+    Iterator<Map.Entry<String, CompressionCodec>> itr =
+            codecs.entrySet().iterator();
     buf.append("{ ");
     if (itr.hasNext()) {
       Map.Entry<String, CompressionCodec> entry = itr.next();
@@ -110,8 +110,8 @@ public class CompressionCodecFactory {
    */
   public static List<Class<? extends CompressionCodec>> getCodecClasses(
       Configuration conf) {
-    List<Class<? extends CompressionCodec>> result
-      = new ArrayList<Class<? extends CompressionCodec>>();
+    List<Class<? extends CompressionCodec>> result =
+            new ArrayList<Class<? extends CompressionCodec>>();
     // Add codec classes discovered via service loading
     synchronized (CODEC_PROVIDERS) {
       // CODEC_PROVIDERS is a lazy collection. Synchronize so it is
@@ -200,11 +200,13 @@ public class CompressionCodecFactory {
       String filename = file.getName();
       String reversedFilename =
           new StringBuilder(filename).reverse().toString();
-      SortedMap<String, CompressionCodec> subMap = 
-        codecs.headMap(reversedFilename);
+      String lowerReversedFilename =
+              StringUtils.toLowerCase(reversedFilename);
+      SortedMap<String, CompressionCodec> subMap =
+              codecs.headMap(lowerReversedFilename);
       if (!subMap.isEmpty()) {
         String potentialSuffix = subMap.lastKey();
-        if (reversedFilename.startsWith(potentialSuffix)) {
+        if (lowerReversedFilename.startsWith(potentialSuffix)) {
           result = codecs.get(potentialSuffix);
         }
       }
@@ -224,57 +226,57 @@ public class CompressionCodecFactory {
     return codecsByClassName.get(classname);
   }
 
-    /**
-     * Find the relevant compression codec for the codec's canonical class name
-     * or by codec alias.
-     * <p>
-     * Codec aliases are case insensitive.
-     * <p>
-     * The code alias is the short class name (without the package name).
-     * If the short class name ends with 'Codec', then there are two aliases for
-     * the codec, the complete short class name and the short class name without
-     * the 'Codec' ending. For example for the 'GzipCodec' codec class name the
-     * alias are 'gzip' and 'gzipcodec'.
-     *
-     * @param codecName the canonical class name of the codec
-     * @return the codec object
-     */
-    public CompressionCodec getCodecByName(String codecName) {
-      if (codecsByClassName == null) {
-        return null;
-      }
-      CompressionCodec codec = getCodecByClassName(codecName);
-      if (codec == null) {
-        // trying to get the codec by name in case the name was specified
-        // instead a class
-        codec = codecsByName.get(StringUtils.toLowerCase(codecName));
-      }
-      return codec;
+  /**
+   * Find the relevant compression codec for the codec's canonical class name
+   * or by codec alias.
+   * <p>
+   * Codec aliases are case insensitive.
+   * <p>
+   * The code alias is the short class name (without the package name).
+   * If the short class name ends with 'Codec', then there are two aliases for
+   * the codec, the complete short class name and the short class name without
+   * the 'Codec' ending. For example for the 'GzipCodec' codec class name the
+   * alias are 'gzip' and 'gzipcodec'.
+   *
+   * @param codecName the canonical class name of the codec
+   * @return the codec object
+   */
+  public CompressionCodec getCodecByName(String codecName) {
+    if (codecsByClassName == null) {
+      return null;
     }
+    CompressionCodec codec = getCodecByClassName(codecName);
+    if (codec == null) {
+      // trying to get the codec by name in case the name was specified
+      // instead a class
+      codec = codecsByName.get(StringUtils.toLowerCase(codecName));
+    }
+    return codec;
+  }
 
-    /**
-     * Find the relevant compression codec for the codec's canonical class name
-     * or by codec alias and returns its implemetation class.
-     * <p>
-     * Codec aliases are case insensitive.
-     * <p>
-     * The code alias is the short class name (without the package name).
-     * If the short class name ends with 'Codec', then there are two aliases for
-     * the codec, the complete short class name and the short class name without
-     * the 'Codec' ending. For example for the 'GzipCodec' codec class name the
-     * alias are 'gzip' and 'gzipcodec'.
-     *
-     * @param codecName the canonical class name of the codec
-     * @return the codec class
-     */
-    public Class<? extends CompressionCodec> getCodecClassByName(
-        String codecName) {
-      CompressionCodec codec = getCodecByName(codecName);
-      if (codec == null) {
-        return null;
-      }
-      return codec.getClass();
+  /**
+   * Find the relevant compression codec for the codec's canonical class name
+   * or by codec alias and returns its implemetation class.
+   * <p>
+   * Codec aliases are case insensitive.
+   * <p>
+   * The code alias is the short class name (without the package name).
+   * If the short class name ends with 'Codec', then there are two aliases for
+   * the codec, the complete short class name and the short class name without
+   * the 'Codec' ending. For example for the 'GzipCodec' codec class name the
+   * alias are 'gzip' and 'gzipcodec'.
+   *
+   * @param codecName the canonical class name of the codec
+   * @return the codec class
+   */
+  public Class<? extends CompressionCodec> getCodecClassByName(
+      String codecName) {
+    CompressionCodec codec = getCodecByName(codecName);
+    if (codec == null) {
+      return null;
     }
+    return codec.getClass();
+  }
 
   /**
    * Removes a suffix from a filename, if it has it.
@@ -323,8 +325,12 @@ public class CompressionCodecFactory {
                 len = in.read(buffer);
               }
             } finally {
-              if(out != null) { out.close(); }
-              if(in  != null) { in.close(); }
+              if(out != null) {
+                out.close();
+              }
+              if(in  != null) {
+                in.close();
+              }
             }
           } else {
             CompressionInputStream in = null;
@@ -338,7 +344,9 @@ public class CompressionCodecFactory {
                 len = in.read(buffer);
               }
             } finally {
-              if(in != null) { in.close(); }
+              if(in != null) {
+                in.close();
+              }
             }
           }
         }

+ 36 - 13
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/TestCodecFactory.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.conf.Configuration;
 
 import org.junit.Test;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.fail;
 
 public class TestCodecFactory {
@@ -45,8 +46,8 @@ public class TestCodecFactory {
     }
     
     @Override
-    public CompressionOutputStream createOutputStream(OutputStream out) 
-    throws IOException {
+    public CompressionOutputStream createOutputStream(OutputStream out)
+            throws IOException {
       return null;
     }
     
@@ -62,21 +63,21 @@ public class TestCodecFactory {
 
     @Override
     public CompressionInputStream createInputStream(InputStream in, 
-                                                    Decompressor decompressor) 
-    throws IOException {
+                                                    Decompressor decompressor)
+            throws IOException {
       return null;
     }
 
     @Override
-    public CompressionInputStream createInputStream(InputStream in) 
-    throws IOException {
+    public CompressionInputStream createInputStream(InputStream in)
+            throws IOException {
       return null;
     }
 
     @Override
     public CompressionOutputStream createOutputStream(OutputStream out, 
-                                                      Compressor compressor) 
-    throws IOException {
+                                                      Compressor compressor)
+            throws IOException {
       return null;
     }
 
@@ -125,7 +126,7 @@ public class TestCodecFactory {
   }
   
   /**
-   * Returns a factory for a given set of codecs
+   * Returns a factory for a given set of codecs.
    * @param classes the codec classes to include
    * @return a new factory
    */
@@ -137,15 +138,21 @@ public class TestCodecFactory {
   
   private static void checkCodec(String msg, 
                                  Class expected, CompressionCodec actual) {
-    assertEquals(msg + " unexpected codec found",
-                 expected.getName(),
-                 actual.getClass().getName());
+    if (expected == null) {
+      assertNull(msg, actual);
+    } else if (actual == null) {
+      fail(msg + " result was null");
+    } else {
+      assertEquals(msg + " unexpected codec found",
+              expected.getName(),
+              actual.getClass().getName());
+    }
   }
 
   @Test
   public void testFinding() {
     CompressionCodecFactory factory =
-      new CompressionCodecFactory(new Configuration());
+            new CompressionCodecFactory(new Configuration());
     CompressionCodec codec = factory.getCodec(new Path("/tmp/foo.bar"));
     assertEquals("default factory foo codec", null, codec);
     codec = factory.getCodecByClassName(BarCodec.class.getCanonicalName());
@@ -153,6 +160,8 @@ public class TestCodecFactory {
     
     codec = factory.getCodec(new Path("/tmp/foo.gz"));
     checkCodec("default factory for .gz", GzipCodec.class, codec);
+    codec = factory.getCodec(new Path("/tmp/foo.GZ"));
+    checkCodec("default factory for .GZ", GzipCodec.class, codec);
     codec = factory.getCodecByClassName(GzipCodec.class.getCanonicalName());
     checkCodec("default factory for gzip codec", GzipCodec.class, codec);
     codec = factory.getCodecByName("gzip");
@@ -168,6 +177,8 @@ public class TestCodecFactory {
 
     codec = factory.getCodec(new Path("/tmp/foo.bz2"));
     checkCodec("default factory for .bz2", BZip2Codec.class, codec);
+    codec = factory.getCodec(new Path("/tmp/foo.BZ2"));
+    checkCodec("default factory for .BZ2", BZip2Codec.class, codec);
     codec = factory.getCodecByClassName(BZip2Codec.class.getCanonicalName());
     checkCodec("default factory for bzip2 codec", BZip2Codec.class, codec);
     codec = factory.getCodecByName("bzip2");
@@ -221,16 +232,22 @@ public class TestCodecFactory {
                                      FooBarCodec.class});
     codec = factory.getCodec(new Path("/tmp/.foo.bar.gz"));
     checkCodec("full factory gz codec", GzipCodec.class, codec);
+    codec = factory.getCodec(new Path("/tmp/.foo.bar.GZ"));
+    checkCodec("full factory GZ codec", GzipCodec.class, codec);
     codec = factory.getCodecByClassName(GzipCodec.class.getCanonicalName());
     checkCodec("full codec gz codec", GzipCodec.class, codec);
      
     codec = factory.getCodec(new Path("/tmp/foo.bz2"));
     checkCodec("full factory for .bz2", BZip2Codec.class, codec);
+    codec = factory.getCodec(new Path("/tmp/foo.BZ2"));
+    checkCodec("full factory for .BZ2", BZip2Codec.class, codec);
     codec = factory.getCodecByClassName(BZip2Codec.class.getCanonicalName());
     checkCodec("full codec bzip2 codec", BZip2Codec.class, codec);
 
     codec = factory.getCodec(new Path("/tmp/foo.bar"));
     checkCodec("full factory bar codec", BarCodec.class, codec);
+    codec = factory.getCodec(new Path("/tmp/foo.BAR"));
+    checkCodec("full factory BAR codec", BarCodec.class, codec);
     codec = factory.getCodecByClassName(BarCodec.class.getCanonicalName());
     checkCodec("full factory bar codec", BarCodec.class, codec);
     codec = factory.getCodecByName("bar");
@@ -240,6 +257,8 @@ public class TestCodecFactory {
 
     codec = factory.getCodec(new Path("/tmp/foo/baz.foo.bar"));
     checkCodec("full factory foo bar codec", FooBarCodec.class, codec);
+    codec = factory.getCodec(new Path("/tmp/foo/baz.FOO.bar"));
+    checkCodec("full factory FOO bar codec", FooBarCodec.class, codec);
     codec = factory.getCodecByClassName(FooBarCodec.class.getCanonicalName());
     checkCodec("full factory foo bar codec", FooBarCodec.class, codec);
     codec = factory.getCodecByName("foobar");
@@ -249,6 +268,8 @@ public class TestCodecFactory {
 
     codec = factory.getCodec(new Path("/tmp/foo.foo"));
     checkCodec("full factory foo codec", FooCodec.class, codec);
+    codec = factory.getCodec(new Path("/tmp/FOO.FOO"));
+    checkCodec("full factory FOO codec", FooCodec.class, codec);
     codec = factory.getCodecByClassName(FooCodec.class.getCanonicalName());
     checkCodec("full factory foo codec", FooCodec.class, codec);
     codec = factory.getCodecByName("foo");
@@ -259,6 +280,8 @@ public class TestCodecFactory {
     factory = setClasses(new Class[]{NewGzipCodec.class});
     codec = factory.getCodec(new Path("/tmp/foo.gz"));
     checkCodec("overridden factory for .gz", NewGzipCodec.class, codec);
+    codec = factory.getCodec(new Path("/tmp/foo.GZ"));
+    checkCodec("overridden factory for .GZ", NewGzipCodec.class, codec);
     codec = factory.getCodecByClassName(NewGzipCodec.class.getCanonicalName());
     checkCodec("overridden factory for gzip codec", NewGzipCodec.class, codec);