Просмотр исходного кода

commit 4640e24bd88304f22968e1433ad5804a12192506
Author: Yahoo\! <ltucker@yahoo-inc.com>
Date: Tue Aug 11 10:07:23 2009 -0700

Walk back the 0.20 branch to the correct point
Revert "MAPREDUCE-735. Fixes a problem in the KeyFieldHelper to do with the end index for some inputs. Contributed by Amar Kamat."

This reverts commit 66a3d58e6f3ca322bac445739021e765009e677b.


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.20-security-patches@1076966 13f79535-47bb-0310-9956-ffa450edef68

Owen O'Malley 14 лет назад
Родитель
Сommit
6889142193

+ 2 - 5
src/mapred/org/apache/hadoop/mapred/lib/KeyFieldBasedPartitioner.java

@@ -67,7 +67,7 @@ public class KeyFieldBasedPartitioner<K2, V2> implements Partitioner<K2, V2> {
 
     List <KeyDescription> allKeySpecs = keyFieldHelper.keySpecs();
     if (allKeySpecs.size() == 0) {
-      return getPartition(key.toString().hashCode(), numReduceTasks);
+      return (key.toString().hashCode() & Integer.MAX_VALUE) % numReduceTasks;
     }
 
     try {
@@ -96,7 +96,7 @@ public class KeyFieldBasedPartitioner<K2, V2> implements Partitioner<K2, V2> {
       currentHash = hashCode(keyBytes, startChar, endChar, 
           currentHash);
     }
-    return getPartition(currentHash, numReduceTasks);
+    return (currentHash & Integer.MAX_VALUE) % numReduceTasks;
   }
   
   protected int hashCode(byte[] b, int start, int end, int currentHash) {
@@ -106,7 +106,4 @@ public class KeyFieldBasedPartitioner<K2, V2> implements Partitioner<K2, V2> {
     return currentHash;
   }
 
-  protected int getPartition(int hash, int numReduceTasks) {
-    return (hash & Integer.MAX_VALUE) % numReduceTasks;
-  }
 }

+ 3 - 10
src/mapred/org/apache/hadoop/mapred/lib/KeyFieldHelper.java

@@ -47,13 +47,6 @@ class KeyFieldHelper {
     int endChar = 0;
     boolean numeric;
     boolean reverse;
-    @Override
-    public String toString() {
-      return "-k" 
-             + beginFieldIdx + "." + beginChar + "," 
-             + endFieldIdx + "." + endChar 
-             + (numeric ? "n" : "") + (reverse ? "r" : "");
-    }
   }
   
   private List<KeyDescription> allKeySpecs = new ArrayList<KeyDescription>();
@@ -137,7 +130,7 @@ class KeyFieldHelper {
     if (k.endFieldIdx == 0) {
       //there is no end field specified for this keyspec. So the remaining
       //part of the key is considered in its entirety.
-      return end - 1; 
+      return end; 
     }
     if (lengthIndices[0] >= k.endFieldIdx) {
       int position = 0;
@@ -151,9 +144,9 @@ class KeyFieldHelper {
       if (position + k.endChar <= (end - start)) {
         return start + position + k.endChar - 1;
       }
-      return end - 1;
+      return end;
     }
-    return end - 1;
+    return end;
   }
   public void parseOption(String option) {
     if (option == null || option.equals("")) {

+ 0 - 2
src/test/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java

@@ -133,8 +133,6 @@ public class TestKeyFieldBasedComparator extends HadoopTestCase {
     configure("-k10,10",2);
     
     localTestWithoutMRJob("-k9,9", 1);
-
-    localTestWithoutMRJob("-k9n", 1);
   }
   
   byte[] line1_bytes = line1.getBytes();

+ 2 - 88
src/test/org/apache/hadoop/mapred/lib/TestKeyFieldBasedPartitioner.java

@@ -29,98 +29,12 @@ public class TestKeyFieldBasedPartitioner extends TestCase {
    * Test is key-field-based partitioned works with empty key.
    */
   public void testEmptyKey() throws Exception {
-    int numReducers = 10;
     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(), numReducers));
-    
-    // check if the hashcode is correct when no keyspec is specified
-    kfbp = new KeyFieldBasedPartitioner<Text, Text>();
-    conf = new JobConf();
-    kfbp.configure(conf);
-    String input = "abc\tdef\txyz";
-    int hashCode = input.hashCode();
-    int expectedPartition = kfbp.getPartition(hashCode, numReducers);
-    assertEquals("Partitioner doesnt work as expected", expectedPartition, 
-                 kfbp.getPartition(new Text(input), new Text(), numReducers));
-    
-    // check if the hashcode is correct with specified keyspec
-    kfbp = new KeyFieldBasedPartitioner<Text, Text>();
-    conf = new JobConf();
-    conf.set("mapred.text.key.partitioner.options", "-k2,2");
-    kfbp.configure(conf);
-    String expectedOutput = "def";
-    byte[] eBytes = expectedOutput.getBytes();
-    hashCode = kfbp.hashCode(eBytes, 0, eBytes.length - 1, 0);
-    expectedPartition = kfbp.getPartition(hashCode, numReducers);
-    assertEquals("Partitioner doesnt work as expected", expectedPartition, 
-                 kfbp.getPartition(new Text(input), new Text(), numReducers));
-    
-    // test with invalid end index in keyspecs
-    kfbp = new KeyFieldBasedPartitioner<Text, Text>();
-    conf = new JobConf();
-    conf.set("mapred.text.key.partitioner.options", "-k2,5");
-    kfbp.configure(conf);
-    expectedOutput = "def\txyz";
-    eBytes = expectedOutput.getBytes();
-    hashCode = kfbp.hashCode(eBytes, 0, eBytes.length - 1, 0);
-    expectedPartition = kfbp.getPartition(hashCode, numReducers);
-    assertEquals("Partitioner doesnt work as expected", expectedPartition, 
-                 kfbp.getPartition(new Text(input), new Text(), numReducers));
-    
-    // test with 0 end index in keyspecs
-    kfbp = new KeyFieldBasedPartitioner<Text, Text>();
-    conf = new JobConf();
-    conf.set("mapred.text.key.partitioner.options", "-k2");
-    kfbp.configure(conf);
-    expectedOutput = "def\txyz";
-    eBytes = expectedOutput.getBytes();
-    hashCode = kfbp.hashCode(eBytes, 0, eBytes.length - 1, 0);
-    expectedPartition = kfbp.getPartition(hashCode, numReducers);
-    assertEquals("Partitioner doesnt work as expected", expectedPartition, 
-                 kfbp.getPartition(new Text(input), new Text(), numReducers));
-    
-    // test with invalid keyspecs
-    kfbp = new KeyFieldBasedPartitioner<Text, Text>();
-    conf = new JobConf();
-    conf.set("mapred.text.key.partitioner.options", "-k10");
-    kfbp.configure(conf);
-    assertEquals("Partitioner doesnt work as expected", 0, 
-                 kfbp.getPartition(new Text(input), new Text(), numReducers));
-    
-    // test with multiple keyspecs
-    kfbp = new KeyFieldBasedPartitioner<Text, Text>();
-    conf = new JobConf();
-    conf.set("mapred.text.key.partitioner.options", "-k2,2 -k4,4");
-    kfbp.configure(conf);
-    input = "abc\tdef\tpqr\txyz";
-    expectedOutput = "def";
-    eBytes = expectedOutput.getBytes();
-    hashCode = kfbp.hashCode(eBytes, 0, eBytes.length - 1, 0);
-    expectedOutput = "xyz";
-    eBytes = expectedOutput.getBytes();
-    hashCode = kfbp.hashCode(eBytes, 0, eBytes.length - 1, hashCode);
-    expectedPartition = kfbp.getPartition(hashCode, numReducers);
-    assertEquals("Partitioner doesnt work as expected", expectedPartition, 
-                 kfbp.getPartition(new Text(input), new Text(), numReducers));
-    
-    // test with invalid start index in keyspecs
-    kfbp = new KeyFieldBasedPartitioner<Text, Text>();
-    conf = new JobConf();
-    conf.set("mapred.text.key.partitioner.options", "-k2,2 -k30,21 -k4,4 -k5");
-    kfbp.configure(conf);
-    expectedOutput = "def";
-    eBytes = expectedOutput.getBytes();
-    hashCode = kfbp.hashCode(eBytes, 0, eBytes.length - 1, 0);
-    expectedOutput = "xyz";
-    eBytes = expectedOutput.getBytes();
-    hashCode = kfbp.hashCode(eBytes, 0, eBytes.length - 1, hashCode);
-    expectedPartition = kfbp.getPartition(hashCode, numReducers);
-    assertEquals("Partitioner doesnt work as expected", expectedPartition, 
-                 kfbp.getPartition(new Text(input), new Text(), numReducers));
+                 0, kfbp.getPartition(new Text(), new Text(), 10));
   }
-}
+}

