Browse Source

MAPREDUCE-4771. KeyFieldBasedPartitioner not partitioning properly when configured (jlowe via bobby)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1405975 13f79535-47bb-0310-9956-ffa450edef68
Robert Joseph Evans 12 years ago
parent
commit
0eadfcfcf5

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

@@ -631,6 +631,9 @@ Release 0.23.5 - UNRELEASED
 
     MAPREDUCE-4763 repair test TestUmbilicalProtocolWithJobToken (Ivan A.
     Veselovsky via bobby)
+
+    MAPREDUCE-4771. KeyFieldBasedPartitioner not partitioning properly when
+    configured (jlowe via bobby)
  
 Release 0.23.4 - UNRELEASED
 

+ 1 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/KeyFieldBasedPartitioner.java

@@ -63,6 +63,7 @@ public class KeyFieldBasedPartitioner<K2, V2> extends Partitioner<K2, V2>
 
   public void setConf(Configuration conf) {
     this.conf = conf;
+    keyFieldHelper = new KeyFieldHelper();
     String keyFieldSeparator = 
       conf.get(MRJobConfig.MAP_OUTPUT_KEY_FIELD_SEPERATOR, "\t");
     keyFieldHelper.setKeyFieldSeparator(keyFieldSeparator);

+ 19 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedPartitioner.java

@@ -17,17 +17,18 @@
  */
 package org.apache.hadoop.mapred.lib;
 
+import static org.junit.Assert.assertEquals;
+
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.lib.KeyFieldBasedPartitioner;
-
-import junit.framework.TestCase;
+import org.junit.Test;
 
-public class TestKeyFieldBasedPartitioner extends TestCase {
+public class TestKeyFieldBasedPartitioner {
 
   /**
    * Test is key-field-based partitioned works with empty key.
    */
+  @Test
   public void testEmptyKey() throws Exception {
     KeyFieldBasedPartitioner<Text, Text> kfbp = 
       new KeyFieldBasedPartitioner<Text, Text>();
@@ -37,4 +38,18 @@ public class TestKeyFieldBasedPartitioner extends TestCase {
     assertEquals("Empty key should map to 0th partition", 
                  0, kfbp.getPartition(new Text(), new Text(), 10));
   }
+
+  @Test
+  public void testMultiConfigure() {
+    KeyFieldBasedPartitioner<Text, Text> kfbp =
+      new KeyFieldBasedPartitioner<Text, Text>();
+    JobConf conf = new JobConf();
+    conf.set(KeyFieldBasedPartitioner.PARTITIONER_OPTIONS, "-k1,1");
+    kfbp.setConf(conf);
+    Text key = new Text("foo\tbar");
+    Text val = new Text("val");
+    int partNum = kfbp.getPartition(key, val, 4096);
+    kfbp.configure(conf);
+    assertEquals(partNum, kfbp.getPartition(key,val, 4096));
+  }
 }