You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2010/04/26 18:05:26 UTC

svn commit: r938104 - in /hadoop/hbase/branches/0.20_pre_durability: CHANGES.txt src/java/org/apache/hadoop/hbase/io/ImmutableBytesWritable.java src/test/org/apache/hadoop/hbase/io/TestImmutableBytesWritable.java

Author: stack
Date: Mon Apr 26 16:05:26 2010
New Revision: 938104

URL: http://svn.apache.org/viewvc?rev=938104&view=rev
Log:
HBASE-2378 Bulk insert with multiple reducers broken due to improper ImmutableBytesWritable comparator (Plus some fixeup to CHANGES.txt)

Added:
    hadoop/hbase/branches/0.20_pre_durability/src/test/org/apache/hadoop/hbase/io/TestImmutableBytesWritable.java
Modified:
    hadoop/hbase/branches/0.20_pre_durability/CHANGES.txt
    hadoop/hbase/branches/0.20_pre_durability/src/java/org/apache/hadoop/hbase/io/ImmutableBytesWritable.java

Modified: hadoop/hbase/branches/0.20_pre_durability/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20_pre_durability/CHANGES.txt?rev=938104&r1=938103&r2=938104&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20_pre_durability/CHANGES.txt (original)
+++ hadoop/hbase/branches/0.20_pre_durability/CHANGES.txt Mon Apr 26 16:05:26 2010
@@ -1,8 +1,7 @@
 HBase Change Log
 
-Release 0.20.4 - Thu Apr 15 16:29:44 PDT 2010
+Release 0.20.4 - Mon Apr 26 08:39:23 PDT 2010
   INCOMPATIBLE CHANGES
-   HBASE-2165  Improve fragmentation display and implementation
    HBASE-2248  Provide new non-copy mechanism to assure atomic reads in get
                and scan (Ryan Rawson via Stack)
    HBASE-2446  Remove 'indexed' contrib
@@ -50,7 +49,7 @@ Release 0.20.4 - Thu Apr 15 16:29:44 PDT
    HBASE-2453  Revisit compaction policies after HBASE-2248 commit
                (Jonathan Gray via Stack)
    HBASE-2456  deleteChangedReaderObserver spitting warnings after HBASE-2248 
-   HBASE-2456  Client stuck in TreeMap,remove (Todd Lipcon via Stack)
+   HBASE-2458  Client stuck in TreeMap,remove (Todd Lipcon via Stack)
    HBASE-2460  add_table.rb deletes any tables for which the target table
                name is a prefix (Todd Lipcon via Stack)
    HBASE-2457  RS gets stuck compacting region ad infinitum
@@ -60,7 +59,8 @@ Release 0.20.4 - Thu Apr 15 16:29:44 PDT
                (Todd Lipcon via Stack)
    HBASE-2481  Client is not getting UnknownScannerExceptions; they are being
                eaten (Jean-Daniel Cryans via Stack)
-   HBASE-2458  Client stuck in TreeMap,remove (Todd Lipcon via Stack)
+   HBASE-2378  Bulk insert with multiple reducers broken due to improper
+               ImmutableBytesWritable comparator
 
   IMPROVEMENTS
    HBASE-2180  Bad read performance from synchronizing hfile.fddatainputstream

Modified: hadoop/hbase/branches/0.20_pre_durability/src/java/org/apache/hadoop/hbase/io/ImmutableBytesWritable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20_pre_durability/src/java/org/apache/hadoop/hbase/io/ImmutableBytesWritable.java?rev=938104&r1=938103&r2=938104&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20_pre_durability/src/java/org/apache/hadoop/hbase/io/ImmutableBytesWritable.java (original)
+++ hadoop/hbase/branches/0.20_pre_durability/src/java/org/apache/hadoop/hbase/io/ImmutableBytesWritable.java Mon Apr 26 16:05:26 2010
@@ -178,10 +178,7 @@ implements WritableComparable<ImmutableB
    *         negative if left is smaller than right.
    */
   public int compareTo(final byte [] that) {
-    int diff = this.length - that.length;
-    return (diff != 0)?
-      diff:
-      WritableComparator.compareBytes(this.bytes, 0, this.length, that,
+    return WritableComparator.compareBytes(this.bytes, 0, this.length, that,
         0, that.length);
   }
 

Added: hadoop/hbase/branches/0.20_pre_durability/src/test/org/apache/hadoop/hbase/io/TestImmutableBytesWritable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20_pre_durability/src/test/org/apache/hadoop/hbase/io/TestImmutableBytesWritable.java?rev=938104&view=auto
==============================================================================
--- hadoop/hbase/branches/0.20_pre_durability/src/test/org/apache/hadoop/hbase/io/TestImmutableBytesWritable.java (added)
+++ hadoop/hbase/branches/0.20_pre_durability/src/test/org/apache/hadoop/hbase/io/TestImmutableBytesWritable.java Mon Apr 26 16:05:26 2010
@@ -0,0 +1,96 @@
+/**
+ * Copyright 2009 The Apache Software Foundation
+ *
+ * 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.hbase.io;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+public class TestImmutableBytesWritable extends TestCase {
+  public void testComparison() throws Exception {
+    runTests("aa", "b", -1);
+    runTests("aa", "aa", 0);
+    runTests("aa", "ab", -1);
+    runTests("aa", "aaa", -1);
+    runTests("", "", 0);
+    runTests("", "a", -1);
+  }
+
+  private void runTests(String aStr, String bStr, int signum)
+    throws Exception {
+    ImmutableBytesWritable a = new ImmutableBytesWritable(
+      Bytes.toBytes(aStr));
+    ImmutableBytesWritable b = new ImmutableBytesWritable(
+      Bytes.toBytes(bStr));
+
+    doComparisonsOnObjects(a, b, signum);
+    doComparisonsOnRaw(a, b, signum);
+  }
+
+
+  private int signum(int i) {
+    if (i > 0) return 1;
+    if (i == 0) return 0;
+    return -1;
+  }
+
+  private void doComparisonsOnRaw(ImmutableBytesWritable a,
+                                  ImmutableBytesWritable b,
+                                  int expectedSignum)
+    throws IOException {
+    ImmutableBytesWritable.Comparator comparator =
+      new ImmutableBytesWritable.Comparator();
+
+    ByteArrayOutputStream baosA = new ByteArrayOutputStream();
+    ByteArrayOutputStream baosB = new ByteArrayOutputStream();
+
+    a.write(new DataOutputStream(baosA));
+    b.write(new DataOutputStream(baosB));
+
+    assertEquals(
+      "Comparing " + a + " and " + b + " as raw",
+      signum(comparator.compare(baosA.toByteArray(), 0, baosA.size(),
+                                baosB.toByteArray(), 0, baosB.size())),
+      expectedSignum);
+
+    assertEquals(
+      "Comparing " + a + " and " + b + " as raw (inverse)",
+      -signum(comparator.compare(baosB.toByteArray(), 0, baosB.size(),
+                                 baosA.toByteArray(), 0, baosA.size())),
+      expectedSignum);
+  }
+
+  private void doComparisonsOnObjects(ImmutableBytesWritable a,
+                                      ImmutableBytesWritable b,
+                                      int expectedSignum) {
+    ImmutableBytesWritable.Comparator comparator =
+      new ImmutableBytesWritable.Comparator();
+    assertEquals(
+      "Comparing " + a + " and " + b + " as objects",
+      signum(comparator.compare(a, b)), expectedSignum);
+    assertEquals(
+      "Comparing " + a + " and " + b + " as objects (inverse)",
+      -signum(comparator.compare(b, a)), expectedSignum);
+  }
+}
\ No newline at end of file