You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mahout.apache.org by dr...@apache.org on 2010/03/06 17:38:03 UTC

svn commit: r919798 - in /lucene/mahout/trunk/utils/src: main/java/org/apache/mahout/utils/nlp/collocations/llr/ test/java/org/apache/mahout/utils/nlp/collocations/llr/

Author: drew
Date: Sat Mar  6 16:38:03 2010
New Revision: 919798

URL: http://svn.apache.org/viewvc?rev=919798&view=rev
Log:
MAHOUT-317: llr collocation improvements 

Added:
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKey.java   (with props)
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyGroupComparator.java   (with props)
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitioner.java   (with props)
    lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyGroupComparatorTest.java   (with props)
    lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitionerTest.java   (with props)
    lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyTest.java   (with props)
Modified:
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocCombiner.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocDriver.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocMapper.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocReducer.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/Gram.java
    lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/CollocMapperTest.java
    lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/CollocReducerTest.java
    lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramTest.java

Modified: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocCombiner.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocCombiner.java?rev=919798&r1=919797&r2=919798&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocCombiner.java (original)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocCombiner.java Sat Mar  6 16:38:03 2010
@@ -18,7 +18,6 @@
 package org.apache.mahout.utils.nlp.collocations.llr;
 
 import java.io.IOException;
-import java.util.HashMap;
 import java.util.Iterator;
 
 import org.apache.hadoop.mapred.MapReduceBase;
@@ -26,52 +25,27 @@
 import org.apache.hadoop.mapred.Reducer;
 import org.apache.hadoop.mapred.Reporter;
 
