Pārlūkot izejas kodu

MAPREDUCE-2. Fixes a bug in KeyFieldBasedPartitioner in handling empty keys. Contributed by Amar Kamat.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.20@788632 13f79535-47bb-0310-9956-ffa450edef68
Sharad Agarwal 16 gadi atpakaļ
vecāks
revīzija
5d76320ca0

+ 3 - 0
CHANGES.txt

@@ -144,6 +144,9 @@ Release 0.20.1 - Unreleased
     lack of quota. Allow quota to be set even if the limit is lower than
     current consumption. (Boris Shkolnik via rangadi)
 
+    MAPREDUCE-2. Fixes a bug in KeyFieldBasedPartitioner in handling empty
+    keys. (Amar Kamat via sharad)
+
 Release 0.20.0 - 2009-04-15
 
   INCOMPATIBLE CHANGES

+ 9 - 0
src/mapred/org/apache/hadoop/mapred/lib/KeyFieldBasedPartitioner.java

@@ -76,12 +76,21 @@ public class KeyFieldBasedPartitioner<K2, V2> implements Partitioner<K2, V2> {
       throw new RuntimeException("The current system does not " +
           "support UTF-8 encoding!", e);
     }
+    // return 0 if the key is empty
+    if (keyBytes.length == 0) {
+      return 0;
+    }
+    
     int []lengthIndicesFirst = keyFieldHelper.getWordLengths(keyBytes, 0, 
         keyBytes.length);
     int currentHash = 0;
     for (KeyDescription keySpec : allKeySpecs) {
       int startChar = keyFieldHelper.getStartOffset(keyBytes, 0, keyBytes.length, 
           lengthIndicesFirst, keySpec);
+       // no key found! continue
+      if (startChar < 0) {
+        continue;
+      }
       int endChar = keyFieldHelper.getEndOffset(keyBytes, 0, keyBytes.length, 
           lengthIndicesFirst, keySpec);
       currentHash = hashCode(keyBytes, startChar, endChar, 

+ 40 - 0
src/test/org/apache/hadoop/mapred/lib/TestKeyFieldBasedPartitioner.java

@@ -0,0 +1,40 @@
+/**
+ * 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.mapred.lib;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.lib.KeyFieldBasedPartitioner;
+
+import junit.framework.TestCase;
+
+public class TestKeyFieldBasedPartitioner extends TestCase {
+
+  /**
+   * Test is key-field-based partitioned works with empty key.
+   */
+  public void testEmptyKey() throws Exception {
+    KeyFieldBasedPartitioner<Text, Text> kfbp = 
+      new KeyFieldBasedPartitioner<Text, Text>();
+    JobConf conf = new JobConf();
+    conf.setInt("num.key.fields.for.partition", 10);
+    kfbp.configure(conf);
+    assertEquals("Empty key should map to 0th partition", 
+                 0, kfbp.getPartition(new Text(), new Text(), 10));
+  }
+}