You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mahout.apache.org by ro...@apache.org on 2010/01/17 00:10:31 UTC

svn commit: r900034 - in /lucene/mahout/trunk: examples/src/main/java/org/apache/mahout/text/ utils/src/main/java/org/apache/mahout/utils/vectors/text/ utils/src/test/java/org/apache/mahout/utils/vectors/text/

Author: robinanil
Date: Sat Jan 16 23:10:31 2010
New Revision: 900034

URL: http://svn.apache.org/viewvc?rev=900034&view=rev
Log:
MAHOUT-237 Dictionary Vectorizer: running version which was tested over Wikipedia article dumps

Added:
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/DocumentTokenizerMapper.java
Modified:
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/text/SparseVectorsFromSequenceFiles.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/text/WikipediaMapper.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/text/WikipediaToSequenceFile.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizer.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/PartialVectorGenerator.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/TermCountMapper.java
    lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizerTest.java

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/text/SparseVectorsFromSequenceFiles.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/text/SparseVectorsFromSequenceFiles.java?rev=900034&r1=900033&r2=900034&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/text/SparseVectorsFromSequenceFiles.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/text/SparseVectorsFromSequenceFiles.java Sat Jan 16 23:10:31 2010
@@ -25,6 +25,7 @@
 import org.apache.commons.cli2.builder.GroupBuilder;
 import org.apache.commons.cli2.commandline.Parser;
 import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.mahout.utils.vectors.text.DictionaryVectorizer;
 
 /**
@@ -58,7 +59,7 @@
             .create();
     
     Option analyzerNameOpt =
-        obuilder.withLongName("analyzerName").withRequired(true).withArgument(
+        obuilder.withLongName("analyzerName").withArgument(
             abuilder.withName("analyzerName").withMinimum(1).withMaximum(1)
                 .create()).withDescription("The class name of the analyzer")
             .withShortName("a").create();
@@ -93,9 +94,15 @@
       minSupport = Integer.parseInt(minSupportString);
     }
     String analyzerName = (String) cmdLine.getValue(analyzerNameOpt);
-    Analyzer analyzer = (Analyzer) Class.forName(analyzerName).newInstance();
-    
+    Class<? extends Analyzer> analyzerClass = StandardAnalyzer.class;
+    if (cmdLine.hasOption(analyzerNameOpt)) {
+      String className = cmdLine.getValue(analyzerNameOpt).toString();
+      analyzerClass = (Class<? extends Analyzer>) Class.forName(className);
+      // try instantiating it, b/c there isn't any point in setting it if
+      // you can't instantiate it
+      analyzerClass.newInstance();
+    }
     DictionaryVectorizer.createTermFrequencyVectors(inputDir, outputDir,
-        analyzer, minSupport, chunkSize);
+        analyzerClass, minSupport, chunkSize);
   }
 }

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/text/WikipediaMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/text/WikipediaMapper.java?rev=900034&r1=900033&r2=900034&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/text/WikipediaMapper.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/text/WikipediaMapper.java Sat Jan 16 23:10:31 2010
@@ -48,16 +48,15 @@
   private static final Logger log =
       LoggerFactory.getLogger(WikipediaMapper.class);
   private static final Pattern SPACE_NON_ALPHA_PATTERN =
-      Pattern.compile("[\\s\\W]");
-  
-  private static final Pattern DOCUMENT =
-      Pattern.compile("<text xml:space=\"preserve\">(.*)</text>");
-  private static final Pattern TITLE = Pattern.compile("<title>(.*)</title>");
+      Pattern.compile("[\\s]");
+  private static final String START_DOC = "<text xml:space=\"preserve\">";
+  private static final String END_DOC = "</text>";
+  private static final Pattern TITLE = Pattern.compile("<title>(.*)<\\/title>");
   
+  private static final String REDIRECT = "<redirect />";
   private Set<String> inputCategories;
   private boolean exactMatchOnly;
   private boolean all;
-  private Analyzer analyzer;
   
   @Override
   public void map(LongWritable key,
@@ -66,8 +65,16 @@
                   Reporter reporter) throws IOException {
     
     String content = value.toString();
-    String document = getDocument(content);
-    String title = getTitle(content);
+    if (content.indexOf(REDIRECT) != -1) return;
+    String document = "";
+    String title = "";
+    try {
+      document = getDocument(content);
+      title = getTitle(content);
+    } catch (Exception e) {
+      reporter.getCounter("Wikipedia", "Parse errors").increment(1);
+      return;
+    }
     
     if (!all) {
       String catMatch = findMatchingCategory(document);
@@ -81,13 +88,9 @@
   }
   
   private String getDocument(String xml) {
-    Matcher m = DOCUMENT.matcher(xml);
-    String ret = "";
-    if (m.find()) {
-      ret = m.group(1);
-    }
-    return ret;
-    
+    int start = xml.indexOf(START_DOC) + START_DOC.length();
+    int end = xml.indexOf(END_DOC, start);
+    return xml.substring(start, end);
   }
   
   private String getTitle(String xml) {
@@ -150,8 +153,6 @@
         + " All: "
         + all
         + " Exact Match: "
-        + exactMatchOnly
-        + " Analyzer: "
-        + analyzer.getClass().getName());
+        + exactMatchOnly);
   }
 }

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/text/WikipediaToSequenceFile.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/text/WikipediaToSequenceFile.java?rev=900034&r1=900033&r2=900034&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/text/WikipediaToSequenceFile.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/text/WikipediaToSequenceFile.java Sat Jan 16 23:10:31 2010
@@ -104,7 +104,7 @@
     
     Option allOpt =
         obuilder.withLongName("all").withDescription(
-            "If set, Select all files. Default is true").withShortName("all")
+            "If set, Select all files. Default is false").withShortName("all")
             .create();
     
     Option helpOpt =
@@ -133,9 +133,9 @@
         catFile = (String) cmdLine.getValue(categoriesOpt);
       }
       
-      boolean all = true;
+      boolean all = false;
       if (cmdLine.hasOption(allOpt)) {
-        cmdLine.getValue(allOpt);
+        all = true;
       }
       runJob(inputPath, outputPath, catFile, cmdLine.hasOption(exactMatchOpt),
           all);
@@ -177,7 +177,7 @@
           + " All Files: "
           + all);
     }
-    conf.set("xmlinput.start", "<page");
+    conf.set("xmlinput.start", "<page>");
     conf.set("xmlinput.end", "</page>");
     conf.setOutputKeyClass(Text.class);
     conf.setOutputValueClass(Text.class);
@@ -193,7 +193,13 @@
     conf.set("io.serializations",
         "org.apache.hadoop.io.serializer.JavaSerialization,"
             + "org.apache.hadoop.io.serializer.WritableSerialization");
-    
+
+    /*conf.set("mapred.compress.map.output", "true");
+    conf.set("mapred.map.output.compression.type", "BLOCK");
+    conf.set("mapred.output.compress", "true");
+    conf.set("mapred.output.compression.type", "BLOCK");
+    conf.set("mapred.output.compression.codec", "org.apache.hadoop.io.compress.GzipCodec");
+    */
     FileSystem dfs = FileSystem.get(outPath.toUri(), conf);
     if (dfs.exists(outPath)) {
       dfs.delete(outPath, true);

Modified: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizer.java?rev=900034&r1=900033&r2=900034&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizer.java (original)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizer.java Sat Jan 16 23:10:31 2010
@@ -42,7 +42,6 @@
 import org.apache.hadoop.mapred.SequenceFileOutputFormat;
 import org.apache.hadoop.mapred.lib.IdentityMapper;
 import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.mahout.math.VectorWritable;
 
 /**
@@ -73,7 +72,7 @@
   
   private static final String OUTPUT_FILES_PATTERN = "/part-*";
   
-  private static final int SEQUENCEFILE_BYTE_OVERHEAD = 4;
+  private static final int SEQUENCEFILE_BYTE_OVERHEAD = 45;
   
   private static final String VECTOR_OUTPUT_FOLDER = "/partial-vectors-";
   
@@ -117,7 +116,7 @@
    */
   public static void createTermFrequencyVectors(String input,
                                                 String output,
-                                                Analyzer analyzer,
+                                                Class<? extends Analyzer> analyzerClass,
                                                 int minSupport,
                                                 int chunkSizeInMegabytes) throws IOException,
                                                                          InterruptedException,
@@ -132,7 +131,7 @@
     Path inputPath = new Path(input);
     Path wordCountPath = new Path(output + WORDCOUNT_OUTPUT_FOLDER);
     
-    startWordCounting(inputPath, wordCountPath, analyzer);
+    startWordCounting(inputPath, analyzerClass, wordCountPath);
     List<Path> dictionaryChunks =
         createDictionaryChunks(minSupport, wordCountPath, output,
             chunkSizeInMegabytes);
@@ -143,8 +142,8 @@
       Path partialVectorOutputPath =
           getPath(output + VECTOR_OUTPUT_FOLDER, partialVectorIndex++);
       partialVectorPaths.add(partialVectorOutputPath);
-      makePartialVectors(input, dictionaryChunk, partialVectorOutputPath,
-          analyzer);
+      makePartialVectors(input, dictionaryChunk, analyzerClass,
+          partialVectorOutputPath);
     }
     
     createVectorFromPartialVectors(partialVectorPaths, output
@@ -221,9 +220,8 @@
         
         int fieldSize =
             SEQUENCEFILE_BYTE_OVERHEAD
-                + key.toString().getBytes(CHARSET).length
-                + Long.SIZE
-                / 8;
+                + (key.toString().length() * 2)
+                + (Long.SIZE / 8);
         currentChunkSize += fieldSize;
         writer.append(key, new LongWritable(i++));
         freqWriter.append(key, value);
@@ -261,8 +259,6 @@
         "org.apache.hadoop.io.serializer.JavaSerialization,"
             + "org.apache.hadoop.io.serializer.WritableSerialization");
     // this conf parameter needs to be set enable serialisation of conf values