-/** Combiner for pass1 of the CollocationDriver */
-public class CollocCombiner extends MapReduceBase implements Reducer<Gram,Gram,Gram,Gram> {
+/** Combiner for pass1 of the CollocationDriver. Combines frequencies for values for the same key */
+public class CollocCombiner extends MapReduceBase implements Reducer<GramKey,Gram,GramKey,Gram> {
   
-  /**
-   * collocation finder: pass 1 colloc phase:
-   * 
-   * given input from the mapper, k:h_subgram:1 v:ngram:1 k:t_subgram:1 v:ngram:1
-   * 
-   * count ngrams and subgrams.
-   * 
-   * output is:
-   * 
-   * k:h_subgram:subgramfreq v:ngram:ngramfreq k:t_subgram:subgramfreq v:ngram:ngramfreq
-   * 
-   * Each ngram's frequency is essentially counted twice, frequency should be the same for the head and tail.
-   * Fix this to count only for the head and move the count into the value?
-   */
   @Override
-  public void reduce(Gram subgramKey,
-                     Iterator<Gram> ngramValues,
-                     OutputCollector<Gram,Gram> output,
+  public void reduce(GramKey key,
+                     Iterator<Gram> values,
+                     OutputCollector<GramKey,Gram> output,
                      Reporter reporter) throws IOException {
+
+    int freq = 0;
+    Gram value = null;
     
-    HashMap<Gram,Gram> ngramSet = new HashMap<Gram,Gram>();
-    int subgramFrequency = 0;
-    
-    while (ngramValues.hasNext()) {
-      Gram ngram = ngramValues.next();
-      subgramFrequency += ngram.getFrequency();
-      
-      Gram ngramCanon = ngramSet.get(ngram);
-      if (ngramCanon == null) {
-        // t is potentially reused, so create a new object to populate the HashMap
-        Gram ngramEntry = new Gram(ngram);
-        ngramSet.put(ngramEntry, ngramEntry);
-      } else {
-        ngramCanon.incrementFrequency(ngram.getFrequency());
-      }
+    // accumulate frequencies from values.
+    while (values.hasNext()) {
+      value = values.next();
+      freq += value.getFrequency();
     }
+
+    value.setFrequency(freq);
     
-    // emit subgram:subgramFreq ngram:ngramFreq pairs
-    subgramKey.setFrequency(subgramFrequency);
-    
-    for (Gram ngram : ngramSet.keySet()) {
-      output.collect(subgramKey, ngram);
-    }
+    output.collect(key, value);
   }
   
 }

Modified: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocDriver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocDriver.java?rev=919798&r1=919797&r2=919798&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocDriver.java (original)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocDriver.java Sat Mar  6 16:38:03 2010
@@ -27,6 +27,7 @@
 import org.apache.commons.cli2.builder.DefaultOptionBuilder;
 import org.apache.commons.cli2.builder.GroupBuilder;
 import org.apache.commons.cli2.commandline.Parser;
+import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.Text;
@@ -38,6 +39,8 @@
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
 import org.apache.hadoop.mapred.SequenceFileOutputFormat;
 import org.apache.hadoop.mapred.lib.IdentityMapper;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.mahout.common.CommandLineUtil;
@@ -47,8 +50,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/** Driver for LLR collocation discovery mapreduce job */
-public class CollocDriver {
+/** Driver for LLR Collocation discovery mapreduce job */
+public class CollocDriver extends Configured implements Tool {
   public static final String DEFAULT_OUTPUT_DIRECTORY = "output";
   public static final String SUBGRAM_OUTPUT_DIRECTORY = "subgrams";
   public static final String NGRAM_OUTPUT_DIRECTORY = "ngrams";
@@ -64,10 +67,13 @@
   private CollocDriver() {
   }
 
+  public static void main(String[] args) throws Exception {
+    ToolRunner.run(new CollocDriver(), args);
+  }
   /**
    * @param args
    */
-  public static void main(String[] args) throws Exception {
+  public int run(String[] args) throws Exception {
     DefaultOptionBuilder obuilder = new DefaultOptionBuilder();
     ArgumentBuilder abuilder = new ArgumentBuilder();
     GroupBuilder gbuilder = new GroupBuilder();
@@ -131,7 +137,7 @@
       
       if (cmdLine.hasOption(helpOpt)) {
         CommandLineUtil.printHelp(group);
-        return;
+        return 1;
       }
       
       String input = cmdLine.getValue(inputOpt).toString();
@@ -202,8 +208,10 @@
     } catch (OptionException e) {
       log.error("Exception", e);
       CommandLineUtil.printHelp(group);
+      return 1;
     }
     
+    return 0;
   }
   
   /**
@@ -247,9 +255,12 @@
                                           int reduceTasks,
                                           int minSupport) throws IOException {
     JobConf conf = new JobConf(CollocDriver.class);
+    conf.setJobName(CollocDriver.class.getSimpleName() + ".generateCollocations:" + input);
     
-    conf.setMapOutputKeyClass(Gram.class);
+    conf.setMapOutputKeyClass(GramKey.class);
     conf.setMapOutputValueClass(Gram.class);
+    conf.setPartitionerClass(GramKeyPartitioner.class);
+    conf.setOutputValueGroupingComparator(GramKeyGroupComparator.class);
     
     conf.setOutputKeyClass(Gram.class);
     conf.setOutputValueClass(Gram.class);
@@ -284,6 +295,8 @@
                                              float minLLRValue,
                                              int reduceTasks) throws IOException {
     JobConf conf = new JobConf(CollocDriver.class);
+    conf.setJobName(CollocDriver.class.getSimpleName() + ".computeNGrams: " + output);
+    
     
     conf.setLong(LLRReducer.NGRAM_TOTAL, nGramTotal);
     conf.setBoolean(EMIT_UNIGRAMS, emitUnigrams);

Modified: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocMapper.java?rev=919798&r1=919797&r2=919798&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocMapper.java (original)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocMapper.java Sat Mar  6 16:38:03 2010
@@ -37,11 +37,11 @@
 import org.slf4j.LoggerFactory;
 
 /**
- * Runs pass 1 of the Collocation discovery job on input of SequeceFile<Text,Text>, where the key is a
- * document id and the value is the document contents. . Delegates to NGramCollector to perform tokenization,
- * ngram-creation and output collection.
+ * Pass 1 of the Collocation discovery job which generated ngrams and emits ngrams an their component n-1grams. 
+ * Input is a SequeceFile<Text,StringTuple>, where the key is a document id and the value is the tokenized documents.
+ * <p/>
  */
-public class CollocMapper extends MapReduceBase implements Mapper<Text,StringTuple,Gram,Gram> {
+public class CollocMapper extends MapReduceBase implements Mapper<Text,StringTuple,GramKey,Gram> {
   
   public static final String MAX_SHINGLE_SIZE = "maxShingleSize";
   public static final int DEFAULT_MAX_SHINGLE_SIZE = 2;
@@ -71,21 +71,32 @@
   
   /**
    * Collocation finder: pass 1 map phase.
-   * 
-   * Receives a token stream which gets passed through the ShingleFilter. The ShingleFilter delivers ngrams of
+   * <p/>
+   * Receives a token stream which gets passed through a Lucene ShingleFilter. The ShingleFilter delivers ngrams of
    * the appropriate size which are then decomposed into head and tail subgrams which are collected in the
    * following manner
    * 
-   * k:h_subgram v:ngram k:t_subgram v:ngram
+   * <pre>
+   * k:head_key,           v:head_subgram
+   * k:head_key,ngram_key, v:ngram
+   * k:tail_key,           v:tail_subgram
+   * k:tail_key,ngram_key, v:ngram
+   * </pre>
    * 
-   * The 'h_' or 't_' prefix is used to specify whether the subgram in question is the head or tail of the
+   * The 'head' or 'tail' prefix is used to specify whether the subgram in question is the head or tail of the
    * ngram. In this implementation the head of the ngram is a (n-1)gram, and the tail is a (1)gram.
-   * 
-   * For example, given 'click and clack' and an ngram length of 3: k:'h_click and' v:'click and clack'
-   * k;'t_clack' v:'click and clack'
+   * <p/>
+   * For example, given 'click and clack' and an ngram length of 3: 
+   * <pre>
+   * k: head_'click and'                         v:head_'click and'
+   * k: head_'click and',ngram_'click and clack' v:ngram_'click and clack'
+   * k: tail_'clack',                            v:tail_'clack'
+   * k: tail_'clack',ngram_'click and clack'     v:ngram_'click and clack'
+   * </pre>
    * 
    * Also counts the total number of ngrams encountered and adds it to the counter
    * CollocDriver.Count.NGRAM_TOTAL
+   * </p>
    * 
    * @param collector
    *          The collector to send output to
@@ -98,7 +109,7 @@
    */
   @Override
   public void map(Text key, StringTuple value,
-                  final OutputCollector<Gram,Gram> collector, Reporter reporter) throws IOException {
+                  final OutputCollector<GramKey,Gram> collector, Reporter reporter) throws IOException {
     
     ShingleFilter sf = new ShingleFilter(new IteratorTokenStream(value.getEntries().iterator()), maxShingleSize);
     int count = 0; // ngram count
@@ -119,16 +130,33 @@
     } while (sf.incrementToken());
     
     try {
+      final byte[] empty = new byte[0];
+      final GramKey gramKey = new GramKey();
+      
       ngrams.forEachPair(new ObjectIntProcedure<String>() {
         @Override
         public boolean apply(String term, int frequency) {
           // obtain components, the leading (n-1)gram and the trailing unigram.
           int i = term.lastIndexOf(' '); // TODO: fix for non-whitespace delimited languages.
           if (i != -1) { // bigram, trigram etc
-            Gram ngram = new Gram(term, frequency, Gram.Type.NGRAM);
+            
             try {
-              collector.collect(new Gram(term.substring(0, i), frequency, Gram.Type.HEAD), ngram);
-              collector.collect(new Gram(term.substring(i + 1), frequency, Gram.Type.TAIL), ngram);
+              Gram ngram = new Gram(term, frequency, Gram.Type.NGRAM);
+              Gram head  = new Gram(term.substring(0, i), frequency, Gram.Type.HEAD);
+              Gram tail  = new Gram(term.substring(i + 1), frequency, Gram.Type.TAIL);
+              
+              gramKey.set(head, empty);
+              collector.collect(gramKey, head);
+              
+              gramKey.set(head, ngram.getBytes());
+              collector.collect(gramKey, ngram);
+              
+              gramKey.set(tail, empty);
+              collector.collect(gramKey, tail);
+              
+              gramKey.set(tail, ngram.getBytes());
+              collector.collect(gramKey, ngram);
+              
             } catch (IOException e) {
               throw new IllegalStateException(e);
             }
@@ -142,7 +170,8 @@
         public boolean apply(String term, int frequency) {
           try {
             Gram unigram = new Gram(term, frequency, Gram.Type.UNIGRAM);
-            collector.collect(unigram, unigram);
+            gramKey.set(unigram, empty);
+            collector.collect(gramKey, unigram);
           } catch (IOException e) {
             throw new IllegalStateException(e);
           }

Modified: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocReducer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocReducer.java?rev=919798&r1=919797&r2=919798&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocReducer.java (original)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocReducer.java Sat Mar  6 16:38:03 2010
@@ -32,7 +32,7 @@
 /**
  * Reducer for Pass 1 of the collocation identification job. Generates counts for ngrams and subgrams.
  */
-public class CollocReducer extends MapReduceBase implements Reducer<Gram,Gram,Gram,Gram> {
+public class CollocReducer extends MapReduceBase implements Reducer<GramKey,Gram,Gram,Gram> {
 
   private static final Logger log = LoggerFactory.getLogger(CollocReducer.class);
 
@@ -40,7 +40,11 @@
   public static final int DEFAULT_MIN_SUPPORT = 2;
   
   public enum Skipped {
-    LESS_THAN_MIN_SUPPORT
+    LESS_THAN_MIN_SUPPORT,
+    MALFORMED_KEY_TUPLE,
+    MALFORMED_TUPLE,
+    MALFORMED_TYPES,
+    MALFORMED_UNIGRAM
   }
 
   private int minSupport;
@@ -60,51 +64,140 @@
   
   /**
    * collocation finder: pass 1 reduce phase:
+   * <p/>
+   * given input from the mapper, 
    * 
-   * given input from the mapper, k:h_subgram v:ngram k:t_subgram v:ngram
-   * 
-   * count ngrams and subgrams.
-   * 
+   * <pre>
+   * k:head_subgram,ngram,  v:ngram:partial freq
+   * k:head_subgram         v:head_subgram:partial freq
+   * k:tail_subgram,ngram,  v:ngram:partial freq
+   * k:tail_subgram         v:tail_subgram:partial freq
+   * k:unigram              v:unigram:partial freq
+   * </pre>
+   * sum gram frequencies and output for llr calculation
+   * <p/>
    * output is:
-   * 
-   * k:ngram:ngramfreq v:h_subgram:h_subgramfreq k:ngram:ngramfreq v:t_subgram:t_subgramfreq
-   * 
-   * Each ngram's frequency is essentially counted twice, frequency should be the same for the head and tail.
-   * Fix this to count only for the head and move the count into the value?
+   * <pre>
+   * k:ngram:ngramfreq      v:head_subgram:head_subgramfreq
+   * k:ngram:ngramfreq      v:tail_subgram:tail_subgramfreq
+   * k:unigram:unigramfreq  v:unigram:unigramfreq
+   * </pre>
+   * Each ngram's frequency is essentially counted twice, once for head, once for tail. 
+   * frequency should be the same for the head and tail. Fix this to count only for the 
+   * head and move the count into the value?
    */
   @Override
-  public void reduce(Gram subgramKey,
-                     Iterator<Gram> ngramValues,
+  public void reduce(GramKey key,
+                     Iterator<Gram> values,
                      OutputCollector<Gram,Gram> output,
                      Reporter reporter) throws IOException {
     
-    HashMap<Gram,Gram> ngramSet = new HashMap<Gram,Gram>();
-    int subgramFrequency = 0;
+    Gram.Type keyType = key.getType();
+
+    if (keyType == Gram.Type.UNIGRAM) {
+      // sum frequencies for unigrams.
+      processUnigram(key, values, output, reporter);
+    }
+    else if (keyType == Gram.Type.HEAD || keyType == Gram.Type.TAIL) {
+      // sum frequencies for subgrams, ngram and collect for each ngram.
+      processSubgram(key, values, output, reporter);
+    }
+    else {
+      reporter.incrCounter(Skipped.MALFORMED_TYPES, 1);
+    }
+  }
+
+  /** Sum frequencies for unigrams and deliver to the collector 
+   * 
+   * @param keyFirst
+   * @param values
+   * @param output
+   * @param reporter
+   * @throws IOException
+   */
+  protected void processUnigram(GramKey key, Iterator<Gram> values,
+      OutputCollector<Gram, Gram> output, Reporter reporter) throws IOException {
+
+    int freq = 0;
+    Gram value = null;
     
-    while (ngramValues.hasNext()) {
-      Gram ngram = ngramValues.next();
-      subgramFrequency += ngram.getFrequency();
+    // accumulate frequencies from values.
+    while (values.hasNext()) {
+      value = values.next();
+      freq += value.getFrequency();
+    }
+
+    if (freq < minSupport) {
+      reporter.incrCounter(Skipped.LESS_THAN_MIN_SUPPORT, 1);
+      return;
+    }
+
+    value.setFrequency(freq);
+    output.collect(value, value);
+
+  }
       
-      Gram ngramCanon = ngramSet.get(ngram);
-      if (ngramCanon == null) {
-        // t is potentially reused, so create a new object to populate the
-        // HashMap
-        Gram ngramEntry = new Gram(ngram);
-        ngramSet.put(ngramEntry, ngramEntry);
-      } else {
-        ngramCanon.incrementFrequency(ngram.getFrequency());
+  /** Sum frequencies for subgram, ngrams and deliver ngram, subgram pairs to the collector.
+   *  <p/>
+   *  Sort order guarantees that the subgram/subgram pairs will be seen first and then
+   *  subgram/ngram1 pairs, subgram/ngram2 pairs ... subgram/ngramN pairs, so frequencies for
+   *  ngrams can be calcualted here as well.
+   *  <p/>
+   *  We end up calculating frequencies for ngrams for each sugram (head, tail) here, which is
+   *  some extra work.
+   *  
+   * @param keyFirst
+   * @param values
+   * @param output
+   * @param reporter
+   * @throws IOException
+   */
+  protected void processSubgram(GramKey key, Iterator<Gram> values, 
+      OutputCollector<Gram,Gram> output, Reporter reporter) throws IOException {
+
+    Gram subgram      = null;
+    Gram currentNgram = null;
+        
+    while (values.hasNext()) {
+      Gram value = values.next();
+
+      if (value.getType() == Gram.Type.HEAD || value.getType() == Gram.Type.TAIL) { 
+        // collect frequency for subgrams.
+        if (subgram == null) {
+          subgram = new Gram(value);
+        }
+        else {
+          subgram.incrementFrequency(value.getFrequency());
+        }
+      }
+      else if (!value.equals(currentNgram)) {
+        // we've collected frequency for all subgrams and we've encountered a new ngram. 
+        // collect the old ngram if there was one and we have sufficient support and
+        // create the new ngram.
+        if (currentNgram != null) {
+          if (currentNgram.getFrequency() < minSupport) {
+            reporter.incrCounter(Skipped.LESS_THAN_MIN_SUPPORT, 1);
+          }
+          else {
+            output.collect(currentNgram, subgram);
+          }
+        }
+
+        currentNgram = new Gram(value);
+      }
+      else {
+        currentNgram.incrementFrequency(value.getFrequency());
       }
     }
     
-    // emit ngram:ngramFreq, subgram:subgramFreq pairs.
-    subgramKey.setFrequency(subgramFrequency);
-    
-    for (Gram ngram : ngramSet.keySet()) {
-      if (ngram.getFrequency() < minSupport) {
+    // collect last ngram.
+    if (currentNgram != null) {
+      if (currentNgram.getFrequency() < minSupport) {
         reporter.incrCounter(Skipped.LESS_THAN_MIN_SUPPORT, 1);
-        continue;
+        return;
       }
-      output.collect(ngram, subgramKey);
+      
+      output.collect(currentNgram, subgram);
     }
   }
 }

Modified: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/Gram.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/Gram.java?rev=919798&r1=919797&r2=919798&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/Gram.java (original)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/Gram.java Sat Mar  6 16:38:03 2010
@@ -42,7 +42,8 @@
     UNIGRAM('u'),
     NGRAM('n');
     
-    private final char x;
+    final char x;
+    
     Type(char c) {
       this.x = c;
     }
@@ -101,14 +102,14 @@
     try {  
       // extra character is used for storing type which is part 
       // of the sort key.
-      ByteBuffer bb = Text.encode(ngram + '\0', true);
+      ByteBuffer bb = Text.encode('\0' + ngram, true);
       bytes = bb.array();
       length = bb.limit();
     } catch (CharacterCodingException e) {
       throw new IllegalStateException("Should not have happened ",e);
     }
     
-    encodeType(type, bytes, length-1);
+    encodeType(type, bytes, 0);
     this.frequency = frequency;
   }
   
@@ -127,7 +128,7 @@
    * @return the gram is at the head of its text unit or tail or unigram.
    */
   public Type getType() {
-    return decodeType(bytes, length-1);
+    return decodeType(bytes, 0);
   }
 
   /**
@@ -135,7 +136,7 @@
    */
   public String getString() {
     try {
-      return Text.decode(bytes, 0, length-1);
+      return Text.decode(bytes, 1, length-1);
     } catch (CharacterCodingException e) {
       throw new IllegalStateException("Should not have happened " + e.toString()); 
     }

Added: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKey.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKey.java?rev=919798&view=auto
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKey.java (added)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKey.java Sat Mar  6 16:38:03 2010
@@ -0,0 +1,133 @@
+/**
+ * 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.mahout.utils.nlp.collocations.llr;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.charset.CharacterCodingException;
+
+import org.apache.hadoop.io.BinaryComparable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.mahout.utils.nlp.collocations.llr.Gram.Type;
+
+/** A GramKey, based on the identity fields of Gram (type, string) plus a byte[] used for secondary ordering */
+public class GramKey extends BinaryComparable implements
+    WritableComparable<BinaryComparable> {
+
+  int primaryLength;
+  int length;
+  byte[] bytes;
+  
+  public GramKey() {
+    
+  }
+  
+  /** create a GramKey based on the specified Gram and order
+   * 
+   * @param gram
+   * @param order
+   */
+  public GramKey(Gram gram, byte[] order) {
+    set(gram, order);
+  }
+  
+  /** set the gram held by this key */
+  public void set(Gram gram, byte[] order) {
+    primaryLength = gram.getLength();
+    length = primaryLength + order.length;
+    setCapacity(length, false);
+    System.arraycopy(gram.getBytes(), 0, bytes, 0, primaryLength);
+    if (order.length > 0) {
+      System.arraycopy(order, 0, bytes, primaryLength, order.length);
+    }
+  }
+
+  @Override
+  public byte[] getBytes() {
+    return bytes;
+  }
+
+  @Override
+  public int getLength() {
+    return length;
+  }
+
+  public int getPrimaryLength() {
+    return primaryLength;
+  }
+  
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    int newLength = WritableUtils.readVInt(in);
+    int newPrimaryLength = WritableUtils.readVInt(in);
+    setCapacity(newLength, false);
+    in.readFully(bytes, 0, newLength);
+    length = newLength;
+    primaryLength = newPrimaryLength;
+
+  }
+  
+  @Override
+  public void write(DataOutput out) throws IOException {
+    WritableUtils.writeVInt(out, length);
+    WritableUtils.writeVInt(out, primaryLength);
+    out.write(bytes, 0, length);
+  }
+  
+  /* Cribbed from o.a.hadoop.io.Text:
+   * Sets the capacity of this object to <em>at least</em>
+   * <code>len</code> bytes. If the current buffer is longer,
+   * then the capacity and existing content of the buffer are
+   * unchanged. If <code>len</code> is larger
+   * than the current capacity, this object's capacity is
+   * increased to match.
+   * @param len the number of bytes we need
+   * @param keepData should the old data be kept
+   */
+  private void setCapacity(int len, boolean keepData) {
+    if (bytes == null || bytes.length < len) {
+      byte[] newBytes = new byte[len];
+      if (bytes != null && keepData) {
+        System.arraycopy(bytes, 0, newBytes, 0, length);
+      }
+      bytes = newBytes;
+    }
+  }
+  
+  /**
+   * @return the gram is at the head of its text unit or tail or unigram.
+   */
+  public Type getType() {
+    return Gram.decodeType(bytes, 0);
+  }
+
+  public String getPrimaryString() {
+    try {
+      return Text.decode(bytes, 1, primaryLength-1);
+    } catch (CharacterCodingException e) {
+      throw new RuntimeException("Should not have happened " + e.toString()); 
+    }
+  }
+  
+  public String toString() {
+    return '\'' + getPrimaryString() + "'[" + getType().x + "]";
+  }
+}

Propchange: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKey.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyGroupComparator.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyGroupComparator.java?rev=919798&view=auto
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyGroupComparator.java (added)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyGroupComparator.java Sat Mar  6 16:38:03 2010
@@ -0,0 +1,41 @@
+/**
+ * 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.mahout.utils.nlp.collocations.llr;
+
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableComparator;
+
+/** Group GramKeys based on their Gram, ignoring the secondary sort key, so that all keys with the same Gram are sent
+ *  to the same call of the reduce method, sorted in natural order (for GramKeys).
+ */
+public class GramKeyGroupComparator extends WritableComparator {
+
+  protected GramKeyGroupComparator() {
+    super(GramKey.class, true);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public int compare(WritableComparable a, WritableComparable b) {
+    GramKey gka = (GramKey) a;
+    GramKey gkb = (GramKey) b;
+
+    return WritableComparator.compareBytes(gka.getBytes(), 0, gka.getPrimaryLength(), gkb.getBytes(), 0, gkb.getPrimaryLength());
+  }
+
+}

Propchange: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyGroupComparator.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitioner.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitioner.java?rev=919798&view=auto
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitioner.java (added)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitioner.java Sat Mar  6 16:38:03 2010
@@ -0,0 +1,51 @@
+/**
+ * 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.mahout.utils.nlp.collocations.llr;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Partitioner;
+
+/** Partition GramKeys based on their Gram, ignoring the secondary sort key so that all GramKeys with the same
+ *  gram are sent to the same partition.
+ */
+public class GramKeyPartitioner implements Partitioner<GramKey, Gram> {
+
+  private static final String HASH_OFFSET_PROPERTY_NAME = "grampartitioner.hash.offset";
+    
+  public static void setOffsets(Configuration conf, int left, int right) {
+    conf.setInt(HASH_OFFSET_PROPERTY_NAME, left);
+  }
+  
+  int offset;
+
+  @Override
+  public int getPartition(GramKey key, Gram value, int numPartitions) {
+    // see: http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapreduce/lib/partition/BinaryPartitioner.java?revision=816664&view=markup
+    int length = key.getLength()-1;
+    int right   = (offset + length) % length;
+    int hash   = WritableComparator.hashBytes(key.getBytes(), right);
+    return (hash & Integer.MAX_VALUE) % numPartitions;
+  }
+
+  @Override
+  public void configure(JobConf conf) {
+    offset = conf.getInt(HASH_OFFSET_PROPERTY_NAME, -1);
+  }
+}

Propchange: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitioner.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/CollocMapperTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/CollocMapperTest.java?rev=919798&r1=919797&r2=919798&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/CollocMapperTest.java (original)
+++ lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/CollocMapperTest.java Sat Mar  6 16:38:03 2010
@@ -35,14 +35,14 @@
 import org.junit.Test;
 
 /**
- * Test for CollocMapper FIXME: Add negative test cases
+ * Test for CollocMapper 
  */
 @SuppressWarnings("deprecation")
 public class CollocMapperTest {
   
-  private OutputCollector<Gram,Gram> collector;
+  private OutputCollector<GramKey,Gram> collector;
   private Reporter reporter;
-  
+
   @Before
   @SuppressWarnings("unchecked")
   public void setUp() {
@@ -82,9 +82,14 @@
       if (v[1].equals("of times")) {
         frequency = 2;
       }
+      
       Gram subgram = new Gram(v[0].substring(2), frequency, p);
-      Gram ngram = new Gram(v[1], frequency, Type.NGRAM);
-      collector.collect(subgram, ngram);
+      Gram ngram = new Gram(v[1], frequency, Gram.Type.NGRAM);
+      
+      GramKey subgramKey = new GramKey(subgram, new byte[0]);
+      GramKey subgramNgramKey = new GramKey(subgram, ngram.getBytes());
+      collector.collect(subgramKey, subgram);
+      collector.collect(subgramNgramKey, ngram);
     }
     
     reporter.incrCounter(CollocMapper.Count.NGRAM_TOTAL, 7);
@@ -129,6 +134,7 @@
                                          {"u_worst", "worst"}, {"u_of", "of"},
                                          {"u_the", "the"}, {"u_best", "best"},
                                          {"u_times", "times"},};
+
     // set up expectations for mocks. ngram max size = 2
     for (String[] v : values) {
       Type p = v[0].startsWith("h") ? Gram.Type.HEAD : Gram.Type.TAIL;
@@ -138,9 +144,23 @@
           || v[1].equals("the")) {
         frequency = 2;
       }
-      Gram subgram = new Gram(v[0].substring(2), frequency, p);
-      Gram ngram = new Gram(v[1], frequency, p == Gram.Type.UNIGRAM ? Gram.Type.UNIGRAM : Gram.Type.NGRAM);
-      collector.collect(subgram, ngram);
+      
+      
+     
+      if (p == Gram.Type.UNIGRAM) {
+        Gram unigram = new Gram(v[1], frequency, Gram.Type.UNIGRAM);
+        GramKey unigramKey = new GramKey(unigram, new byte[0]);
+        collector.collect(unigramKey, unigram);
+      }
+      else {
+        Gram subgram = new Gram(v[0].substring(2), frequency, p);
+        Gram ngram = new Gram(v[1], frequency, Gram.Type.NGRAM);
+        
+        GramKey subgramKey = new GramKey(subgram, new byte[0]);
+        GramKey subgramNgramKey = new GramKey(subgram, ngram.getBytes());
+        collector.collect(subgramKey, subgram);
+        collector.collect(subgramNgramKey, ngram);
+      }
     }
     
     reporter.incrCounter(CollocMapper.Count.NGRAM_TOTAL, 7);

Modified: lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/CollocReducerTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/CollocReducerTest.java?rev=919798&r1=919797&r2=919798&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/CollocReducerTest.java (original)
+++ lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/CollocReducerTest.java Sat Mar  6 16:38:03 2010
@@ -33,7 +33,7 @@
 import org.junit.Test;
 
 /**
- * Test the CollocReducer FIXME: add negative test cases.
+ * Test the CollocReducer
  */
 public class CollocReducerTest {
   
@@ -52,10 +52,11 @@
     // test input, input[*][0] is the key,
     // input[*][1..n] are the values passed in via
     // the iterator.
-    Gram[][] input = {{new Gram("the", UNIGRAM), new Gram("the", UNIGRAM), new Gram("the", UNIGRAM)},
-                                   {new Gram("the", HEAD), new Gram("the best", NGRAM), new Gram("the worst", NGRAM)},
-                                   {new Gram("of", HEAD), new Gram("of times", NGRAM), new Gram("of times", NGRAM)},
-                                   {new Gram("times", TAIL), new Gram("of times", NGRAM), new Gram("of times", NGRAM)}};
+    Gram[][] input = {
+        {new Gram("the", UNIGRAM), new Gram("the", UNIGRAM), new Gram("the", UNIGRAM)},
+        {new Gram("the", HEAD), new Gram("the best", NGRAM), new Gram("the worst", NGRAM)},
+        {new Gram("of", HEAD), new Gram("of times", NGRAM), new Gram("of times", NGRAM)},
+        {new Gram("times", TAIL), new Gram("of times", NGRAM), new Gram("of times", NGRAM)}};
     
     // expected results.
     Gram[][] values = {{new Gram("the", 2, UNIGRAM), new Gram("the", 2, UNIGRAM)},
@@ -64,6 +65,8 @@
                                     {new Gram("of times", 2, NGRAM), new Gram("of", 2, HEAD)},
                                     {new Gram("of times", 2, NGRAM), new Gram("times", 2, TAIL)}};
     
+    byte[] empty = new byte[0];
+    
     // set up expectations
     for (Gram[] v : values) {
       output.collect(v[0], v[1]);
@@ -73,10 +76,14 @@
     // play back the input data.
     CollocReducer c = new CollocReducer();
     
+    GramKey key = new GramKey();
+    
     for (Gram[] ii : input) {
+      key.set(ii[0], empty);
+
       List<Gram> vv = new LinkedList<Gram>();
-      vv.addAll(Arrays.asList(ii).subList(1, ii.length));
-      c.reduce(ii[0], vv.iterator(), output, reporter);
+      vv.addAll(Arrays.asList(ii));
+      c.reduce(key, vv.iterator(), output, reporter);
     }
     
     EasyMock.verify(reporter, output);

Added: lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyGroupComparatorTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyGroupComparatorTest.java?rev=919798&view=auto
==============================================================================
--- lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyGroupComparatorTest.java (added)
+++ lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyGroupComparatorTest.java Sat Mar  6 16:38:03 2010
@@ -0,0 +1,46 @@
+/**
+ * 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.mahout.utils.nlp.collocations.llr;
+
+import junit.framework.Assert;
+
+import org.junit.Test;
+
+
+public class GramKeyGroupComparatorTest {
+  @Test
+  public void testComparator() {
+    byte[] empty = new byte[0];
+    byte[] foo   = new byte[1];
+    foo[0] = (byte) 1;
+    
+    GramKey a = new GramKey(new Gram("foo", 1, Gram.Type.HEAD), empty); // base
+    GramKey b = new GramKey(new Gram("foo", 1, Gram.Type.HEAD), foo);   // vary byte
+    GramKey c = new GramKey(new Gram("foo", 2, Gram.Type.HEAD), empty); // vary freq
+    GramKey d = new GramKey(new Gram("foo", 1, Gram.Type.TAIL), empty); // vary type
+    GramKey e = new GramKey(new Gram("bar", 5, Gram.Type.HEAD), empty); // vary string
+    
+    GramKeyGroupComparator cmp = new GramKeyGroupComparator();
+    
+    Assert.assertTrue(0 == cmp.compare(a, b));
+    Assert.assertTrue(0 == cmp.compare(a, c));
+    Assert.assertTrue(0 > cmp.compare(a, d));
+    Assert.assertTrue(0 < cmp.compare(a, e));
+    Assert.assertTrue(0 < cmp.compare(d, e));
+  }
+}

Propchange: lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyGroupComparatorTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitionerTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitionerTest.java?rev=919798&view=auto
==============================================================================
--- lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitionerTest.java (added)
+++ lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitionerTest.java Sat Mar  6 16:38:03 2010
@@ -0,0 +1,54 @@
+/**
+ * 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.mahout.utils.nlp.collocations.llr;
+
+import junit.framework.Assert;
+
+import org.junit.Test;
+
+
+public class GramKeyPartitionerTest {
+  @Test
+  public void testPartition() {
+    byte[] empty = new byte[0];
+    byte[] foo = new byte[1];
+    foo[0] = 1;
+    
+    byte[] bar = new byte[1];
+    foo[0] = 2;
+    
+    GramKey a = new GramKey(new Gram("foo", 1, Gram.Type.HEAD), empty);
+    GramKey b = new GramKey(new Gram("foo", 1, Gram.Type.HEAD), foo);
+    GramKey c = new GramKey(new Gram("foo", 2, Gram.Type.HEAD), bar);
+    GramKey d = new GramKey(new Gram("foo", 1, Gram.Type.TAIL), empty);
+    GramKey e = new GramKey(new Gram("foo", 2, Gram.Type.TAIL), foo);
+    
+    GramKeyPartitioner p = new GramKeyPartitioner();
+    int numPartitions = 5;
+    
+    int ap = p.getPartition(a, null, numPartitions);
+    int bp = p.getPartition(b, null, numPartitions);
+    int cp = p.getPartition(c, null, numPartitions);
+    int dp = p.getPartition(d, null, numPartitions);
+    int ep = p.getPartition(e, null, numPartitions);
+    
+    Assert.assertEquals(ap, bp);
+    Assert.assertEquals(ap, cp);
+    Assert.assertEquals(dp, ep);
+  }
+}

Propchange: lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitionerTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyTest.java?rev=919798&view=auto
==============================================================================
--- lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyTest.java (added)
+++ lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyTest.java Sat Mar  6 16:38:03 2010
@@ -0,0 +1,101 @@
+/**
+ * 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.mahout.utils.nlp.collocations.llr;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+
+import junit.framework.Assert;
+import junit.framework.TestCase;
+
+import org.junit.Test;
+
+
+public class GramKeyTest {
+  @Test
+  public void testGramKeySort() {
+    byte[] empty = new byte[0];
+    byte[] foo = new byte[1]; foo[0] = 1;
+    byte[] bar = new byte[1]; bar[0] = 2;
+    
+    
+    // byte argument in GramKey breaks tie between equal grams
+    GramKey[] input =
+    {
+      new GramKey(new Gram("bar", 1, Gram.Type.UNIGRAM), empty),
+      new GramKey(new Gram("bar", 1, Gram.Type.UNIGRAM), empty),
+      new GramKey(new Gram("bar", 1, Gram.Type.UNIGRAM), foo),
+      new GramKey(new Gram("bar", 8, Gram.Type.NGRAM), foo),
+      new GramKey(new Gram("bar", 8, Gram.Type.NGRAM), empty),
+      new GramKey(new Gram("foo", 2, Gram.Type.HEAD), foo),
+      new GramKey(new Gram("foo", 3, Gram.Type.HEAD), empty),
+      new GramKey(new Gram("foo", 4, Gram.Type.TAIL), foo),
+      new GramKey(new Gram("foo", 5, Gram.Type.TAIL), foo),
+      new GramKey(new Gram("bar", 6, Gram.Type.HEAD), foo),
+      new GramKey(new Gram("bar", 7, Gram.Type.TAIL), empty),
+    };
+    
+    int[] expect = {
+        9, 6, 5, 10, 7, 8, 0, 1, 2, 4, 3
+    };
+    
+    GramKey[] sorted = new GramKey[input.length];
+    
+    System.arraycopy(input, 0, sorted, 0, input.length);
+    
+    Arrays.sort(sorted);
+
+    for (int i=0; i < input.length; i++) {
+      TestCase.assertSame(input[expect[i]], sorted[i]);
+    }
+  }
+  
+  @Test
+  public void testWritable() throws IOException {
+    byte[] foo = new byte[0];
+    byte[] bar = new byte[1]; bar[0] = 2;
+    
+    GramKey one = new GramKey(new Gram("foo", 2, Gram.Type.HEAD), foo);
+    GramKey two = new GramKey(new Gram("foobar", 3, Gram.Type.UNIGRAM), bar);
+
+    Assert.assertEquals("foo", one.getPrimaryString());
+    Assert.assertEquals("foobar", two.getPrimaryString());
+    
+    Assert.assertEquals(Gram.Type.UNIGRAM, two.getType());
+    
+    ByteArrayOutputStream bout = new ByteArrayOutputStream();
+    DataOutputStream out = new DataOutputStream(bout);
+    
+    two.write(out);
+    
+    byte[] b = bout.toByteArray();
+    
+    ByteArrayInputStream bin = new ByteArrayInputStream(b);
+    DataInputStream din = new DataInputStream(bin);
+    
+    one.readFields(din);
+
+    Assert.assertTrue(Arrays.equals(two.getBytes(), one.getBytes()));
+    Assert.assertEquals(Gram.Type.UNIGRAM, one.getType());
+    
+  }
+}

Propchange: lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramTest.java?rev=919798&r1=919797&r2=919798&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramTest.java (original)
+++ lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramTest.java Sat Mar  6 16:38:03 2010
@@ -22,6 +22,7 @@
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.HashMap;
 
 import junit.framework.Assert;
@@ -153,7 +154,6 @@
     };
     
     for (int i = 0; i < input.length; i++) {
-      System.err.println(i);
       Assert.assertEquals(freq[i], input[i].getFrequency());
       Assert.assertEquals(memb[i], input[i] == map.get(input[i]));
     }
@@ -189,4 +189,34 @@
    Assert.assertEquals(Gram.Type.UNIGRAM, one.getType());
    
  }
+ 
+ @Test
+ public void testSorting() throws IOException {
+   Gram[] input =
+   {
+    new Gram("foo", 2, Gram.Type.HEAD),
+    new Gram("foo", 3, Gram.Type.HEAD),
+    new Gram("foo", 4, Gram.Type.TAIL),
+    new Gram("foo", 5, Gram.Type.TAIL),
+    new Gram("bar", 6, Gram.Type.HEAD),
+    new Gram("bar", 7, Gram.Type.TAIL),
+    new Gram("bar", 8, Gram.Type.NGRAM),
+    new Gram("bar", Gram.Type.UNIGRAM)
+   };
+   
+   Gram[] sorted = new Gram[input.length];
+   
+   int[] expectations = {
+       4, 0, 1, 5, 2, 3, 7, 6
+   };
+   
+   
+   System.arraycopy(input, 0, sorted, 0, input.length);
+   
+   Arrays.sort(sorted);
+   
+   for (int i=0; i < sorted.length; i++) {
+     Assert.assertSame(input[expectations[i]], sorted[i]);
+   }
+ }
 }