Browse Source

HDFS-8619. Merging change from trunk to branch-2

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1378129 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 12 năm trước cách đây
mục cha
commit
c0b2ddc53b

+ 3 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -39,6 +39,9 @@ Release 0.23.3 - UNRELEASED
     HADOOP-8239. Add subclasses of MD5MD5CRC32FileChecksum to support file
     checksum with CRC32C.  (Kihwal Lee via szetszwo)
 
+    HADOOP-8619. WritableComparator must implement no-arg constructor.
+    (Chris Douglas via Suresh)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 4 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableComparator.java

@@ -87,6 +87,10 @@ public class WritableComparator implements RawComparator {
   private final WritableComparable key2;
   private final DataInputBuffer buffer;
 
+  protected WritableComparator() {
+    this(null);
+  }
+
   /** Construct for a {@link WritableComparable} implementation. */
   protected WritableComparator(Class<? extends WritableComparable> keyClass) {
     this(keyClass, false);

+ 49 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/serializer/TestWritableSerialization.java

@@ -18,25 +18,34 @@
 
 package org.apache.hadoop.io.serializer;
 
+import java.io.Serializable;
+
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
 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 org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.TestGenericWritable.Baz;
 import org.apache.hadoop.io.TestGenericWritable.FooGenericWritable;
+import org.apache.hadoop.io.WritableComparator;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
 
-public class TestWritableSerialization extends TestCase {
+public class TestWritableSerialization {
 
   private static final Configuration conf = new Configuration();
 
+  @Test
   public void testWritableSerialization() throws Exception {
     Text before = new Text("test writable"); 
     Text after = SerializationTestUtil.testSerialization(conf, before);
     assertEquals(before, after);
   }
   
+  @Test
   public void testWritableConfigurable() throws Exception {
     
     //set the configuration parameter
@@ -52,4 +61,42 @@ public class TestWritableSerialization extends TestCase {
     assertEquals(baz, result);
     assertNotNull(result.getConf());
   }
+
+  @Test
+  @SuppressWarnings({"rawtypes", "unchecked"})
+  public void testWritableComparatorJavaSerialization() throws Exception {
+    Serialization ser = new JavaSerialization();
+
+    Serializer<TestWC> serializer = ser.getSerializer(TestWC.class);
+    DataOutputBuffer dob = new DataOutputBuffer();
+    serializer.open(dob);
+    TestWC orig = new TestWC(0);
+    serializer.serialize(orig);
+    serializer.close();
+
+    Deserializer<TestWC> deserializer = ser.getDeserializer(TestWC.class);
+    DataInputBuffer dib = new DataInputBuffer();
+    dib.reset(dob.getData(), 0, dob.getLength());
+    deserializer.open(dib);
+    TestWC deser = deserializer.deserialize(null);
+    deserializer.close();
+    assertEquals(orig, deser);
+  }
+
+  static class TestWC extends WritableComparator implements Serializable {
+    static final long serialVersionUID = 0x4344;
+    final int val;
+    TestWC() { this(7); }
+    TestWC(int val) { this.val = val; }
+    @Override
+    public boolean equals(Object o) {
+      if (o instanceof TestWC) {
+        return ((TestWC)o).val == val;
+      }
+      return false;
+    }
+    @Override
+    public int hashCode() { return val; }
+  }
+
 }