소스 검색

HADOOP-6698. Revert the io.serialization package to 0.20.2's api. Reverted HADOOP-6165, HADOOP-6443, HADOOP-6323, and HADOOP-6420.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@939412 13f79535-47bb-0310-9956-ffa450edef68
Thomas White 15 년 전
부모
커밋
750fb2dbc1
28개의 변경된 파일109개의 추가작업 그리고 1374개의 파일을 삭제
  1. 0 11
      CHANGES.txt
  2. 1 1
      src/java/core-default.xml
  3. 0 133
      src/java/org/apache/hadoop/conf/Configuration.java
  4. 13 15
      src/java/org/apache/hadoop/io/DefaultStringifier.java
  5. 14 36
      src/java/org/apache/hadoop/io/SequenceFile.java
  6. 0 1
      src/java/org/apache/hadoop/io/serializer/Deserializer.java
  7. 0 46
      src/java/org/apache/hadoop/io/serializer/DeserializerBase.java
  8. 0 7
      src/java/org/apache/hadoop/io/serializer/DeserializerComparator.java
  9. 9 49
      src/java/org/apache/hadoop/io/serializer/JavaSerialization.java
  10. 0 47
      src/java/org/apache/hadoop/io/serializer/LegacyDeserializer.java
  11. 0 96
      src/java/org/apache/hadoop/io/serializer/LegacySerialization.java
  12. 0 54
      src/java/org/apache/hadoop/io/serializer/LegacySerializer.java
  13. 0 1
      src/java/org/apache/hadoop/io/serializer/Serialization.java
  14. 0 117
      src/java/org/apache/hadoop/io/serializer/SerializationBase.java
  15. 12 50
      src/java/org/apache/hadoop/io/serializer/SerializationFactory.java
  16. 0 42
      src/java/org/apache/hadoop/io/serializer/SerializerBase.java
  17. 13 62
      src/java/org/apache/hadoop/io/serializer/WritableSerialization.java
  18. 0 48
      src/java/org/apache/hadoop/io/serializer/avro/AvroComparator.java
  19. 0 64
      src/java/org/apache/hadoop/io/serializer/avro/AvroGenericSerialization.java
  20. 6 16
      src/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerialization.java
  21. 24 47
      src/java/org/apache/hadoop/io/serializer/avro/AvroSerialization.java
  22. 7 16
      src/java/org/apache/hadoop/io/serializer/avro/AvroSpecificSerialization.java
  23. 4 6
      src/java/org/apache/hadoop/util/ReflectionUtils.java
  24. 0 44
      src/test/core/org/apache/hadoop/conf/TestConfiguration.java
  25. 6 22
      src/test/core/org/apache/hadoop/io/serializer/SerializationTestUtil.java
  26. 0 175
      src/test/core/org/apache/hadoop/io/serializer/TestRawComparators.java
  27. 0 127
      src/test/core/org/apache/hadoop/io/serializer/TestWritableSerialization.java
  28. 0 41
      src/test/core/org/apache/hadoop/io/serializer/avro/TestAvroSerialization.java

+ 0 - 11
CHANGES.txt

@@ -32,9 +32,6 @@ Trunk (unreleased changes)
     HADOOP-6392. Run namenode and jobtracker on separate EC2 instances.
     (tomwhite)
 
-    HADOOP-6323. Add comparators to the serialization API.
-    (Aaron Kimball via cutting)
-
     HADOOP-6433. Introduce asychronous deletion of files via a pool of
     threads. This can be used to delete files in the Distributed
     Cache. (Zheng Shao via dhruba)
@@ -129,18 +126,12 @@ Trunk (unreleased changes)
     HADOOP-6472. add tokenCache option to GenericOptionsParser for passing
      file with secret keys to a map reduce job. (boryas)
 
-    HADOOP-6443. Serialization classes accept invalid metadata.
-    (Aaron Kimball via tomwhite)
-
     HADOOP-3205. Read multiple chunks directly from FSInputChecker subclass
     into user buffers. (Todd Lipcon via tomwhite)
 
     HADOOP-6479. TestUTF8 assertions could fail with better text.
     (Steve Loughran via tomwhite)
 
-    HADOOP-6420. Add functionality permitting subsets of Configuration to be
-    interpreted as Map<String,String>. (Aaron Kimball via cdouglas)
-
     HADOOP-6155. Deprecate RecordIO anticipating Avro. (Tom White via cdouglas)
 
     HADOOP-6492. Make some Avro serialization APIs public.
@@ -564,8 +555,6 @@ Release 0.21.0 - Unreleased
     the io package and makes it available to other users (MAPREDUCE-318). 
     (Jothi Padmanabhan via ddas)
 
-    HADOOP-6165. Add metadata to Serializations. (tomwhite)
-
     HADOOP-6105. Adds support for automatically handling deprecation of
     configuration keys. (V.V.Chaitanya Krishna via yhemanth)
     

+ 1 - 1
src/java/core-default.xml

@@ -124,7 +124,7 @@
 
 <property>
   <name>io.serializations</name>
-  <value>org.apache.hadoop.io.serializer.WritableSerialization,org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization,org.apache.hadoop.io.serializer.avro.AvroReflectSerialization,org.apache.hadoop.io.serializer.avro.AvroGenericSerialization</value>
+  <value>org.apache.hadoop.io.serializer.WritableSerialization,org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization,org.apache.hadoop.io.serializer.avro.AvroReflectSerialization</value>
   <description>A list of serialization classes that can be used for
   obtaining serializers and deserializers.</description>
 </property>

+ 0 - 133
src/java/org/apache/hadoop/conf/Configuration.java

@@ -31,7 +31,6 @@ import java.io.OutputStreamWriter;
 import java.io.Reader;
 import java.io.Writer;
 import java.net.URL;
-import java.util.AbstractMap;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -1047,138 +1046,6 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     set(name, StringUtils.arrayToString(values));
   }
 