-    
-    conf.set(ANALYZER_CLASS, StandardAnalyzer.class.getName());
     conf
         .setJobName("DictionaryVectorizer Vector generator to group Partial Vectors");
     
@@ -324,11 +320,11 @@
    */
   private static void makePartialVectors(String input,
                                          Path dictionaryFilePath,
-                                         Path output,
-                                         Analyzer analyzer) throws IOException,
-                                                           InterruptedException,
-                                                           ClassNotFoundException,
-                                                           URISyntaxException {
+                                         Class<? extends Analyzer> analyzerClass,
+                                         Path output) throws IOException,
+                                                     InterruptedException,
+                                                     ClassNotFoundException,
+                                                     URISyntaxException {
     
     Configurable client = new JobClient();
     JobConf conf = new JobConf(DictionaryVectorizer.class);
@@ -337,7 +333,7 @@
             + "org.apache.hadoop.io.serializer.WritableSerialization");
     // this conf parameter needs to be set enable serialisation of conf values
     
-    conf.set(ANALYZER_CLASS, StandardAnalyzer.class.getName());
+    conf.set(ANALYZER_CLASS, analyzerClass.getName());
     conf.setJobName("DictionaryVectorizer Partial Vector running over input: "
         + input
         + " using dictionary file"
@@ -353,11 +349,10 @@
     
     FileOutputFormat.setOutputPath(conf, output);
     
-    conf.setMapperClass(IdentityMapper.class);
+    conf.setMapperClass(DocumentTokenizerMapper.class);
     conf.setInputFormat(SequenceFileInputFormat.class);
     conf.setReducerClass(PartialVectorGenerator.class);
     conf.setOutputFormat(SequenceFileOutputFormat.class);
-    
     FileSystem dfs = FileSystem.get(output.toUri(), conf);
     if (dfs.exists(output)) {
       dfs.delete(output, true);
@@ -379,10 +374,10 @@
    * @throws ClassNotFoundException
    */
   private static void startWordCounting(Path input,
-                                        Path output,
-                                        Analyzer analyzer) throws IOException,
-                                                          InterruptedException,
-                                                          ClassNotFoundException {
+                                        Class<? extends Analyzer> analyzerClass,
+                                        Path output) throws IOException,
+                                                    InterruptedException,
+                                                    ClassNotFoundException {
     
     Configurable client = new JobClient();
     JobConf conf = new JobConf(DictionaryVectorizer.class);
@@ -391,7 +386,7 @@
             + "org.apache.hadoop.io.serializer.WritableSerialization");
     // this conf parameter needs to be set enable serialisation of conf values
     
-    conf.set(ANALYZER_CLASS, StandardAnalyzer.class.getName());
+    conf.set(ANALYZER_CLASS, analyzerClass.getName());
     conf.setJobName("DictionaryVectorizer Word Count running over input: "
         + input.toString());
     conf.setOutputKeyClass(Text.class);

Added: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/DocumentTokenizerMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/DocumentTokenizerMapper.java?rev=900034&view=auto
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/DocumentTokenizerMapper.java (added)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/DocumentTokenizerMapper.java Sat Jan 16 23:10:31 2010
@@ -0,0 +1,91 @@
+/**
+ * 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.vectors.text;
+
+import java.io.IOException;
+import java.io.StringReader;
+import java.util.HashMap;
+import java.util.Map;
+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.JobConf;
+import org.apache.hadoop.mapred.MapReduceBase;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.analysis.tokenattributes.TermAttribute;
+import org.apache.mahout.common.parameters.ClassParameter;
+
+/**
+ * TextVectorizer Term Count Mapper. Tokenizes a text document and outputs
+ * useful tokens space separated
+ */
+public class DocumentTokenizerMapper extends MapReduceBase implements
+    Mapper<Text,Text,Text,Text> {
+  
+  private Analyzer analyzer;
+  private StringBuilder document = new StringBuilder();
+  @Override
+  public void map(Text key,
+                  Text value,
+                  OutputCollector<Text,Text> output,
+                  Reporter reporter) throws IOException {
+    TokenStream stream =
+        analyzer
+            .tokenStream(key.toString(), new StringReader(value.toString()));
+    TermAttribute termAtt =
+        (TermAttribute) stream.addAttribute(TermAttribute.class);
+    document.setLength(0);
+    String sep = "";
+    while (stream.incrementToken()) {
+      String word = new String(termAtt.termBuffer(), 0, termAtt.termLength());
+      if (word != "") {
+        document.append(sep).append(word);
+        sep = " ";
+      }
+    }
+    output.collect(key, new Text(document.toString()) );
+    
+  }
+  
+  @Override
+  public void configure(JobConf job) {
+    super.configure(job);
+    try {
+      ClassLoader ccl = Thread.currentThread().getContextClassLoader();
+      Class<?> cl =
+          ccl.loadClass(job.get(DictionaryVectorizer.ANALYZER_CLASS,
+              StandardAnalyzer.class.getName()));
+      analyzer = (Analyzer) cl.newInstance();
+    } catch (ClassNotFoundException e) {
+      throw new IllegalStateException(e);
+    } catch (InstantiationException e) {
+      throw new IllegalStateException(e);
+    } catch (IllegalAccessException e) {
+      throw new IllegalStateException(e);
+    }
+  }
+  
+}

Modified: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/PartialVectorGenerator.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/PartialVectorGenerator.java?rev=900034&r1=900033&r2=900034&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/PartialVectorGenerator.java (original)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/PartialVectorGenerator.java Sat Jan 16 23:10:31 2010
@@ -18,14 +18,12 @@
 package org.apache.mahout.utils.vectors.text;
 
 import java.io.IOException;
-import java.io.StringReader;
 import java.net.URI;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
-import java.util.Map.Entry;
+import java.util.StringTokenizer;
 
-import org.apache.commons.lang.mutable.MutableInt;
 import org.apache.hadoop.filecache.DistributedCache;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -38,9 +36,6 @@
 import org.apache.hadoop.mapred.Reducer;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.Token;
-import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.mahout.math.RandomAccessSparseVector;
 import org.apache.mahout.math.VectorWritable;
 
@@ -48,64 +43,44 @@
  * Converts a document in to a sparse vector
  */
 public class PartialVectorGenerator extends MapReduceBase implements
-    Reducer<Text,Text,Text, VectorWritable> {
+    Reducer<Text,Text,Text,VectorWritable> {
   private Analyzer analyzer;
-  private Map<String,Integer> dictionary = new HashMap<String,Integer>();
+  private Map<String,int[]> dictionary = new HashMap<String,int[]>();
   private FileSystem fs; // local filesystem
   private URI[] localFiles; // local filenames from the distributed cache
-
+  
   private VectorWritable vectorWritable = new VectorWritable();
-
-  @Override
+  
   public void reduce(Text key,
                      Iterator<Text> values,
                      OutputCollector<Text,VectorWritable> output,
                      Reporter reporter) throws IOException {
+    if (values.hasNext() == false) return;
+    Text value = values.next();
+    String valueString = value.toString();
+    StringTokenizer stream = new StringTokenizer(valueString, " ");
     
-    if (values.hasNext()) {
-      Text value = values.next();
-      TokenStream ts =
-          analyzer.tokenStream(key.toString(), new StringReader(value
-              .toString()));
-      
-      Map<String,MutableInt> termFrequency = new HashMap<String,MutableInt>();
-      
-      Token token = new Token();
-      int count = 0;
-      while ((token = ts.next(token)) != null) {
-        String tk = new String(token.termBuffer(), 0, token.termLength());
-        if(dictionary.containsKey(tk) == false) continue;
-        if (termFrequency.containsKey(tk) == false) {
-          count += tk.length() + 1;
-          termFrequency.put(tk, new MutableInt(0));
-        }
-        termFrequency.get(tk).increment();
-      }
-      
-      RandomAccessSparseVector vector =
-          new RandomAccessSparseVector(key.toString(), Integer.MAX_VALUE, termFrequency
-              .size());
+    RandomAccessSparseVector vector =
+        new RandomAccessSparseVector(key.toString(), Integer.MAX_VALUE,
+            valueString.length() / 5); // guess at initial size
+    
+    while (stream.hasMoreTokens()) {
+      String tk = stream.nextToken();
+      if (dictionary.containsKey(tk) == false) continue;
+      int tokenKey = dictionary.get(tk)[0];
+      vector.setQuick(tokenKey, vector.getQuick(tokenKey) + 1);
       
-      for (Entry<String,MutableInt> pair : termFrequency.entrySet()) {
-        String tk = pair.getKey();
-        if (dictionary.containsKey(tk) == false) continue;
-        vector.setQuick(dictionary.get(tk).intValue(), pair.getValue()
-            .doubleValue());
-      }
-      vectorWritable.set(vector);
-      output.collect(key, vectorWritable);
     }
+    
+    vectorWritable.set(vector);
+    output.collect(key, vectorWritable);
+    
   }
   
   @Override
   public void configure(JobConf job) {
     super.configure(job);
     try {
-      ClassLoader ccl = Thread.currentThread().getContextClassLoader();
-      Class<?> cl =
-          ccl.loadClass(job.get(DictionaryVectorizer.ANALYZER_CLASS,
-              StandardAnalyzer.class.getName()));
-      analyzer = (Analyzer) cl.newInstance();
       
       localFiles = DistributedCache.getCacheFiles(job);
       if (localFiles == null || localFiles.length < 1) {
@@ -121,15 +96,9 @@
       
       // key is word value is id
       while (reader.next(key, value)) {
-        dictionary.put(key.toString(), Long.valueOf(value.get()).intValue());
-        // System.out.println(key.toString() + "=>" + value.get());
+        dictionary.put(key.toString(), new int[] {Long.valueOf(value.get())
+            .intValue()});
       }
-    } catch (ClassNotFoundException e) {
-      throw new IllegalStateException(e);
-    } catch (InstantiationException e) {
-      throw new IllegalStateException(e);
-    } catch (IllegalAccessException e) {
-      throw new IllegalStateException(e);
     } catch (IOException e) {
       throw new IllegalStateException(e);
     }

Modified: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/TermCountMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/TermCountMapper.java?rev=900034&r1=900033&r2=900034&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/TermCountMapper.java (original)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/TermCountMapper.java Sat Jan 16 23:10:31 2010
@@ -35,6 +35,8 @@
 import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.analysis.tokenattributes.TermAttribute;
+import org.apache.mahout.common.parameters.ClassParameter;
 
 /**
  * TextVectorizer Term Count Mapper. Tokenizes a text document and outputs the
@@ -51,20 +53,20 @@
                   Text value,
                   OutputCollector<Text,LongWritable> output,
                   Reporter reporter) throws IOException {
-    TokenStream ts =
+    TokenStream stream =
         analyzer
             .tokenStream(key.toString(), new StringReader(value.toString()));
-    
-    Token token = new Token();
-    Map<String,MutableLong> wordCount = new HashMap<String,MutableLong>();
-    while ((token = ts.next(token)) != null) {
-      char[] termBuffer = token.termBuffer();
-      String word = new String(termBuffer, 0, token.termLength());
+    Map<String,MutableLong> wordCount = new HashMap<String,MutableLong>();    
+    TermAttribute termAtt =
+        (TermAttribute) stream.addAttribute(TermAttribute.class);
+    while (stream.incrementToken()) {
+      String word = new String(termAtt.termBuffer(), 0, termAtt.termLength());
       if (wordCount.containsKey(word) == false) {
         wordCount.put(word, new MutableLong(0));
       }
       wordCount.get(word).increment();
     }
+    
     for (Entry<String,MutableLong> entry : wordCount.entrySet()) {
       output.collect(new Text(entry.getKey()), new LongWritable(entry
           .getValue().longValue()));

Modified: lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizerTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizerTest.java?rev=900034&r1=900033&r2=900034&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizerTest.java (original)
+++ lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizerTest.java Sat Jan 16 23:10:31 2010
@@ -29,7 +29,9 @@
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.util.Version;
 
 /**
  * Test the dictionary Vector
@@ -49,8 +51,8 @@
   
   public static final String DELIM = " .,?;:!\t\n\r";
   
-  public static final String ERRORSET = "`1234567890"
-                                        + "-=~@#$%^&*()_+[]{}'\"/<>|\\";
+  public static final String ERRORSET =
+      "`1234567890" + "-=~@#$%^&*()_+[]{}'\"/<>|\\";
   
   private static Random random = new Random();
   
@@ -60,20 +62,20 @@
     return DELIM.charAt(random.nextInt(DELIM.length()));
   }
   
-  private static String getRandomDocument() {
-    int length = (AVG_DOCUMENT_LENGTH >> 1)
-                 + random.nextInt(AVG_DOCUMENT_LENGTH);
-    StringBuilder sb = new StringBuilder(length * AVG_SENTENCE_LENGTH
-                                         * AVG_WORD_LENGTH);
+  public static String getRandomDocument() {
+    int length =
+        (AVG_DOCUMENT_LENGTH >> 1) + random.nextInt(AVG_DOCUMENT_LENGTH);
+    StringBuilder sb =
+        new StringBuilder(length * AVG_SENTENCE_LENGTH * AVG_WORD_LENGTH);
     for (int i = 0; i < length; i++) {
       sb.append(getRandomSentence());
     }
     return sb.toString();
   }
   
-  private static String getRandomSentence() {
-    int length = (AVG_SENTENCE_LENGTH >> 1)
-                 + random.nextInt(AVG_SENTENCE_LENGTH);
+  public static String getRandomSentence() {
+    int length =
+        (AVG_SENTENCE_LENGTH >> 1) + random.nextInt(AVG_SENTENCE_LENGTH);
     StringBuilder sb = new StringBuilder(length * AVG_WORD_LENGTH);
     for (int i = 0; i < length; i++) {
       sb.append(getRandomString()).append(' ');
@@ -82,7 +84,7 @@
     return sb.toString();
   }
   
-  private static String getRandomString() {
+  public static String getRandomString() {
     int length = (AVG_WORD_LENGTH >> 1) + random.nextInt(AVG_WORD_LENGTH);
     StringBuilder sb = new StringBuilder(length);
     for (int i = 0; i < length; i++) {
@@ -120,17 +122,18 @@
     Configuration conf = new Configuration();
     String pathString = "testdata/documents/docs.file";
     Path path = new Path(pathString);
-    SequenceFile.Writer writer = new SequenceFile.Writer(fs, conf, path,
-        Text.class, Text.class);
+    SequenceFile.Writer writer =
+        new SequenceFile.Writer(fs, conf, path, Text.class, Text.class);
     
     for (int i = 0; i < NUM_DOCS; i++) {
       writer.append(new Text("Document::ID::" + i), new Text(
           getRandomDocument()));
     }
     writer.close();
+    Class<? extends Analyzer> analyzer =
+        new StandardAnalyzer(Version.LUCENE_CURRENT).getClass();
     DictionaryVectorizer.createTermFrequencyVectors(pathString,
-      "output/wordcount", new StandardAnalyzer(), 2, 100);
-    
+        "output/wordcount", analyzer, 2, 100);
     
   }
 }