+ 0 - 425
src/test/org/apache/hadoop/mapred/lib/TestKeyFieldHelper.java

@@ -1,425 +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.mapred.lib;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import junit.framework.TestCase;
-
-public class TestKeyFieldHelper extends TestCase {
-  private static final Log LOG = LogFactory.getLog(TestKeyFieldHelper.class);
-  /**
-   * Test is key-field-helper's parse option.
-   */
-  public void testparseOption() throws Exception {
-    KeyFieldHelper helper = new KeyFieldHelper();
-    helper.setKeyFieldSeparator("\t");
-    String keySpecs = "-k1.2,3.4";
-    String eKeySpecs = keySpecs;
-    helper.parseOption(keySpecs);
-    String actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    
-    // test -k a.b
-    keySpecs = "-k 1.2";
-    eKeySpecs = "-k1.2,0.0";
-    helper = new KeyFieldHelper();
-    helper.parseOption(keySpecs);
-    actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    
-    keySpecs = "-nr -k1.2,3.4";
-    eKeySpecs = "-k1.2,3.4nr";
-    helper = new KeyFieldHelper();
-    helper.parseOption(keySpecs);
-    actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    
-    keySpecs = "-nr -k1.2,3.4n";
-    eKeySpecs = "-k1.2,3.4n";
-    helper = new KeyFieldHelper();
-    helper.parseOption(keySpecs);
-    actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    
-    keySpecs = "-nr -k1.2,3.4r";
-    eKeySpecs = "-k1.2,3.4r";
-    helper = new KeyFieldHelper();
-    helper.parseOption(keySpecs);
-    actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    
-    keySpecs = "-nr -k1.2,3.4 -k5.6,7.8n -k9.10,11.12r -k13.14,15.16nr";
-    //1st
-    eKeySpecs = "-k1.2,3.4nr";
-    helper = new KeyFieldHelper();
-    helper.parseOption(keySpecs);
-    actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    // 2nd
-    eKeySpecs = "-k5.6,7.8n";
-    actKeySpecs = helper.keySpecs().get(1).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    //3rd
-    eKeySpecs = "-k9.10,11.12r";
-    actKeySpecs = helper.keySpecs().get(2).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    //4th
-    eKeySpecs = "-k13.14,15.16nr";
-    actKeySpecs = helper.keySpecs().get(3).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    
-    keySpecs = "-k1.2n,3.4";
-    eKeySpecs = "-k1.2,3.4n";
-    helper = new KeyFieldHelper();
-    helper.parseOption(keySpecs);
-    actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    
-    keySpecs = "-k1.2r,3.4";
-    eKeySpecs = "-k1.2,3.4r";
-    helper = new KeyFieldHelper();
-    helper.parseOption(keySpecs);
-    actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    
-    keySpecs = "-k1.2nr,3.4";
-    eKeySpecs = "-k1.2,3.4nr";
-    helper = new KeyFieldHelper();
-    helper.parseOption(keySpecs);
-    actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    
-    keySpecs = "-k1.2,3.4n";
-    eKeySpecs = "-k1.2,3.4n";
-    helper = new KeyFieldHelper();
-    helper.parseOption(keySpecs);
-    actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    
-    keySpecs = "-k1.2,3.4r";
-    eKeySpecs = "-k1.2,3.4r";
-    helper = new KeyFieldHelper();
-    helper.parseOption(keySpecs);
-    actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    
-    keySpecs = "-k1.2,3.4nr";
-    eKeySpecs = "-k1.2,3.4nr";
-    helper = new KeyFieldHelper();
-    helper.parseOption(keySpecs);
-    actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    
-    keySpecs = "-nr -k1.2,3.4 -k5.6,7.8";
-    eKeySpecs = "-k1.2,3.4nr";
-    helper = new KeyFieldHelper();
-    helper.parseOption(keySpecs);
-    actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    eKeySpecs = "-k5.6,7.8nr";
-    actKeySpecs = helper.keySpecs().get(1).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    
-    keySpecs = "-n -k1.2,3.4 -k5.6,7.8";
-    eKeySpecs = "-k1.2,3.4n";
-    helper = new KeyFieldHelper();
-    helper.parseOption(keySpecs);
-    actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    eKeySpecs = "-k5.6,7.8n";
-    actKeySpecs = helper.keySpecs().get(1).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    
-    keySpecs = "-r -k1.2,3.4 -k5.6,7.8";
-    eKeySpecs = "-k1.2,3.4r";
-    helper = new KeyFieldHelper();
-    helper.parseOption(keySpecs);
-    actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    eKeySpecs = "-k5.6,7.8r";
-    actKeySpecs = helper.keySpecs().get(1).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    
-    keySpecs = "-k1.2,3.4n -k5.6,7.8";
-    eKeySpecs = "-k1.2,3.4n";
-    helper = new KeyFieldHelper();
-    helper.parseOption(keySpecs);
-    actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    eKeySpecs = "-k5.6,7.8";
-    actKeySpecs = helper.keySpecs().get(1).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    
-    keySpecs = "-k1.2,3.4r -k5.6,7.8";
-    eKeySpecs = "-k1.2,3.4r";
-    helper = new KeyFieldHelper();
-    helper.parseOption(keySpecs);
-    actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    eKeySpecs = "-k5.6,7.8";
-    actKeySpecs = helper.keySpecs().get(1).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    
-    keySpecs = "-k1.2,3.4nr -k5.6,7.8";
-    eKeySpecs = "-k1.2,3.4nr";
-    helper = new KeyFieldHelper();
-    helper.parseOption(keySpecs);
-    actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    eKeySpecs = "-k5.6,7.8";
-    actKeySpecs = helper.keySpecs().get(1).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    
-    keySpecs = "-n";
-    eKeySpecs = "-k1.1,0.0n";
-    helper = new KeyFieldHelper();
-    helper.parseOption(keySpecs);
-    actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    
-    keySpecs = "-r";
-    eKeySpecs = "-k1.1,0.0r";
-    helper = new KeyFieldHelper();
-    helper.parseOption(keySpecs);
-    actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-    
-    keySpecs = "-nr";
-    eKeySpecs = "-k1.1,0.0nr";
-    helper = new KeyFieldHelper();
-    helper.parseOption(keySpecs);
-    actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
-  }
-  
-  /**
-   * Test is key-field-helper's getWordLengths.
-   */
-  public void testGetWordLengths() throws Exception {
-    KeyFieldHelper helper = new KeyFieldHelper();
-    helper.setKeyFieldSeparator("\t");
-    // test getWordLengths with unspecified key-specifications
-    String input = "hi";
-    int[] result = helper.getWordLengths(input.getBytes(), 0, 2);
-    assertTrue(equals(result, new int[] {1}));
-    
-    // set the key specs
-    helper.setKeyFieldSpec(1, 2);
-    
-    // test getWordLengths with 3 words
-    input = "hi\thello there";
-    result = helper.getWordLengths(input.getBytes(), 0, input.length());
-    assertTrue(equals(result, new int[] {2, 2, 11}));
-    
-    // test getWordLengths with 4 words but with a different separator
-    helper.setKeyFieldSeparator(" ");
-    input = "hi hello\tthere you";
-    result = helper.getWordLengths(input.getBytes(), 0, input.length());
-    assertTrue(equals(result, new int[] {3, 2, 11, 3}));
-    
-    // test with non zero start index
-    input = "hi hello there you where me there";
-    //                 .....................
-    result = helper.getWordLengths(input.getBytes(), 10, 33);
-    assertTrue(equals(result, new int[] {5, 4, 3, 5, 2, 3}));
-    
-    input = "hi hello there you where me ";
-    //                 ..................
-    result = helper.getWordLengths(input.getBytes(), 10, input.length());
-    assertTrue(equals(result, new int[] {5, 4, 3, 5, 2, 0}));
-    
-    input = "";
-    result = helper.getWordLengths(input.getBytes(), 0, 0);
-    assertTrue(equals(result, new int[] {1, 0}));
-    
-    input = "  abc";
-    result = helper.getWordLengths(input.getBytes(), 0, 5);
-    assertTrue(equals(result, new int[] {3, 0, 0, 3}));
-    
-    input = "  abc";
-    result = helper.getWordLengths(input.getBytes(), 0, 2);
-    assertTrue(equals(result, new int[] {3, 0, 0, 0}));
-    
-    input = " abc ";
-    result = helper.getWordLengths(input.getBytes(), 0, 2);
-    assertTrue(equals(result, new int[] {2, 0, 1}));
-    
-    helper.setKeyFieldSeparator("abcd");
-    input = "abc";
-    result = helper.getWordLengths(input.getBytes(), 0, 3);
-    assertTrue(equals(result, new int[] {1, 3}));
-  }
-  
-  /**
-   * Test is key-field-helper's getStartOffset/getEndOffset.
-   */
-  public void testgetStartEndOffset() throws Exception {
-    KeyFieldHelper helper = new KeyFieldHelper();
-    helper.setKeyFieldSeparator("\t");
-    // test getStartOffset with -k1,2
-    helper.setKeyFieldSpec(1, 2);
-    String input = "hi\thello";
-    String expectedOutput = input;
-    testKeySpecs(input, expectedOutput, helper);
-    
-    // test getStartOffset with -k1.0,0 .. should result into start = -1
-    helper = new KeyFieldHelper();
-    helper.setKeyFieldSeparator("\t");
-    helper.parseOption("-k1.0,0");
-    testKeySpecs(input, null, helper);
-    
-    // test getStartOffset with -k1,0
-    helper = new KeyFieldHelper();
-    helper.setKeyFieldSeparator("\t");
-    helper.parseOption("-k1,0");
-    expectedOutput = input;
-    testKeySpecs(input, expectedOutput, helper);
-    
-    // test getStartOffset with -k1.2,0
-    helper = new KeyFieldHelper();
-    helper.setKeyFieldSeparator("\t");
-    helper.parseOption("-k1.2,0");
-    expectedOutput = "i\thello";
-    testKeySpecs(input, expectedOutput, helper);
-    
-    // test getWordLengths with -k1.0,2.3
-    helper = new KeyFieldHelper();
-    helper.setKeyFieldSeparator("\t");
-    helper.parseOption("-k1.1,2.3");
-    expectedOutput = "hi\thel";
-    testKeySpecs(input, expectedOutput, helper);
-    
-    // test getWordLengths with -k1.2,2.3
-    helper = new KeyFieldHelper();
-    helper.setKeyFieldSeparator("\t");
-    helper.parseOption("-k1.2,2.3");
-    expectedOutput = "i\thel";
-    testKeySpecs(input, expectedOutput, helper);
-    
-    // test getStartOffset with -k1.2,3.0
-    helper = new KeyFieldHelper();
-    helper.setKeyFieldSeparator("\t");
-    helper.parseOption("-k1.2,3.0");
-    expectedOutput = "i\thello";
-    testKeySpecs(input, expectedOutput, helper);
-    
-    // test getStartOffset with -k2,2
-    helper = new KeyFieldHelper();
-    helper.setKeyFieldSeparator("\t");
-    helper.parseOption("-k2,2");
-    expectedOutput = "hello";
-    testKeySpecs(input, expectedOutput, helper);
-    
-    // test getStartOffset with -k3.0,4.0
-    helper = new KeyFieldHelper();
-    helper.setKeyFieldSeparator("\t");
-    helper.parseOption("-k3.1,4.0");
-    testKeySpecs(input, null, helper);
-    
-    // test getStartOffset with -k2.1
-    helper = new KeyFieldHelper();
-    input = "123123123123123hi\thello\thow";
-    helper.setKeyFieldSeparator("\t");
-    helper.parseOption("-k2.1");
-    expectedOutput = "hello\thow";
-    testKeySpecs(input, expectedOutput, helper, 15, input.length());
-    
-    // test getStartOffset with -k2.1,4 with end ending on \t
-    helper = new KeyFieldHelper();
-    input = "123123123123123hi\thello\t\thow\tare";
-    helper.setKeyFieldSeparator("\t");
-    helper.parseOption("-k2.1,3");
-    expectedOutput = "hello\t";
-    testKeySpecs(input, expectedOutput, helper, 17, input.length());
-    
-    // test getStartOffset with -k2.1 with end ending on \t
-    helper = new KeyFieldHelper();
-    input = "123123123123123hi\thello\thow\tare";
-    helper.setKeyFieldSeparator("\t");
-    helper.parseOption("-k2.1");
-    expectedOutput = "hello\thow\t";
-    testKeySpecs(input, expectedOutput, helper, 17, 28);
-    
-    // test getStartOffset with -k2.1,3 with smaller length
-    helper = new KeyFieldHelper();
-    input = "123123123123123hi\thello\thow";
-    helper.setKeyFieldSeparator("\t");
-    helper.parseOption("-k2.1,3");
-    expectedOutput = "hello";
-    testKeySpecs(input, expectedOutput, helper, 15, 23);
-  }
-  
-  private void testKeySpecs(String input, String expectedOutput, 
-                            KeyFieldHelper helper) {
-    testKeySpecs(input, expectedOutput, helper, 0, -1);
-  }
-  
-  private void testKeySpecs(String input, String expectedOutput, 
-                            KeyFieldHelper helper, int s1, int e1) {
-    LOG.info("input : " + input);
-    String keySpecs = helper.keySpecs().get(0).toString();
-    LOG.info("keyspecs : " + keySpecs);
-    byte[] inputBytes = input.getBytes(); // get the input bytes
-    if (e1 == -1) {
-      e1 = inputBytes.length;
-    }
-    LOG.info("length : " + e1);
-    // get the word lengths
-    int[] indices = helper.getWordLengths(inputBytes, s1, e1);
-    // get the start index
-    int start = helper.getStartOffset(inputBytes, s1, e1, indices, 
-                                      helper.keySpecs().get(0));
-    LOG.info("start : " + start);
-    if (expectedOutput == null) {
-      assertEquals("Expected -1 when the start index is invalid", -1, start);
-      return;
-    }
-    // get the end index
-    int end = helper.getEndOffset(inputBytes, s1, e1, indices, 
-                                  helper.keySpecs().get(0));
-    LOG.info("end : " + end);
-    //my fix
-    end = (end >= inputBytes.length) ? inputBytes.length -1 : end;
-    int length = end + 1 - start;
-    LOG.info("length : " + length);
-    byte[] outputBytes = new byte[length];
-    System.arraycopy(inputBytes, start, outputBytes, 0, length);
-    String output = new String(outputBytes);
-    LOG.info("output : " + output);
-    LOG.info("expected-output : " + expectedOutput);
-    assertEquals(keySpecs + " failed on input '" + input + "'", 
-                 expectedOutput, output);
-  }
-
-  // check for equality of 2 int arrays
-  private boolean equals(int[] test, int[] expected) {
-    // check array length
-    if (test[0] != expected[0]) {
-      return false;
-    }
-    // if length is same then check the contents
-    for (int i = 0; i < test[0] && i < expected[0]; ++i) {
-      if (test[i] != expected[i]) {
-        return false;
-      }
-    }
-    return true;
-  }
-}