mahout-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From robina...@apache.org
Subject svn commit: r908040 - in /lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils: nlp/collocations/llr/ vectors/text/term/
Date Tue, 09 Feb 2010 14:11:51 GMT
Author: robinanil
Date: Tue Feb  9 14:11:50 2010
New Revision: 908040

URL: http://svn.apache.org/viewvc?rev=908040&view=rev
Log:
Adding minSupport and minLLRValue parameter for pruning low frequency ngrams in Collocations
Map/Reduce Job

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/CollocReducer.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/Gram.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/LLRReducer.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/term/TermCountMapper.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=908040&r1=908039&r2=908040&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
Tue Feb  9 14:11:50 2010
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+
 package org.apache.mahout.utils.nlp.collocations.llr;
 
 import java.io.IOException;
@@ -27,55 +28,53 @@
 import org.apache.hadoop.mapred.Reporter;
 
 public class CollocCombiner extends MapReduceBase implements
-    Reducer<Gram,Gram,Gram,Gram> {
-  
-  /**
-   * collocation finder: pass 1 collec 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 key,
-                     Iterator<Gram> value,
-                     OutputCollector<Gram,Gram> output,
-                     Reporter reporter) throws IOException {
-    
-    HashMap<Gram,Gram> set = new HashMap<Gram,Gram>();
-    int subgramFrequency = 0;
-    
-    while (value.hasNext()) {
-      Gram t = value.next();
-      subgramFrequency += t.getFrequency();
+    Reducer<Gram, Gram, Gram, Gram> {
+
+    /** collocation finder: pass 1 collec 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 key, Iterator<Gram> value,
+        OutputCollector<Gram, Gram> output, Reporter reporter) throws IOException {
+
+      HashMap<Gram,Gram> set = new HashMap<Gram,Gram>();
+      int subgramFrequency = 0;
+
+      while (value.hasNext()) {
+        Gram t = value.next();
+        subgramFrequency += t.getFrequency();
+        
+        Gram s = set.get(t);
+        if (s == null) {
+          // t is potentially reused, so create a new object to populate the HashMap
+          Gram e = new Gram(t);
+          set.put(e,e);
+        }
+        else {
+          s.incrementFrequency(t.getFrequency());
+        }
+      }
+
+      // emit subgram:subgramFreq ngram:ngramFreq pairs
+      key.setFrequency(subgramFrequency);
       
-      Gram s = set.get(t);
-      if (s == null) {
-        // t is potentially reused, so create a new object to populate the
-        // HashMap
-        Gram e = new Gram(t);
-        set.put(e, e);
-      } else {
-        s.incrementFrequency(t.getFrequency());
+      for (Gram t: set.keySet()) {
+        output.collect(key, t);
       }
     }
-    
-    // emit subgram:subgramFreq ngram:ngramFreq pairs
-    key.setFrequency(subgramFrequency);
-    
-    for (Gram t : set.keySet()) {
-      output.collect(key, t);
-    }
-  }
-  
+
 }

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=908040&r1=908039&r2=908040&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
Tue Feb  9 14:11:50 2010
@@ -74,23 +74,38 @@
         .withDescription("The Path write output to").withShortName("o")
         .create();
     
-    Option maxNGramSizeOpt = obuilder
-        .withLongName("maxNGramSize")
-        .withRequired(false)
-        .withArgument(
+    Option maxNGramSizeOpt = obuilder.withLongName("maxNGramSize")
+        .withRequired(false).withArgument(
           abuilder.withName("size").withMinimum(1).withMaximum(1).create())
         .withDescription(
-          "The maximum size of ngrams to create (2 = bigrams, 3 = trigrams, etc)")
+          "(Optional) The maximum size of ngrams to create"
+              + " (2 = bigrams, 3 = trigrams, etc) Default Value:2")
         .withShortName("n").create();
     
+    Option minSupportOpt = obuilder.withLongName("minSupport").withArgument(
+      abuilder.withName("minSupport").withMinimum(1).withMaximum(1).create())
+        .withDescription("(Optional) Minimum Support. Default Value: 2")
+        .withShortName("s").create();
+    
+    Option minLLROpt = obuilder
+        .withLongName("minLLR")
+        .withRequired(false)
+        .withArgument(
+          abuilder.withName("minDF").withMinimum(1).withMaximum(1).create())
+        .withDescription(
+          "(Optional)The minimum Log Likelihood Ratio(Float)  Default is 0.00")
+        .withShortName("ml").create();
+    
     Option overwriteOutput = obuilder.withLongName("overwrite").withRequired(
       false).withDescription("If set, overwrite the output directory")
         .withShortName("w").create();
     
     Option analyzerNameOpt = obuilder.withLongName("analyzerName")
-        .withRequired(false).withArgument(
+        .withRequired(false)
+        .withArgument(
           abuilder.withName("analyzerName").withMinimum(1).withMaximum(1)
-              .create()).withDescription(
+              .create())
+        .withDescription(
           "Class name of analyzer to use for tokenization").withShortName("a")
         .create();
     
@@ -99,7 +114,8 @@
     
     Group group = gbuilder.withName("Options").withOption(inputOpt).withOption(
       outputOpt).withOption(maxNGramSizeOpt).withOption(overwriteOutput)
-        .withOption(analyzerNameOpt).withOption(helpOpt).create();
+        .withOption(analyzerNameOpt).withOption(minSupportOpt).withOption(
+          minLLROpt).withOption(helpOpt).create();
     
     try {
       Parser parser = new Parser();
@@ -134,11 +150,24 @@
         analyzerName = cmdLine.getValue(analyzerNameOpt).toString();
       }
       
+      int minSupport = 2;
+      if (cmdLine.hasOption(minSupportOpt)) {
+        minSupport = Integer.parseInt(cmdLine.getValue(minSupportOpt)
+            .toString());
+      }
+      
+      float minLLRValue = 1.0f;
+      if (cmdLine.hasOption(minLLROpt)) {
+        minLLRValue = Float
+            .parseFloat(cmdLine.getValue(minLLROpt).toString());
+      }
+      
       // parse input and extract collocations
-      long ngramCount = runPass1(input, output, maxNGramSize, analyzerName);
+      long ngramCount = runPass1(input, output, maxNGramSize, analyzerName,
+        minSupport);
       
       // tally collocations and perform LLR calculation
-      runPass2(ngramCount, output);
+      runPass2(ngramCount, output, minLLRValue);
       
     } catch (OptionException e) {
       log.error("Exception", e);
@@ -147,11 +176,14 @@
     
   }
   
-  /** pass1: generate collocations, ngrams */
+  /**
+   * pass1: generate collocations, ngrams
+   */
   public static long runPass1(String input,
                               String output,
                               int maxNGramSize,
-                              String analyzerClass) throws IOException {
+                              String analyzerClass,
+                              int minSupport) throws IOException {
     JobConf conf = new JobConf(CollocDriver.class);
     
     conf.setMapOutputKeyClass(Gram.class);
@@ -171,7 +203,8 @@
     
     conf.setOutputFormat(SequenceFileOutputFormat.class);
     conf.setReducerClass(CollocReducer.class);
-    conf.set(NGramCollector.MAX_SHINGLE_SIZE, String.valueOf(maxNGramSize));
+    conf.setInt(NGramCollector.MAX_SHINGLE_SIZE, maxNGramSize);
+    conf.setInt(CollocReducer.MIN_SUPPORT, minSupport);
     
     if (analyzerClass != null) {
       conf.set(NGramCollector.ANALYZER_CLASS, analyzerClass);
@@ -181,8 +214,12 @@
     return job.getCounters().findCounter(NGRAM_TOTAL).getValue();
   }
   
-  /** pass2: perform the LLR calculation */
-  public static void runPass2(long nGramTotal, String output) throws IOException {
+  /**
+   * pass2: perform the LLR calculation
+   */
+  public static void runPass2(long nGramTotal,
+                              String output,
+                              float minLLRValue) throws IOException {
     JobConf conf = new JobConf(CollocDriver.class);
     conf.set(LLRReducer.NGRAM_TOTAL, String.valueOf(nGramTotal));
     
@@ -200,6 +237,8 @@
     conf.setInputFormat(SequenceFileInputFormat.class);
     conf.setOutputFormat(TextOutputFormat.class);
     conf.setReducerClass(LLRReducer.class);
+    
+    conf.setFloat(LLRReducer.MIN_LLR, minLLRValue);
     JobClient.runJob(conf);
   }
 }

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=908040&r1=908039&r2=908040&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
Tue Feb  9 14:11:50 2010
@@ -21,6 +21,7 @@
 import java.util.HashMap;
 import java.util.Iterator;
 
+import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MapReduceBase;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reducer;
@@ -32,6 +33,12 @@
  */
 public class CollocReducer extends MapReduceBase implements
     Reducer<Gram,Gram,Gram,Gram> {
+  public static enum Skipped {
+    LESS_THAN_MIN_SUPPORT;
+  };
+  
+  public static final String MIN_SUPPORT = "minSupport";
+  private int minSupport;
   
   /**
    * collocation finder: pass 1 reduce phase:
@@ -77,7 +84,17 @@
     key.setFrequency(subgramFrequency);
     
     for (Gram t : set.keySet()) {
+      if (t.getFrequency() < minSupport) {
+        reporter.incrCounter(Skipped.LESS_THAN_MIN_SUPPORT, 1);
+        continue;
+      }
       output.collect(t, key);
     }
   }
+  
+  @Override
+  public void configure(JobConf job) {
+    super.configure(job);
+    this.minSupport = job.getInt(MIN_SUPPORT, 2);
+  }
 }

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=908040&r1=908039&r2=908040&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
Tue Feb  9 14:11:50 2010
@@ -23,7 +23,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableComparable;
 
 /**
@@ -160,11 +160,9 @@
     
     if (head) position = Position.HEAD;
     else position = Position.TAIL;
-    
-    int fieldLen = in.readInt();
-    byte[] entry = new byte[fieldLen];
-    in.readFully(entry);
-    gram = Bytes.toString(entry);
+    Text data = new Text();
+    data.readFields(in);
+    gram = data.toString();
   }
   
   @Override
@@ -174,10 +172,8 @@
     if (position == Position.HEAD) out.writeBoolean(true);
     else out.writeBoolean(false);
     
-    byte[] data = Bytes.toBytes(gram);
-    out.writeInt(data.length);
-    out.write(data);
-    
+    Text data = new Text(gram);
+    data.write(out);
   }
   
   @Override

Modified: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/LLRReducer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/LLRReducer.java?rev=908040&r1=908039&r2=908040&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/LLRReducer.java
(original)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/LLRReducer.java
Tue Feb  9 14:11:50 2010
@@ -43,14 +43,17 @@
     EXTRA_HEAD,
     EXTRA_TAIL,
     MISSING_HEAD,
-    MISSING_TAIL;
+    MISSING_TAIL,
+    LESS_THAN_MIN_LLR;
   };
   
   private static final Logger log = LoggerFactory.getLogger(LLRReducer.class);
   
   public static final String NGRAM_TOTAL = "ngramTotal";
+  public static final String MIN_LLR = "minLLR";
   
-  long ngramTotal;
+  private long ngramTotal;
+  private float minLLRValue;
   private final LLCallback ll;
   
   public LLRReducer() {
@@ -72,6 +75,7 @@
   public void configure(JobConf job) {
     super.configure(job);
     this.ngramTotal = job.getLong(NGRAM_TOTAL, -1);
+    this.minLLRValue = job.getFloat(MIN_LLR, 0.0f);
     
     log.info("NGram Total is " + ngramTotal);
     
@@ -135,14 +139,18 @@
       return;
     }
     
-    int k11 = ngram.getFrequency(); /* a+b */
-    int k12 = gramFreq[0] - ngram.getFrequency(); /* a+!b */
-    int k21 = gramFreq[1] - ngram.getFrequency(); /* !b+a */
+    int k11 = ngram.getFrequency(); /* a&b */
+    int k12 = gramFreq[0] - ngram.getFrequency(); /* a&!b */
+    int k21 = gramFreq[1] - ngram.getFrequency(); /* !b&a */
     int k22 = (int) (ngramTotal - (gramFreq[0] + gramFreq[1] - ngram
-        .getFrequency())); /* !a+!b */
+        .getFrequency())); /* !a&!b */
     
     try {
       double llr = ll.logLikelihoodRatio(k11, k12, k21, k22);
+      if(llr < minLLRValue){
+        reporter.incrCounter(Skipped.LESS_THAN_MIN_LLR, 1);
+        return;
+      }
       DoubleWritable dd = new DoubleWritable(llr);
       Text t = new Text(ngram.getString());
       output.collect(dd, t);

Modified: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/term/TermCountMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/term/TermCountMapper.java?rev=908040&r1=908039&r2=908040&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/term/TermCountMapper.java
(original)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/term/TermCountMapper.java
Tue Feb  9 14:11:50 2010
@@ -18,9 +18,7 @@
 package org.apache.mahout.utils.vectors.text.term;
 
 import java.io.IOException;
-import java.util.Map.Entry;
 
-import org.apache.commons.lang.mutable.MutableLong;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.MapReduceBase;



Mime
View raw message