-  /**
-   * Instantiates a map view over a subset of the entries in
-   * the Configuration. This is instantiated by getMap(), which
-   * binds a prefix of the namespace to the ConfigItemMap. This
-   * mapping reflects changes to the underlying Configuration.
-   *
-   * This map does not support iteration.
-   */
-  protected class ConfigItemMap extends AbstractMap<String, String>
-      implements Map<String, String> {
-
-    private final String prefix;
-
-    public ConfigItemMap(String prefix) {
-      this.prefix = prefix;
-    }
-
-    @Override
-    public boolean containsKey(Object key) {
-      return lookup(key.toString()) != null;
-    }
-
-    @Override
-    public Set<Map.Entry<String, String>> entrySet() {
-      throw new UnsupportedOperationException("unsupported");
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      return o != null && o instanceof ConfigItemMap
-          && prefix.equals(((ConfigItemMap) o).prefix)
-          && Configuration.this == ((ConfigItemMap) o).getConfiguration();
-    }
-
-    private Configuration getConfiguration() {
-      return Configuration.this;
-    }
-
-    @Override
-    public String get(Object key) {
-      if (null == key) {
-        return null;
-      }
-
-      return lookup(key.toString());
-    }
-
-    @Override
-    public int hashCode() {
-      return prefix.hashCode();
-    }
-
-    @Override
-    public String put(String key, String val) {
-      if (null == key) {
-        return null;
-      }
-
-      String ret = get(key);
-      Configuration.this.set(prefix + key, val);
-      return ret;
-    }
-
-    @Override
-    public void putAll(Map<? extends String, ? extends String> m) {
-      for (Map.Entry<? extends String, ? extends String> entry : m.entrySet()) {
-        put(entry.getKey(), entry.getValue());
-      }
-    }
-
-    private String lookup(String subKey) {
-      String configKey = prefix + subKey;
-      Properties props = Configuration.this.getProps();
-      Object val = props.get(configKey);
-      String str = null;
-      if (null != val) {
-        str = substituteVars(val.toString());
-      }
-
-      return str;
-    }
-  }
-
-  /**
-   * Given a string -&gt; string map as a value, embed this in the
-   * Configuration by prepending 'name' to all the keys in the valueMap,
-   * and storing it inside the current Configuration.
-   *
-   * e.g., setMap("foo", { "bar" -&gt; "a", "baz" -&gt; "b" }) would
-   * insert "foo.bar" -&gt; "a" and "foo.baz" -&gt; "b" in this
-   * Configuration.
-   *
-   * @param name the prefix to attach to all keys in the valueMap. This
-   * should not have a trailing "." character.
-   * @param valueMap the map to embed in the Configuration.
-   */
-  public void setMap(String name, Map<String, String> valueMap) {
-    // Store all elements of the map proper.
-    for (Map.Entry<String, String> entry : valueMap.entrySet()) {
-      set(name + "." + entry.getKey(), entry.getValue());
-    }
-  }
-
-  /**
-   * Returns a map containing a view of all configuration properties
-   * whose names begin with "name.*", with the "name." prefix  removed.
-   * e.g., if "foo.bar" -&gt; "a" and "foo.baz" -&gt; "b" are in the
-   * Configuration, getMap("foo") would return { "bar" -&gt; "a",
-   * "baz" -&gt; "b" }.
-   *
-   * Map name deprecation is handled via "prefix deprecation"; the individual
-   * keys created in a configuration by inserting a map do not need to be
-   * individually deprecated -- it is sufficient to deprecate the 'name'
-   * associated with the map and bind that to a new name. e.g., if "foo"
-   * is deprecated for "newfoo," and the configuration contains entries for
-   * "newfoo.a" and "newfoo.b", getMap("foo") will return a map containing
-   * the keys "a" and "b".
-   *
-   * The returned map does not support iteration; it is a lazy view over
-   * the slice of the configuration whose keys begin with 'name'. Updates
-   * to the underlying configuration are reflected in the returned map,
-   * and updates to the map will modify the underlying configuration.
-   *
-   * @param name The prefix of the key names to extract into the output map.
-   * @return a String-&gt;String map that contains all (k, v) pairs
-   * where 'k' begins with 'name.'; the 'name.' prefix is removed in the output.
-   */
-  public Map<String, String> getMap(String name) {
-    String prefix = handleDeprecation(name) + ".";
-    return new ConfigItemMap(prefix);
-  }
-
   /**
    * Load a class by name.
    * 

+ 13 - 15
src/java/org/apache/hadoop/io/DefaultStringifier.java

@@ -21,21 +21,20 @@ package org.apache.hadoop.io;
 import java.io.IOException;
 import java.nio.charset.UnsupportedCharsetException;
 import java.util.ArrayList;
-import java.util.Map;
 
 import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.serializer.DeserializerBase;
-import org.apache.hadoop.io.serializer.SerializationBase;
+import org.apache.hadoop.io.serializer.Deserializer;
+import org.apache.hadoop.io.serializer.Serialization;
 import org.apache.hadoop.io.serializer.SerializationFactory;
-import org.apache.hadoop.io.serializer.SerializerBase;
+import org.apache.hadoop.io.serializer.Serializer;
 import org.apache.hadoop.util.GenericsUtil;
 
 /**
  * DefaultStringifier is the default implementation of the {@link Stringifier}
  * interface which stringifies the objects using base64 encoding of the
- * serialized version of the objects. The {@link SerializerBase} and
- * {@link DeserializerBase} are obtained from the {@link SerializationFactory}.
+ * serialized version of the objects. The {@link Serializer} and
+ * {@link Deserializer} are obtained from the {@link SerializationFactory}.
  * <br>
  * DefaultStringifier offers convenience methods to store/load objects to/from
  * the configuration.
@@ -46,9 +45,9 @@ public class DefaultStringifier<T> implements Stringifier<T> {
 
   private static final String SEPARATOR = ",";
 
-  private SerializerBase<T> serializer;
+  private Serializer<T> serializer;
 
-  private DeserializerBase<T> deserializer;
+  private Deserializer<T> deserializer;
 
   private DataInputBuffer inBuf;
 
@@ -57,9 +56,8 @@ public class DefaultStringifier<T> implements Stringifier<T> {
   public DefaultStringifier(Configuration conf, Class<T> c) {
 
     SerializationFactory factory = new SerializationFactory(conf);
-    Map<String, String> metadata = SerializationBase.getMetadataFromClass(c);
-    this.serializer = factory.getSerializer(metadata);
-    this.deserializer = factory.getDeserializer(metadata);
+    this.serializer = factory.getSerializer(c);
+    this.deserializer = factory.getDeserializer(c);
     this.inBuf = new DataInputBuffer();
     this.outBuf = new DataOutputBuffer();
     try {
@@ -104,7 +102,7 @@ public class DefaultStringifier<T> implements Stringifier<T> {
    * @param item the object to be stored
    * @param keyName the name of the key to use
    * @throws IOException : forwards Exceptions from the underlying 
-   * {@link SerializationBase} classes. 
+   * {@link Serialization} classes. 
    */
   public static <K> void store(Configuration conf, K item, String keyName)
   throws IOException {
@@ -124,7 +122,7 @@ public class DefaultStringifier<T> implements Stringifier<T> {
    * @param itemClass the class of the item
    * @return restored object
    * @throws IOException : forwards Exceptions from the underlying 
-   * {@link SerializationBase} classes.
+   * {@link Serialization} classes.
    */
   public static <K> K load(Configuration conf, String keyName,
       Class<K> itemClass) throws IOException {
@@ -147,7 +145,7 @@ public class DefaultStringifier<T> implements Stringifier<T> {
    * @param keyName the name of the key to use
    * @throws IndexOutOfBoundsException if the items array is empty
    * @throws IOException : forwards Exceptions from the underlying 
-   * {@link SerializationBase} classes.         
+   * {@link Serialization} classes.         
    */
   public static <K> void storeArray(Configuration conf, K[] items,
       String keyName) throws IOException {
@@ -175,7 +173,7 @@ public class DefaultStringifier<T> implements Stringifier<T> {
    * @param itemClass the class of the item
    * @return restored object
    * @throws IOException : forwards Exceptions from the underlying 
-   * {@link SerializationBase} classes.
+   * {@link Serialization} classes.
    */
   public static <K> K[] loadArray(Configuration conf, String keyName,
       Class<K> itemClass) throws IOException {

+ 14 - 36
src/java/org/apache/hadoop/io/SequenceFile.java

@@ -33,9 +33,8 @@ import org.apache.hadoop.io.compress.Decompressor;
 import org.apache.hadoop.io.compress.DefaultCodec;
 import org.apache.hadoop.io.compress.GzipCodec;
 import org.apache.hadoop.io.compress.zlib.ZlibFactory;
-import org.apache.hadoop.io.serializer.DeserializerBase;
-import org.apache.hadoop.io.serializer.SerializationBase;
-import org.apache.hadoop.io.serializer.SerializerBase;
+import org.apache.hadoop.io.serializer.Deserializer;
+import org.apache.hadoop.io.serializer.Serializer;
 import org.apache.hadoop.io.serializer.SerializationFactory;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.util.Progressable;
@@ -706,14 +705,6 @@ public class SequenceFile {
       return new TreeMap<Text, Text>(this.theMetadata);
     }
     
-    public Map<String, String> getMetadataAsStringMap() {
-      Map<String, String> map = new HashMap<String, String>();
-      for (Map.Entry<Text, Text> entry : theMetadata.entrySet()) {
-        map.put(entry.getKey().toString(), entry.getValue().toString());
-      }
-      return map;
-    }
-    
     public void write(DataOutput out) throws IOException {
       out.writeInt(this.theMetadata.size());
       Iterator<Map.Entry<Text, Text>> iter =
@@ -810,9 +801,9 @@ public class SequenceFile {
     Metadata metadata = null;
     Compressor compressor = null;
     
-    protected SerializerBase keySerializer;
-    protected SerializerBase uncompressedValSerializer;
-    protected SerializerBase compressedValSerializer;
+    protected Serializer keySerializer;
+    protected Serializer uncompressedValSerializer;
+    protected Serializer compressedValSerializer;
     
     // Insert a globally unique 16-byte value every few entries, so that one
     // can seek into the middle of a file and then synchronize with record
@@ -923,10 +914,9 @@ public class SequenceFile {
       this.codec = codec;
       this.metadata = metadata;
       SerializationFactory serializationFactory = new SerializationFactory(conf);
-      this.keySerializer = getSerializer(serializationFactory, keyClass, metadata);
+      this.keySerializer = serializationFactory.getSerializer(keyClass);
       this.keySerializer.open(buffer);
-      this.uncompressedValSerializer = getSerializer(serializationFactory,
-        valClass, metadata);
+      this.uncompressedValSerializer = serializationFactory.getSerializer(valClass);
       this.uncompressedValSerializer.open(buffer);
       if (this.codec != null) {
         ReflectionUtils.setConf(this.codec, this.conf);
@@ -934,20 +924,11 @@ public class SequenceFile {
         this.deflateFilter = this.codec.createOutputStream(buffer, compressor);
         this.deflateOut = 
           new DataOutputStream(new BufferedOutputStream(deflateFilter));
-        this.compressedValSerializer = getSerializer(serializationFactory,
-          valClass, metadata);
+        this.compressedValSerializer = serializationFactory.getSerializer(valClass);
         this.compressedValSerializer.open(deflateOut);
       }
     }
     
-    @SuppressWarnings("unchecked")
-    private SerializerBase getSerializer(SerializationFactory sf, Class c,
-	Metadata metadata) {
-      Map<String, String> stringMetadata = metadata.getMetadataAsStringMap();
-      stringMetadata.put(SerializationBase.CLASS_KEY, c.getName());
-      return sf.getSerializer(stringMetadata);
-    }
-    
     /** Returns the class of keys in this file. */
     public Class getKeyClass() { return keyClass; }
 
@@ -1432,8 +1413,8 @@ public class SequenceFile {
     private DataInputStream valIn = null;
     private Decompressor valDecompressor = null;
     
-    private DeserializerBase keyDeserializer;
-    private DeserializerBase valDeserializer;
+    private Deserializer keyDeserializer;
+    private Deserializer valDeserializer;
 
     /**
      * Construct a reader by opening a file from the given file system.
@@ -1630,24 +1611,21 @@ public class SequenceFile {
         SerializationFactory serializationFactory =
           new SerializationFactory(conf);
         this.keyDeserializer =
-          getDeserializer(serializationFactory, getKeyClass(), metadata);
+          getDeserializer(serializationFactory, getKeyClass());
         if (!blockCompressed) {
           this.keyDeserializer.open(valBuffer);
         } else {
           this.keyDeserializer.open(keyIn);
         }
         this.valDeserializer =
-          getDeserializer(serializationFactory, getValueClass(), metadata);
+          getDeserializer(serializationFactory, getValueClass());
         this.valDeserializer.open(valIn);
       }
     }
     
     @SuppressWarnings("unchecked")
-    private DeserializerBase getDeserializer(SerializationFactory sf, Class c,
-	Metadata metadata) {
-      Map<String, String> stringMetadata = metadata.getMetadataAsStringMap();
-      stringMetadata.put(SerializationBase.CLASS_KEY, c.getName());
-      return sf.getDeserializer(stringMetadata);
+    private Deserializer getDeserializer(SerializationFactory sf, Class c) {
+      return sf.getDeserializer(c);
     }
     
     /** Close the file. */

+ 0 - 1
src/java/org/apache/hadoop/io/serializer/Deserializer.java

@@ -34,7 +34,6 @@ import java.io.InputStream;
  * </p>
  * @param <T>
  */
-@Deprecated
 public interface Deserializer<T> {
   /**
    * <p>Prepare the deserializer for reading.</p>

+ 0 - 46
src/java/org/apache/hadoop/io/serializer/DeserializerBase.java

@@ -1,46 +0,0 @@
-/**
- * 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.serializer;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.hadoop.conf.Configured;
-
-public abstract class DeserializerBase<T> extends Configured
-  implements Closeable, Deserializer<T> {
-  
-  /**
-   * <p>Prepare the deserializer for reading.</p>
-   */
-  public abstract void open(InputStream in) throws IOException;
-  
-  /**
-   * <p>
-   * Deserialize the next object from the underlying input stream.
-   * If the object <code>t</code> is non-null then this deserializer
-   * <i>may</i> set its internal state to the next object read from the input
-   * stream. Otherwise, if the object <code>t</code> is null a new
-   * deserialized object will be created.
-   * </p>
-   * @return the deserialized object
-   */
-  public abstract T deserialize(T t) throws IOException;
-  
-}

+ 0 - 7
src/java/org/apache/hadoop/io/serializer/DeserializerComparator.java

@@ -52,13 +52,6 @@ public abstract class DeserializerComparator<T> implements RawComparator<T> {
     this.deserializer.open(buffer);
   }
 
-  protected DeserializerComparator(DeserializerBase<T> deserializer)
-    throws IOException {
-    
-    this.deserializer = deserializer;
-    this.deserializer.open(buffer);
-  }
-
   public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
     try {
       

+ 9 - 49
src/java/org/apache/hadoop/io/serializer/JavaSerialization.java

@@ -34,10 +34,10 @@ import org.apache.hadoop.io.RawComparator;
  * </p>
  * @see JavaSerializationComparator
  */
-public class JavaSerialization extends SerializationBase<Serializable> {
+public class JavaSerialization implements Serialization<Serializable> {
 
   static class JavaSerializationDeserializer<T extends Serializable>
-    extends DeserializerBase<T> {
+    implements Deserializer<T> {
 
     private ObjectInputStream ois;
 
@@ -65,15 +65,10 @@ public class JavaSerialization extends SerializationBase<Serializable> {
 
   }
 
-  static class JavaSerializationSerializer<T extends Serializable>
-      extends SerializerBase<T> {
+  static class JavaSerializationSerializer
+    implements Serializer<Serializable> {
 
     private ObjectOutputStream oos;
-    private Map<String, String> metadata;
-
-    public JavaSerializationSerializer(Map<String, String> metadata) {
-      this.metadata = metadata;
-    }
 
     public void open(OutputStream out) throws IOException {
       oos = new ObjectOutputStream(out) {
@@ -83,7 +78,7 @@ public class JavaSerialization extends SerializationBase<Serializable> {
       };
     }
 
-    public void serialize(T object) throws IOException {
+    public void serialize(Serializable object) throws IOException {
       oos.reset(); // clear (class) back-references
       oos.writeObject(object);
     }
@@ -92,53 +87,18 @@ public class JavaSerialization extends SerializationBase<Serializable> {
       oos.close();
     }
 
-    @Override
-    public Map<String, String> getMetadata() throws IOException {
-      return metadata;
-    }
   }
 
-  public boolean accept(Map<String, String> metadata) {
-    if (!checkSerializationKey(metadata)) {
-      return false;
-    }
-
-    Class<?> c = getClassFromMetadata(metadata);
+  public boolean accept(Class<?> c) {
     return Serializable.class.isAssignableFrom(c);
   }
 
-  public DeserializerBase<Serializable> getDeserializer(
-      Map<String, String> metadata) {
+  public Deserializer<Serializable> getDeserializer(Class<Serializable> c) {
     return new JavaSerializationDeserializer<Serializable>();
   }
 
-  public SerializerBase<Serializable> getSerializer(
-      Map<String, String> metadata) {
-    return new JavaSerializationSerializer<Serializable>(metadata);
+  public Serializer<Serializable> getSerializer(Class<Serializable> c) {
+    return new JavaSerializationSerializer();
   }
 
-  @SuppressWarnings("unchecked")
-  @Override
-  public RawComparator<Serializable> getRawComparator(
-      Map<String, String> metadata) {
-    Class<?> klazz = getClassFromMetadata(metadata);
-    if (null == klazz) {
-      throw new IllegalArgumentException(
-          "Cannot get comparator without " + SerializationBase.CLASS_KEY
-          + " set in metadata");
-    }
-
-    if (Serializable.class.isAssignableFrom(klazz)) {
-      try {
-        return (RawComparator<Serializable>) new JavaSerializationComparator();
-      } catch (IOException ioe) {
-        throw new IllegalArgumentException(
-            "Could not instantiate JavaSerializationComparator for type "
-            + klazz.getName(), ioe);
-      }
-    } else {
-      throw new IllegalArgumentException("Class " + klazz.getName()
-          + " is incompatible with JavaSerialization");
-    }
-  }
 }

+ 0 - 47
src/java/org/apache/hadoop/io/serializer/LegacyDeserializer.java

@@ -1,47 +0,0 @@
-/**
- * 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.serializer;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-@SuppressWarnings("deprecation")
-class LegacyDeserializer<T> extends DeserializerBase<T> {
-  
-  private Deserializer<T> deserializer;
-
-  public LegacyDeserializer(Deserializer<T> deserializer) {
-    this.deserializer = deserializer;
-  }
-
-  @Override
-  public void open(InputStream in) throws IOException {
-    deserializer.open(in);
-  }
-  
-  @Override
-  public T deserialize(T t) throws IOException {
-    return deserializer.deserialize(t);
-  }
-
-  @Override
-  public void close() throws IOException {
-    deserializer.close();
-  }
-
-}

+ 0 - 96
src/java/org/apache/hadoop/io/serializer/LegacySerialization.java

@@ -1,96 +0,0 @@
-/**
- * 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.serializer;
-
-import java.util.Map;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.RawComparator;
-
-/**
- * <p>
- * Wraps a legacy {@link Serialization} as a {@link SerializationBase}.
- * </p>
- * 
- * @param <T>
- */
-@SuppressWarnings("deprecation")
-class LegacySerialization<T> extends SerializationBase<T> {
-
-  private Serialization<T> serialization;
-
-  public LegacySerialization(Serialization<T> serialization,
-      Configuration conf) {
-    this.serialization = serialization;
-    setConf(conf);
-  }
-  
-  Serialization<T> getUnderlyingSerialization() {
-    return serialization;
-  }
-
-  @Deprecated
-  @Override
-  public boolean accept(Class<?> c) {
-    return serialization.accept(c);
-  }
-
-  @Deprecated
-  @Override
-  public Deserializer<T> getDeserializer(Class<T> c) {
-    return serialization.getDeserializer(c);
-  }
-
-  @Deprecated
-  @Override
-  public Serializer<T> getSerializer(Class<T> c) {
-    return serialization.getSerializer(c);
-  }
-  
-  @Override
-  public boolean accept(Map<String, String> metadata) {
-    Class<?> c = getClassFromMetadata(metadata);
-    return accept(c);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Override
-  public SerializerBase<T> getSerializer(Map<String, String> metadata) {
-    Class<T> c = (Class<T>) getClassFromMetadata(metadata);
-    return new LegacySerializer<T>(getSerializer(c));
-  }
-  
-  @SuppressWarnings("unchecked")
-  @Override
-  public DeserializerBase<T> getDeserializer(Map<String, String> metadata) {
-    Class<T> c = (Class<T>) getClassFromMetadata(metadata);
-    return new LegacyDeserializer<T>(getDeserializer(c));
-  }
-
-  @Override
-  public RawComparator<T> getRawComparator(Map<String, String> metadata) {
-    // Since this method is being added to an API meant to provide legacy
-    // compatability with deprecated serializers, leaving this as an incomplete
-    // stub.
-
-    throw new UnsupportedOperationException(
-        "LegacySerialization does not provide raw comparators");
-  }
-
-}

+ 0 - 54
src/java/org/apache/hadoop/io/serializer/LegacySerializer.java

@@ -1,54 +0,0 @@
-/**
- * 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.serializer;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.Collections;
-import java.util.Map;
-
-@SuppressWarnings("deprecation")
-class LegacySerializer<T> extends SerializerBase<T> {
-  
-  private Serializer<T> serializer;
-
-  public LegacySerializer(Serializer<T> serializer) {
-    this.serializer = serializer;
-  }
-
-  @Override
-  public void open(OutputStream out) throws IOException {
-    serializer.open(out);
-  }
-
-  @Override
-  public void serialize(T t) throws IOException {
-    serializer.serialize(t);
-  }
-
-  @Override
-  public void close() throws IOException {
-    serializer.close();
-  }
-
-  @Override
-  public Map<String, String> getMetadata() throws IOException {
-    return Collections.<String, String>emptyMap();
-  }
-
-}

+ 0 - 1
src/java/org/apache/hadoop/io/serializer/Serialization.java

@@ -24,7 +24,6 @@ package org.apache.hadoop.io.serializer;
  * </p>
  * @param <T>
  */
-@Deprecated
 public interface Serialization<T> {
   
   /**

+ 0 - 117
src/java/org/apache/hadoop/io/serializer/SerializationBase.java

@@ -1,117 +0,0 @@
-/**
- * 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.serializer;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.io.RawComparator;
-
-/**
- * <p>
- * Encapsulates a {@link SerializerBase}/{@link DeserializerBase} pair.
- * </p>
- * 
- * @param <T>
- */
-public abstract class SerializationBase<T> extends Configured
-  implements Serialization<T> {
-    
-  public static final String SERIALIZATION_KEY = "Serialization-Class";
-  public static final String CLASS_KEY = "Serialized-Class";
-  
-  public static Map<String, String> getMetadataFromClass(Class<?> c) {
-    Map<String, String> metadata = new HashMap<String, String>();
-    metadata.put(CLASS_KEY, c.getName());
-    return metadata;
-  }
-  
-  @Deprecated
-  @Override
-  public boolean accept(Class<?> c) {
-    return accept(getMetadataFromClass(c));
-  }
-
-  @Deprecated
-  @Override
-  public Deserializer<T> getDeserializer(Class<T> c) {
-    return getDeserializer(getMetadataFromClass(c));
-  }
-
-  @Deprecated
-  @Override
-  public Serializer<T> getSerializer(Class<T> c) {
-    return getSerializer(getMetadataFromClass(c));
-  }
-
-  /**
-   * Allows clients to test whether this {@link SerializationBase} supports the
-   * given metadata.
-   */
-  public abstract boolean accept(Map<String, String> metadata);
-
-  /**
-   * @return a {@link SerializerBase} for the given metadata.
-   */
-  public abstract SerializerBase<T> getSerializer(Map<String, String> metadata);
-
-  /**
-   * @return a {@link DeserializerBase} for the given metadata.
-   */
-  public abstract DeserializerBase<T> getDeserializer(
-      Map<String, String> metadata);
-
-  public Class<?> getClassFromMetadata(Map<String, String> metadata) {
-    String classname = metadata.get(CLASS_KEY);
-    if (classname == null) {
-      return null;
-    }
-    try {
-      return getConf().getClassByName(classname);
-    } catch (ClassNotFoundException e) {
-      throw new IllegalArgumentException(e);
-    }
-  }
-
-  /** Provide a raw comparator for the specified serializable class.
-   * Requires a serialization-specific metadata entry to name the class
-   * to compare (e.g., "Serialized-Class" for JavaSerialization and
-   * WritableSerialization).
-   * @param metadata a set of string mappings providing serialization-specific
-   * arguments that parameterize the data being serialized/compared.
-   * @return a {@link RawComparator} for the given metadata.
-   * @throws UnsupportedOperationException if it cannot instantiate a RawComparator
-   * for this given metadata.
-   */
-  public abstract RawComparator<T> getRawComparator(Map<String,String> metadata);
-
-  /**
-   * Check that the SERIALIZATION_KEY, if set, matches the current class.
-   * @param metadata the serialization metadata to check.
-   * @return true if SERIALIZATION_KEY is unset, or if it matches the current class
-   * (meaning that accept() should continue processing), or false if it is a mismatch,
-   * meaning that accept() should return false.
-   */
-  protected boolean checkSerializationKey(Map<String, String> metadata) {
-    String intendedSerializer = metadata.get(SERIALIZATION_KEY);
-    return intendedSerializer == null ||
-        getClass().getName().equals(intendedSerializer);
-  }
-}

+ 12 - 50
src/java/org/apache/hadoop/io/serializer/SerializationFactory.java

@@ -20,13 +20,11 @@ package org.apache.hadoop.io.serializer;
 
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.io.serializer.avro.AvroGenericSerialization;
 import org.apache.hadoop.io.serializer.avro.AvroReflectSerialization;
 import org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization;
 import org.apache.hadoop.util.ReflectionUtils;
@@ -34,7 +32,7 @@ import org.apache.hadoop.util.StringUtils;
 
 /**
  * <p>
- * A factory for {@link SerializationBase}s.
+ * A factory for {@link Serialization}s.
  * </p>
  */
 public class SerializationFactory extends Configured {
@@ -42,10 +40,7 @@ public class SerializationFactory extends Configured {
   private static final Log LOG =
     LogFactory.getLog(SerializationFactory.class.getName());
 
-  private List<SerializationBase<?>> serializations =
-    new ArrayList<SerializationBase<?>>();
-  private List<SerializationBase<?>> legacySerializations =
-    new ArrayList<SerializationBase<?>>();
+  private List<Serialization<?>> serializations = new ArrayList<Serialization<?>>();
   
   /**
    * <p>
@@ -59,8 +54,7 @@ public class SerializationFactory extends Configured {
     for (String serializerName : conf.getStrings("io.serializations", 
       new String[]{WritableSerialization.class.getName(), 
         AvroSpecificSerialization.class.getName(), 
-        AvroReflectSerialization.class.getName(),
-        AvroGenericSerialization.class.getName()})) {
+        AvroReflectSerialization.class.getName()})) {
       add(conf, serializerName);
     }
   }
@@ -68,64 +62,32 @@ public class SerializationFactory extends Configured {
   @SuppressWarnings("unchecked")
   private void add(Configuration conf, String serializationName) {
     try {
-      Class<?> serializationClass = conf.getClassByName(serializationName);
-      if (SerializationBase.class.isAssignableFrom(serializationClass)) {
-	serializations.add((SerializationBase)
-	    ReflectionUtils.newInstance(serializationClass, getConf()));	
-      } else if (Serialization.class.isAssignableFrom(serializationClass)) {
-	Serialization serialization = (Serialization)
-	    ReflectionUtils.newInstance(serializationClass, getConf());
-	legacySerializations.add(new LegacySerialization(serialization,
-	    getConf()));	
-      } else {
-	LOG.warn("Serialization class " + serializationName + " is not an " +
-			"instance of Serialization or BaseSerialization.");
-      }
+      Class<? extends Serialization> serializionClass =
+	(Class<? extends Serialization>) conf.getClassByName(serializationName);
+      serializations.add((Serialization)
+	  ReflectionUtils.newInstance(serializionClass, getConf()));
     } catch (ClassNotFoundException e) {
       LOG.warn("Serialization class not found: " +
           StringUtils.stringifyException(e));
     }
   }
 
-  @Deprecated
   public <T> Serializer<T> getSerializer(Class<T> c) {
     return getSerialization(c).getSerializer(c);
   }
 
-  @Deprecated
   public <T> Deserializer<T> getDeserializer(Class<T> c) {
     return getSerialization(c).getDeserializer(c);
   }
 
-  @Deprecated
-  public <T> Serialization<T> getSerialization(Class<T> c) {
-    return getSerialization(SerializationBase.getMetadataFromClass(c));
-  }
-  
-  public <T> SerializerBase<T> getSerializer(Map<String, String> metadata) {
-    SerializationBase<T> serialization = getSerialization(metadata);
-    return serialization.getSerializer(metadata);
-  }
-    
-  public <T> DeserializerBase<T> getDeserializer(Map<String, String> metadata) {
-    SerializationBase<T> serialization = getSerialization(metadata);
-    return serialization.getDeserializer(metadata);
-  }
-    
   @SuppressWarnings("unchecked")
-  public <T> SerializationBase<T> getSerialization(Map<String, String> metadata) {
-    for (SerializationBase serialization : serializations) {
-      if (serialization.accept(metadata)) {
-        return (SerializationBase<T>) serialization;
-      }
-    }
-    // Look in the legacy serializations last, since they ignore
-    // non-class metadata
-    for (SerializationBase serialization : legacySerializations) {
-      if (serialization.accept(metadata)) {
-        return (SerializationBase<T>) serialization;
+  public <T> Serialization<T> getSerialization(Class<T> c) {
+    for (Serialization serialization : serializations) {
+      if (serialization.accept(c)) {
+	return (Serialization<T>) serialization;
       }
     }
     return null;
   }
+  
 }

+ 0 - 42
src/java/org/apache/hadoop/io/serializer/SerializerBase.java

@@ -1,42 +0,0 @@
-/**
- * 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.serializer;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.Map;
-
-import org.apache.hadoop.conf.Configured;
-
-public abstract class SerializerBase<T> extends Configured
-  implements Closeable, Serializer<T> {
-
-  /**
-   * <p>Prepare the serializer for writing.</p>
-   */
-  public abstract void open(OutputStream out) throws IOException;
-  
-  /**
-   * <p>Serialize <code>t</code> to the underlying output stream.</p>
-   */
-  public abstract void serialize(T t) throws IOException;
-  
-  public abstract Map<String, String> getMetadata() throws IOException;
-  
-}

+ 13 - 62
src/java/org/apache/hadoop/io/serializer/WritableSerialization.java

@@ -26,20 +26,19 @@ import java.io.OutputStream;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.WritableComparator;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.util.StringUtils;
 
 /**
- * A {@link SerializationBase} for {@link Writable}s that delegates to
+ * A {@link Serialization} for {@link Writable}s that delegates to
  * {@link Writable#write(java.io.DataOutput)} and
  * {@link Writable#readFields(java.io.DataInput)}.
  */
-public class WritableSerialization extends SerializationBase<Writable> {
-  static class WritableDeserializer extends DeserializerBase<Writable> {
+public class WritableSerialization extends Configured
+	implements Serialization<Writable> {
+  static class WritableDeserializer extends Configured
+  	implements Deserializer<Writable> {
 
     private Class<?> writableClass;
     private DataInputStream dataIn;
@@ -78,30 +77,10 @@ public class WritableSerialization extends SerializationBase<Writable> {
     
   }
   
-  static class WritableSerializer extends SerializerBase<Writable> {
+  static class WritableSerializer extends Configured implements
+  	Serializer<Writable> {
     
-    private Map<String, String> metadata;
     private DataOutputStream dataOut;
-    private Class<?> serializedClass;
-    
-    public WritableSerializer(Configuration conf,
-        Map<String, String> metadata) {
-      this.metadata = metadata;
-
-      // If this metadata specifies a serialized class, memoize the
-      // class object for this.
-      String className = this.metadata.get(CLASS_KEY);
-      if (null != className) {
-        try {
-          this.serializedClass = conf.getClassByName(className);
-        } catch (ClassNotFoundException cnfe) {
-          throw new RuntimeException(cnfe);
-        }
-      } else {
-        throw new UnsupportedOperationException("the "
-            + CLASS_KEY + " metadata is missing, but is required.");
-      }
-    }
     
     @Override
     public void open(OutputStream out) {
@@ -114,10 +93,6 @@ public class WritableSerialization extends SerializationBase<Writable> {
 
     @Override
     public void serialize(Writable w) throws IOException {
-      if (serializedClass != w.getClass()) {
-        throw new IOException("Type mismatch in serialization: expected "
-            + serializedClass + "; received " + w.getClass());
-      }
       w.write(dataOut);
     }
 
@@ -126,45 +101,21 @@ public class WritableSerialization extends SerializationBase<Writable> {
       dataOut.close();
     }
 
-    @Override
-    public Map<String, String> getMetadata() throws IOException {
-      return metadata;
-    }
-
   }
 
   @Override
-  public boolean accept(Map<String, String> metadata) {
-    if (!checkSerializationKey(metadata)) {
-      return false;
-    }
-
-    Class<?> c = getClassFromMetadata(metadata);
-    return c == null ? false : Writable.class.isAssignableFrom(c);
+  public boolean accept(Class<?> c) {
+    return Writable.class.isAssignableFrom(c);
   }
 
   @Override
-  public SerializerBase<Writable> getSerializer(Map<String, String> metadata) {
-    return new WritableSerializer(getConf(), metadata);
+  public Serializer<Writable> getSerializer(Class<Writable> c) {
+    return new WritableSerializer();
   }
   
   @Override
-  public DeserializerBase<Writable> getDeserializer(Map<String, String> metadata) {
-    Class<?> c = getClassFromMetadata(metadata);
+  public Deserializer<Writable> getDeserializer(Class<Writable> c) {
     return new WritableDeserializer(getConf(), c);
   }
 
-  @Override
-  @SuppressWarnings("unchecked")
-  public RawComparator<Writable> getRawComparator(Map<String, String> metadata) {
-    Class<?> klazz = getClassFromMetadata(metadata);
-    if (null == klazz) {
-      throw new IllegalArgumentException(
-          "Cannot get comparator without " + SerializationBase.CLASS_KEY
-          + " set in metadata");
-    }
-
-    return (RawComparator) WritableComparator.get(
-        (Class<WritableComparable>)klazz);
-  }
 }

+ 0 - 48
src/java/org/apache/hadoop/io/serializer/avro/AvroComparator.java

@@ -1,48 +0,0 @@
-/**
- * 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.serializer.avro;
-
-import org.apache.avro.Schema;
-import org.apache.avro.io.BinaryData;
-import org.apache.hadoop.io.RawComparator;
-
-/**
- * <p>
- * A {@link RawComparator} that uses Avro to extract data from the
- * source stream and compare their contents without explicit
- * deserialization.
- */
-public class AvroComparator<T extends Comparable<T>>
-    implements RawComparator<T> {
-
-  private final Schema schema;
-
-  public AvroComparator(final Schema s) {
-    this.schema = s;
-  }
-
-  public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
-    return BinaryData.compare(b1, s1, b2, s2, schema);
-  }
-
-  public int compare(T t1, T t2) {
-    return t1.compareTo(t2);
-  }
-
-}

+ 0 - 64
src/java/org/apache/hadoop/io/serializer/avro/AvroGenericSerialization.java

@@ -1,64 +0,0 @@
-/**
- * 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.serializer.avro;
-
-import java.util.Map;
-
-import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericData;
-import org.apache.avro.generic.GenericDatumReader;
-import org.apache.avro.generic.GenericDatumWriter;
-import org.apache.avro.io.DatumReader;
-import org.apache.avro.io.DatumWriter;
-import org.apache.hadoop.io.serializer.SerializationBase;
-
-/**
- * Serialization for Avro Generic classes. For a class to be accepted by this 
- * serialization it must have a schema specified.
- * The schema used is the one set by {@link AvroSerialization#AVRO_SCHEMA_KEY}.
- */
-@SuppressWarnings("unchecked")
-public class AvroGenericSerialization extends AvroSerialization<Object> {
-
-  @Override
-  public boolean accept(Map<String, String> metadata) {
-    if (!checkSerializationKey(metadata)) {
-      return false;
-    }
-
-    return metadata.get(AVRO_SCHEMA_KEY) != null;
-  }
-
-  @Override
-  public DatumReader getReader(Map<String, String> metadata) {
-    Schema schema = Schema.parse(metadata.get(AVRO_SCHEMA_KEY));
-    return new GenericDatumReader<Object>(schema);
-  }
-
-  @Override
-  public Schema getSchema(Map<String, String> metadata) {
-    return Schema.parse(metadata.get(AVRO_SCHEMA_KEY));
-  }
-
-  @Override
-  public DatumWriter getWriter(Map<String, String> metadata) {
-    return new GenericDatumWriter<Object>();
-  }
-
-}

+ 6 - 16
src/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerialization.java

@@ -19,7 +19,6 @@
 package org.apache.hadoop.io.serializer.avro;
 
 import java.util.HashSet;
-import java.util.Map;
 import java.util.Set;
 
 import org.apache.avro.Schema;
@@ -28,7 +27,6 @@ import org.apache.avro.io.DatumWriter;
 import org.apache.avro.reflect.ReflectData;
 import org.apache.avro.reflect.ReflectDatumReader;
 import org.apache.avro.reflect.ReflectDatumWriter;
-import org.apache.avro.specific.SpecificRecord;
 
 /**
  * Serialization for Avro Reflect classes. For a class to be accepted by this 
@@ -50,17 +48,10 @@ public class AvroReflectSerialization extends AvroSerialization<Object>{
   private Set<String> packages; 
 
   @Override
-  public synchronized boolean accept(Map<String, String> metadata) {
+  public synchronized boolean accept(Class<?> c) {
     if (packages == null) {
       getPackages();
     }
-    if (!checkSerializationKey(metadata)) {
-      return false;
-    }
-    Class<?> c = getClassFromMetadata(metadata);
-    if (c == null) {
-      return false;
-    }
     return AvroReflectSerializable.class.isAssignableFrom(c) || 
       packages.contains(c.getPackage().getName());
   }
@@ -76,22 +67,21 @@ public class AvroReflectSerialization extends AvroSerialization<Object>{
   }
 
   @Override
-  public DatumReader getReader(Map<String, String> metadata) {
+  public DatumReader getReader(Class<Object> clazz) {
     try {
-      return new ReflectDatumReader(getClassFromMetadata(metadata));
+      return new ReflectDatumReader(clazz);
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
   }
 
   @Override
-  public Schema getSchema(Map<String, String> metadata) {
-    Class<?> c = getClassFromMetadata(metadata);
-    return ReflectData.get().getSchema(c);
+  public Schema getSchema(Object t) {
+    return ReflectData.get().getSchema(t.getClass());
   }
 
   @Override
-  public DatumWriter getWriter(Map<String, String> metadata) {
+  public DatumWriter getWriter(Class<Object> clazz) {
     return new ReflectDatumWriter();
   }
 

+ 24 - 47
src/java/org/apache/hadoop/io/serializer/avro/AvroSerialization.java

@@ -21,62 +21,57 @@ package org.apache.hadoop.io.serializer.avro;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.util.Map;
 
 import org.apache.avro.Schema;
-import org.apache.avro.io.DecoderFactory;
 import org.apache.avro.io.BinaryDecoder;
 import org.apache.avro.io.BinaryEncoder;
 import org.apache.avro.io.DatumReader;
 import org.apache.avro.io.DatumWriter;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.io.serializer.DeserializerBase;
-import org.apache.hadoop.io.serializer.SerializationBase;
-import org.apache.hadoop.io.serializer.SerializerBase;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.serializer.Deserializer;
+import org.apache.hadoop.io.serializer.Serialization;
+import org.apache.hadoop.io.serializer.Serializer;
 
 /**
  * Base class for providing serialization to Avro types.
  */
-public abstract class AvroSerialization<T> extends SerializationBase<T> {
+public abstract class AvroSerialization<T> extends Configured 
+	implements Serialization<T>{
   
   public static final String AVRO_SCHEMA_KEY = "Avro-Schema";
 
-  public DeserializerBase<T> getDeserializer(Map<String, String> metadata) {
-    return new AvroDeserializer(metadata);
+  public Deserializer<T> getDeserializer(Class<T> c) {
+    return new AvroDeserializer(c);
   }
 
-  public SerializerBase<T> getSerializer(Map<String, String> metadata) {
-    return new AvroSerializer(metadata);
+  public Serializer<T> getSerializer(Class<T> c) {
+    return new AvroSerializer(c);
   }
 
   /**
-   * Return an Avro Schema instance for the given class and metadata.
+   * Return an Avro Schema instance for the given class.
    */
-  public abstract Schema getSchema(Map<String, String> metadata);
+  public abstract Schema getSchema(T t);
 
   /**
-   * Create and return Avro DatumWriter for the given metadata.
+   * Create and return Avro DatumWriter for the given class.
    */
-  public abstract DatumWriter<T> getWriter(Map<String, String> metadata);
+  public abstract DatumWriter<T> getWriter(Class<T> clazz);
 
   /**
-   * Create and return Avro DatumReader for the given metadata.
+   * Create and return Avro DatumReader for the given class.
    */
-  public abstract DatumReader<T> getReader(Map<String, String> metadata);
+  public abstract DatumReader<T> getReader(Class<T> clazz);
 
-  class AvroSerializer extends SerializerBase<T> {
+  class AvroSerializer implements Serializer<T> {
 
-    private Map<String, String> metadata;
     private DatumWriter<T> writer;
     private BinaryEncoder encoder;
     private OutputStream outStream;
-    private Schema schema;
 
-    AvroSerializer(Map<String, String> metadata) {
-      this.metadata = metadata;
-      this.writer = getWriter(metadata);
-      this.schema = getSchema(this.metadata);
-      writer.setSchema(this.schema);
+    AvroSerializer(Class<T> clazz) {
+      this.writer = getWriter(clazz);
     }
 
     @Override
@@ -93,24 +88,20 @@ public abstract class AvroSerialization<T> extends SerializationBase<T> {
 
     @Override
     public void serialize(T t) throws IOException {
+      writer.setSchema(getSchema(t));
       writer.write(t, encoder);
     }
 
-    @Override
-    public Map<String, String> getMetadata() throws IOException {
-      return metadata;
-    }
-
   }
 
-  class AvroDeserializer extends DeserializerBase<T> {
+  class AvroDeserializer implements Deserializer<T> {
 
     private DatumReader<T> reader;
     private BinaryDecoder decoder;
     private InputStream inStream;
 
-    AvroDeserializer(Map<String, String> metadata) {
-      this.reader = getReader(metadata);
+    AvroDeserializer(Class<T> clazz) {
+      this.reader = getReader(clazz);
     }
 
     @Override
@@ -131,18 +122,4 @@ public abstract class AvroSerialization<T> extends SerializationBase<T> {
 
   }
 
-  @Override
-  @SuppressWarnings("unchecked")
-  /**
-   * Provides a raw comparator for Avro-encoded serialized data.
-   * Requires that {@link AvroSerialization#AVRO_SCHEMA_KEY} be provided
-   * in the metadata argument.
-   * @param metadata the Avro-serialization-specific parameters being
-   * provided that detail the schema for the data to deserialize and compare.
-   * @return a RawComparator parameterized for the specified Avro schema.
-   */
-  public RawComparator<T> getRawComparator(Map<String, String> metadata) {
-    Schema schema = getSchema(metadata);
-    return new AvroComparator(schema);
-  }
 }

+ 7 - 16
src/java/org/apache/hadoop/io/serializer/avro/AvroSpecificSerialization.java

@@ -18,13 +18,9 @@
 
 package org.apache.hadoop.io.serializer.avro;
 
-import java.util.Map;
-
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericDatumReader;
 import org.apache.avro.io.DatumReader;
 import org.apache.avro.io.DatumWriter;
-import org.apache.avro.specific.SpecificData;
 import org.apache.avro.specific.SpecificDatumReader;
 import org.apache.avro.specific.SpecificDatumWriter;
 import org.apache.avro.specific.SpecificRecord;
@@ -38,31 +34,26 @@ public class AvroSpecificSerialization
                           extends AvroSerialization<SpecificRecord>{
 
   @Override
-  public boolean accept(Map<String, String> metadata) {
-    if (!checkSerializationKey(metadata)) {
-      return false;
-    }
-    Class<?> c = getClassFromMetadata(metadata);
-    return c == null ? false : SpecificRecord.class.isAssignableFrom(c);
+  public boolean accept(Class<?> c) {
+    return SpecificRecord.class.isAssignableFrom(c);
   }
 
   @Override
-  public DatumReader getReader(Map<String, String> metadata) {
+  public DatumReader getReader(Class<SpecificRecord> clazz) {
     try {
-      return new SpecificDatumReader(getClassFromMetadata(metadata));
+      return new SpecificDatumReader(clazz.newInstance().getSchema());
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
   }
 
   @Override
-  public Schema getSchema(Map<String, String> metadata) {
-    Class<?> c = getClassFromMetadata(metadata);
-    return SpecificData.get().getSchema(c);
+  public Schema getSchema(SpecificRecord t) {
+    return t.getSchema();
   }
 
   @Override
-  public DatumWriter getWriter(Map<String, String> metadata) {
+  public DatumWriter getWriter(Class<SpecificRecord> clazz) {
     return new SpecificDatumWriter();
   }
 

+ 4 - 6
src/java/org/apache/hadoop/util/ReflectionUtils.java

@@ -35,10 +35,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.serializer.DeserializerBase;
-import org.apache.hadoop.io.serializer.SerializationBase;
+import org.apache.hadoop.io.serializer.Deserializer;
 import org.apache.hadoop.io.serializer.SerializationFactory;
-import org.apache.hadoop.io.serializer.SerializerBase;
+import org.apache.hadoop.io.serializer.Serializer;
 
 /**
  * General reflection utils
@@ -275,12 +274,11 @@ public class ReflectionUtils {
     buffer.outBuffer.reset();
     SerializationFactory factory = getFactory(conf);
     Class<T> cls = (Class<T>) src.getClass();
-    Map<String, String> metadata = SerializationBase.getMetadataFromClass(cls);
-    SerializerBase<T> serializer = factory.getSerializer(metadata);
+    Serializer<T> serializer = factory.getSerializer(cls);
     serializer.open(buffer.outBuffer);
     serializer.serialize(src);
     buffer.moveData();
-    DeserializerBase<T> deserializer = factory.getDeserializer(metadata);
+    Deserializer<T> deserializer = factory.getDeserializer(cls);
     deserializer.open(buffer.inBuffer);
     dst = deserializer.deserialize(dst);
     return dst;

+ 0 - 44
src/test/core/org/apache/hadoop/conf/TestConfiguration.java

@@ -24,7 +24,6 @@ import java.io.IOException;
 import java.io.StringWriter;
 import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.Map;
 import java.util.Random;
 import java.util.regex.Pattern;
 
@@ -366,49 +365,6 @@ public class TestConfiguration extends TestCase {
     assertTrue(fail);
   }
 
-  public void testMap() throws IOException {
-    Configuration conf = new Configuration();
-
-    // manually create a map in the config; extract
-    // its values as a map object.
-    conf.set("foo.bar", "A");
-    conf.set("foo.baz", "B");
-    assertEquals("A", conf.get("foo.bar"));
-    assertEquals("B", conf.get("foo.baz"));
-
-    Map<String, String> out = conf.getMap("foo");
-    assertEquals("A", out.get("bar"));
-    assertEquals("B", out.get("baz"));
-
-    Map<String, String> in = new HashMap<String, String>();
-    in.put("yak", "123");
-    in.put("bop", "456");
-    conf.setMap("quux", in);
-
-    // Assert that we can extract individual entries in
-    // the nested map ok.
-    assertEquals("123", conf.get("quux.yak"));
-
-    // Assert that we can get the whole map back out again.
-    out = conf.getMap("quux");
-    assertEquals("123", out.get("yak"));
-    assertEquals("456", out.get("bop"));
-
-    // Test that substitution is handled by getMap().
-    conf.set("subparam", "foo");
-    conf.set("mymap.someprop", "AAA${subparam}BBB");
-    out = conf.getMap("mymap");
-    assertEquals("AAAfooBBB", out.get("someprop"));
-
-    // Test deprecation of maps.
-    Configuration.addDeprecation("oldfoo", new String[]{"newfoo"});
-    conf.set("newfoo.a", "A");
-    conf.set("newfoo.b", "B");
-    out = conf.getMap("oldfoo");
-    assertEquals("A", out.get("a"));
-    assertEquals("B", out.get("b"));
-  }
-
   public void testPattern() throws IOException {
     out = new BufferedWriter(new FileWriter(CONFIG));
     startConfig();

+ 6 - 22
src/test/core/org/apache/hadoop/io/serializer/SerializationTestUtil.java

@@ -17,45 +17,29 @@
  */
 package org.apache.hadoop.io.serializer;
 
-import java.util.Map;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.util.GenericsUtil;
 
 public class SerializationTestUtil {
-
-  /**
-   * A utility that tests serialization/deserialization. 
-   * @param <K> the class of the item
-   * @param conf configuration to use, "io.serializations" is read to 
-   * determine the serialization
-   * @param before item to (de)serialize
-   * @return deserialized item
-   */
-  public static<K> K testSerialization(Configuration conf, K before) 
-      throws Exception {
-    Map<String, String> metadata =
-      SerializationBase.getMetadataFromClass(GenericsUtil.getClass(before));
-    return testSerialization(conf, metadata, before);
-  }
   
   /**
    * A utility that tests serialization/deserialization. 
    * @param conf configuration to use, "io.serializations" is read to 
    * determine the serialization
-   * @param metadata the metadata to pass to the serializer/deserializer
    * @param <K> the class of the item
    * @param before item to (de)serialize
    * @return deserialized item
    */
-  public static <K> K testSerialization(Configuration conf, 
-      Map<String, String> metadata, K before) throws Exception {
+  public static <K> K testSerialization(Configuration conf, K before)
+  	throws Exception {
 
     SerializationFactory factory = new SerializationFactory(conf);
-    SerializerBase<K> serializer = factory.getSerializer(metadata);
-    DeserializerBase<K> deserializer = factory.getDeserializer(metadata);
+    Serializer<K> serializer 
+      = factory.getSerializer(GenericsUtil.getClass(before));
+    Deserializer<K> deserializer 
+      = factory.getDeserializer(GenericsUtil.getClass(before));
 
     DataOutputBuffer out = new DataOutputBuffer();
     serializer.open(out);

+ 0 - 175
src/test/core/org/apache/hadoop/io/serializer/TestRawComparators.java

@@ -1,175 +0,0 @@
-/**
- * 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.serializer;
-
-import static org.apache.hadoop.io.TestGenericWritable.CONF_TEST_KEY;
-import static org.apache.hadoop.io.TestGenericWritable.CONF_TEST_VALUE;
-import junit.framework.TestCase;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.avro.Schema;
-import org.apache.avro.util.Utf8;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.serializer.avro.AvroSerialization;
-import org.apache.hadoop.io.serializer.avro.AvroGenericSerialization;
-import org.apache.hadoop.util.GenericsUtil;
-
-/**
- * Test the getRawComparator API of the various serialization systems.
- */
-public class TestRawComparators extends TestCase {
-
-  private Configuration conf;
-
-  public void setUp() {
-    conf = new Configuration();
-  }
-
-  /** A WritableComparable that is guaranteed to use the
-   * generic WritableComparator.
-   */
-  public static class FooWritable implements WritableComparable<FooWritable> {
-
-    public long val;
-
-    public FooWritable() {
-      this.val = 0;
-    }
-
-    public FooWritable(long v) {
-      this.val = v;
-    }
-
-    public void write(DataOutput out) throws IOException {
-      out.writeLong(val);
-    }
-
-    public void readFields(DataInput in) throws IOException {
-      val = in.readLong();
-    }
-
-    public int compareTo(FooWritable other) {
-      return new Long(val).compareTo(other.val);
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  private void runComparisonTest(Object low, Object high) throws Exception {
-    Map<String, String> metadata =
-          SerializationBase.getMetadataFromClass(GenericsUtil.getClass(low));
-    runComparisonTest(low, high, metadata);
-  }
-
-  @SuppressWarnings("unchecked")
-  private void runComparisonTest(Object low, Object high,
-      Map<String, String> metadata) throws Exception {
-
-    DataOutputBuffer out1 = new DataOutputBuffer();
-    DataOutputBuffer out2 = new DataOutputBuffer();
-    DataInputBuffer in1 = new DataInputBuffer();
-    DataInputBuffer in2 = new DataInputBuffer();
-
-    SerializationFactory factory = new SerializationFactory(conf);
-
-    // Serialize some data to two byte streams.
-    SerializerBase serializer = factory.getSerializer(metadata);
-    assertNotNull("Serializer is null!", serializer);
-
-    serializer.open(out1);
-    serializer.serialize(low);
-    serializer.close();
-
-    serializer.open(out2);
-    serializer.serialize(high);
-    serializer.close();
-
-    // Shift that data into an input buffer.
-    in1.reset(out1.getData(), out1.getLength());
-    in2.reset(out2.getData(), out2.getLength());
-
-    // Get the serialization and then the RawComparator;
-    // use these to compare the data in the input streams and
-    // assert that the low stream (1) is less than the high stream (2).
-
-    SerializationBase serializationBase = factory.getSerialization(metadata);
-    assertNotNull("Null SerializationBase!", serializationBase);
-
-    RawComparator rawComparator = serializationBase.getRawComparator(metadata);
-    assertNotNull("Null raw comparator!", rawComparator);
-    int actual = rawComparator.compare(in1.getData(), 0, in1.getLength(),
-        in2.getData(), 0, in2.getLength());
-    assertTrue("Did not compare FooWritable correctly", actual < 0);
-  }
-
-  public void testBasicWritable() throws Exception {
-    // Test that a WritableComparable can be used with this API
-    // correctly.
-
-    FooWritable low = new FooWritable(10);
-    FooWritable high = new FooWritable(42);
-
-    runComparisonTest(low, high);
-  }
-
-  public void testTextWritable() throws Exception {
-    // Test that a Text object (which uses Writable serialization, and
-    // has its own RawComparator implementation) can be used with this
-    // API correctly.
-
-    Text low = new Text("aaa");
-    Text high = new Text("zzz");
-
-    runComparisonTest(low, high);
-  }
-
-  public void testAvroComparator() throws Exception {
-    // Test a record created via an Avro schema that doesn't have a fixed
-    // class associated with it.
-
-    Schema s1 = Schema.create(Schema.Type.INT);
-
-    // Create a metadata mapping containing an Avro schema and a request to use
-    // Avro generic serialization.
-    Map<String, String> metadata = new HashMap<String, String>();
-    metadata.put(AvroSerialization.AVRO_SCHEMA_KEY, s1.toString());
-    metadata.put(SerializationBase.SERIALIZATION_KEY,
-       AvroGenericSerialization.class.getName());
-
-    runComparisonTest(new Integer(42), new Integer(123), metadata);
-
-    // Now test it with a string record type.
-    Schema s2 = Schema.create(Schema.Type.STRING);
-    metadata.put(AvroSerialization.AVRO_SCHEMA_KEY, s2.toString());
-    runComparisonTest(new Utf8("baz"), new Utf8("meep"), metadata);
-
-  }
-
-}

+ 0 - 127
src/test/core/org/apache/hadoop/io/serializer/TestWritableSerialization.java

@@ -22,22 +22,10 @@ import static org.apache.hadoop.io.TestGenericWritable.CONF_TEST_KEY;
 import static org.apache.hadoop.io.TestGenericWritable.CONF_TEST_VALUE;
 import junit.framework.TestCase;
 
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.TestGenericWritable.Foo;
-import org.apache.hadoop.io.TestGenericWritable.Bar;
 import org.apache.hadoop.io.TestGenericWritable.Baz;
 import org.apache.hadoop.io.TestGenericWritable.FooGenericWritable;
-import org.apache.hadoop.io.serializer.DeserializerBase;
-import org.apache.hadoop.io.serializer.SerializationBase;
-import org.apache.hadoop.io.serializer.SerializerBase;
-import org.apache.hadoop.util.GenericsUtil;
 
 public class TestWritableSerialization extends TestCase {
 
@@ -49,7 +37,6 @@ public class TestWritableSerialization extends TestCase {
     assertEquals(before, after);
   }
   
-  
   public void testWritableConfigurable() throws Exception {
     
     //set the configuration parameter
@@ -65,118 +52,4 @@ public class TestWritableSerialization extends TestCase {
     assertEquals(baz, result);
     assertNotNull(result.getConf());
   }
-
-  @SuppressWarnings("unchecked")
-  public void testIgnoreMisconfiguredMetadata() throws IOException {
-    // If SERIALIZATION_KEY is set, still need class name.
-
-    Configuration conf = new Configuration();
-    Map<String, String> metadata = new HashMap<String, String>();
-    metadata.put(SerializationBase.SERIALIZATION_KEY,
-        WritableSerialization.class.getName());
-    SerializationFactory factory = new SerializationFactory(conf);
-    SerializationBase serialization = factory.getSerialization(metadata);
-    assertNull("Got serializer without any class info", serialization);
-
-    metadata.put(SerializationBase.CLASS_KEY,
-        Text.class.getName());
-    serialization = factory.getSerialization(metadata);
-    assertNotNull("Didn't get serialization!", serialization);
-    assertTrue("Wrong serialization class",
-        serialization instanceof WritableSerialization);
-  }
-
-  @SuppressWarnings("unchecked")
-  public void testReuseSerializer() throws IOException {
-    // Test that we can write multiple objects of the same type
-    // through the same serializer.
-
-    DataOutputBuffer out = new DataOutputBuffer();
-    SerializationFactory factory = new SerializationFactory(
-        new Configuration());
-
-    // Create a few Foo objects and serialize them.
-    Foo foo = new Foo();
-    Foo foo2 = new Foo();
-    Map<String, String> metadata = SerializationBase.getMetadataFromClass(
-        GenericsUtil.getClass(foo));
-
-    SerializerBase fooSerializer = factory.getSerializer(metadata);
-    fooSerializer.open(out);
-    fooSerializer.serialize(foo);
-    fooSerializer.serialize(foo2);
-    fooSerializer.close();
-
-    out.reset();
-
-    // Create a new serializer for Bar objects
-    Bar bar = new Bar();
-    Baz baz = new Baz(); // Baz inherits from Bar.
-    metadata = SerializationBase.getMetadataFromClass(
-        GenericsUtil.getClass(bar));
-    // Check that we can serialize Bar objects.
-    SerializerBase barSerializer = factory.getSerializer(metadata);
-    barSerializer.open(out);
-    barSerializer.serialize(bar); // this should work.
-    try {
-      // This should not work. We should not allow subtype serialization.
-      barSerializer.serialize(baz);
-      fail("Expected IOException serializing baz via bar serializer.");
-    } catch (IOException ioe) {
-      // Expected.
-    }
-
-    try {
-      // This should not work. Disallow unrelated type serialization.
-      barSerializer.serialize(foo);
-      fail("Expected IOException serializing foo via bar serializer.");
-    } catch (IOException ioe) {
-      // Expected.
-    }
-
-    barSerializer.close();
-    out.reset();
-  }
-
-
-  // Test the SerializationBase.checkSerializationKey() method.
-  class DummySerializationBase extends SerializationBase<Object> {
-    public boolean accept(Map<String, String> metadata) {
-      return checkSerializationKey(metadata);
-    }
-
-    public SerializerBase<Object> getSerializer(Map<String, String> metadata) {
-      return null;
-    }
-
-    public DeserializerBase<Object> getDeserializer(Map<String, String> metadata) {
-      return null;
-    }
-
-    public RawComparator<Object> getRawComparator(Map<String, String> metadata) {
-      return null;
-    }
-  }
-
-  public void testSerializationKeyCheck() {
-    DummySerializationBase dummy = new DummySerializationBase();
-    Map<String, String> metadata = new HashMap<String, String>();
-
-    assertTrue("Didn't accept empty metadata", dummy.accept(metadata));
-
-    metadata.put(SerializationBase.SERIALIZATION_KEY,
-        DummySerializationBase.class.getName());
-    assertTrue("Didn't accept valid metadata", dummy.accept(metadata));
-
-    metadata.put(SerializationBase.SERIALIZATION_KEY, "foo");
-    assertFalse("Accepted invalid metadata", dummy.accept(metadata));
-
-    try {
-      dummy.accept((Map<String, String>) null);
-      // Shouldn't get here!
-      fail("Somehow didn't actually test the method we expected");
-    } catch (NullPointerException npe) {
-      // expected this.
-    }
-  }
 }

+ 0 - 41
src/test/core/org/apache/hadoop/io/serializer/avro/TestAvroSerialization.java

@@ -18,46 +18,15 @@
 
 package org.apache.hadoop.io.serializer.avro;
 
-import java.util.HashMap;
-import java.util.Map;
-
 import junit.framework.TestCase;
 
-import org.apache.avro.util.Utf8;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.serializer.SerializationBase;
-import org.apache.hadoop.io.serializer.SerializationFactory;
 import org.apache.hadoop.io.serializer.SerializationTestUtil;
 
 public class TestAvroSerialization extends TestCase {
 
   private static final Configuration conf = new Configuration();
 
-  @SuppressWarnings("unchecked")
-  public void testIgnoreMisconfiguredMetadata() {
-    // If SERIALIZATION_KEY is set, still need class name.
-
-    Configuration conf = new Configuration();
-    Map<String, String> metadata = new HashMap<String, String>();
-    SerializationFactory factory = new SerializationFactory(conf);
-    SerializationBase serialization = null;
-
-    metadata.put(SerializationBase.SERIALIZATION_KEY,
-        AvroGenericSerialization.class.getName());
-    serialization = factory.getSerialization(metadata);
-    assertNull("Got serializer without any class info", serialization);
-
-    metadata.put(SerializationBase.SERIALIZATION_KEY,
-        AvroReflectSerialization.class.getName());
-    serialization = factory.getSerialization(metadata);
-    assertNull("Got serializer without any class info", serialization);
-
-    metadata.put(SerializationBase.SERIALIZATION_KEY,
-        AvroSpecificSerialization.class.getName());
-    serialization = factory.getSerialization(metadata);
-    assertNull("Got serializer without any class info", serialization);
-  }
-
   public void testSpecific() throws Exception {
     AvroRecord before = new AvroRecord();
     before.intField = 5;
@@ -91,16 +60,6 @@ public class TestAvroSerialization extends TestCase {
     assertEquals(before, after);
   }
   
-  public void testGeneric() throws Exception {
-    Utf8 before = new Utf8("hadoop");
-    Map<String, String> metadata = new HashMap<String, String>();
-    metadata.put(SerializationBase.SERIALIZATION_KEY,
-      AvroGenericSerialization.class.getName());
-    metadata.put(AvroSerialization.AVRO_SCHEMA_KEY, "\"string\"");
-    Utf8 after = SerializationTestUtil.testSerialization(conf, metadata, before);
-    assertEquals(before, after);
-  }
-
   public static class InnerRecord {
     public int x = 7;