You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mahout.apache.org by ak...@apache.org on 2015/03/30 18:34:02 UTC

[1/3] mahout git commit: extended TFPartialVectorReducer.java to handle multiple text blocks of one document

Repository: mahout
Updated Branches:
  refs/heads/master 4b1c13332 -> 91c1626df


extended TFPartialVectorReducer.java to handle multiple text blocks of one document


Project: http://git-wip-us.apache.org/repos/asf/mahout/repo
Commit: http://git-wip-us.apache.org/repos/asf/mahout/commit/ccaec1b2
Tree: http://git-wip-us.apache.org/repos/asf/mahout/tree/ccaec1b2
Diff: http://git-wip-us.apache.org/repos/asf/mahout/diff/ccaec1b2

Branch: refs/heads/master
Commit: ccaec1b267d768e890695dd36c49a34d621e1e73
Parents: c9d978a
Author: wobu <bu...@googlemail.com>
Authored: Fri Jul 25 13:01:10 2014 +0200
Committer: wobu <bu...@googlemail.com>
Committed: Fri Jul 25 13:01:10 2014 +0200

----------------------------------------------------------------------
 .../vectorizer/term/TFPartialVectorReducer.java | 16 ++++--
 .../vectorizer/DictionaryVectorizerTest.java    | 60 +++++++++++++++++---
 2 files changed, 65 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mahout/blob/ccaec1b2/mrlegacy/src/main/java/org/apache/mahout/vectorizer/term/TFPartialVectorReducer.java
----------------------------------------------------------------------
diff --git a/mrlegacy/src/main/java/org/apache/mahout/vectorizer/term/TFPartialVectorReducer.java b/mrlegacy/src/main/java/org/apache/mahout/vectorizer/term/TFPartialVectorReducer.java
index e8b24e1..53246ef 100644
--- a/mrlegacy/src/main/java/org/apache/mahout/vectorizer/term/TFPartialVectorReducer.java
+++ b/mrlegacy/src/main/java/org/apache/mahout/vectorizer/term/TFPartialVectorReducer.java
@@ -17,6 +17,7 @@
 
 package org.apache.mahout.vectorizer.term;
 
+import com.google.common.collect.Lists;
 import com.google.common.io.Closeables;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.filecache.DistributedCache;
@@ -44,6 +45,7 @@ import org.apache.mahout.vectorizer.common.PartialVectorMerger;
 import java.io.IOException;
 import java.net.URI;
 import java.util.Iterator;
+import java.util.List;
 
 /**
  * Converts a document in to a sparse vector
@@ -61,15 +63,21 @@ public class TFPartialVectorReducer extends Reducer<Text, StringTuple, Text, Vec
   protected void reduce(Text key, Iterable<StringTuple> values, Context context)
     throws IOException, InterruptedException {
     Iterator<StringTuple> it = values.iterator();
+
     if (!it.hasNext()) {
       return;
     }
-    StringTuple value = it.next();
 
-    Vector vector = new RandomAccessSparseVector(dimension, value.length()); // guess at initial size
+    List<String> value = Lists.newArrayList();
+
+    while (it.hasNext()) {
+      value.addAll(it.next().getEntries());
+    }
+
+    Vector vector = new RandomAccessSparseVector(dimension, value.size()); // guess at initial size
 
     if (maxNGramSize >= 2) {
-      ShingleFilter sf = new ShingleFilter(new IteratorTokenStream(value.getEntries().iterator()), maxNGramSize);
+      ShingleFilter sf = new ShingleFilter(new IteratorTokenStream(value.iterator()), maxNGramSize);
       sf.reset();
       try {
         do {
@@ -85,7 +93,7 @@ public class TFPartialVectorReducer extends Reducer<Text, StringTuple, Text, Vec
         Closeables.close(sf, true);
       }
     } else {
-      for (String term : value.getEntries()) {
+      for (String term : value) {
         if (!term.isEmpty() && dictionary.containsKey(term)) { // unigram
           int termId = dictionary.get(term);
           vector.setQuick(termId, vector.getQuick(termId) + 1);

http://git-wip-us.apache.org/repos/asf/mahout/blob/ccaec1b2/mrlegacy/src/test/java/org/apache/mahout/vectorizer/DictionaryVectorizerTest.java
----------------------------------------------------------------------
diff --git a/mrlegacy/src/test/java/org/apache/mahout/vectorizer/DictionaryVectorizerTest.java b/mrlegacy/src/test/java/org/apache/mahout/vectorizer/DictionaryVectorizerTest.java
index edcc79b..835854f 100644
--- a/mrlegacy/src/test/java/org/apache/mahout/vectorizer/DictionaryVectorizerTest.java
+++ b/mrlegacy/src/test/java/org/apache/mahout/vectorizer/DictionaryVectorizerTest.java
@@ -18,14 +18,17 @@
 package org.apache.mahout.vectorizer;
 
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
-import com.carrotsearch.randomizedtesting.annotations.ThreadLeakLingering;
 import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope;
 import com.google.common.io.Closeables;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.lucene.analysis.Analyzer;
@@ -34,6 +37,7 @@ import org.apache.mahout.common.MahoutTestCase;
 import org.apache.mahout.common.Pair;
 import org.apache.mahout.common.iterator.sequencefile.PathFilters;
 import org.apache.mahout.common.iterator.sequencefile.PathType;
+import org.apache.mahout.common.iterator.sequencefile.SequenceFileDirIterable;
 import org.apache.mahout.common.iterator.sequencefile.SequenceFileDirValueIterable;
 import org.apache.mahout.math.NamedVector;
 import org.apache.mahout.math.RandomAccessSparseVector;
@@ -51,6 +55,7 @@ import org.junit.Test;
 public final class DictionaryVectorizerTest extends MahoutTestCase {
 
   private static final int NUM_DOCS = 100;
+  private static final String SECOND_TEXT_BLOCK_IDENTIFIER = "2NDBLOCK";
 
   private Path inputPath;
   
@@ -69,13 +74,18 @@ public final class DictionaryVectorizerTest extends MahoutTestCase {
       RandomDocumentGenerator gen = new RandomDocumentGenerator();
 
       for (int i = 0; i < NUM_DOCS; i++) {
-        writer.append(new Text("Document::ID::" + i), new Text(gen.getRandomDocument()));
+        writer.append(
+                new Text("Document::ID::" + i),
+                new Text(gen.getRandomDocument()));
+        writer.append(
+                new Text("Document::ID::" + i),
+                new Text(SECOND_TEXT_BLOCK_IDENTIFIER));
       }
     } finally {
       Closeables.close(writer, false);
     }
   }
-  
+
   @Test
   public void testCreateTermFrequencyVectors() throws Exception {
     runTest(false, false);
@@ -85,7 +95,7 @@ public final class DictionaryVectorizerTest extends MahoutTestCase {
   public void testCreateTermFrequencyVectorsNam() throws Exception {
     runTest(false, true);
   }
-  
+
   @Test
   public void testCreateTermFrequencyVectorsSeq() throws Exception {
     runTest(true, false);
@@ -106,7 +116,7 @@ public final class DictionaryVectorizerTest extends MahoutTestCase {
     Path tfVectors = new Path(wordCount, "tf-vectors");
     Path tfidf = getTestTempDirPath("output/tfidf");
     Path tfidfVectors = new Path(tfidf, "tfidf-vectors");
-    
+
     Configuration conf = getConfiguration();
     DocumentProcessor.tokenizeDocuments(inputPath, analyzer, tokenizedDocuments, conf);
     
@@ -123,7 +133,7 @@ public final class DictionaryVectorizerTest extends MahoutTestCase {
                                                     100,
                                                     sequential,
                                                     named);
-    
+
     validateVectors(conf, NUM_DOCS, tfVectors, sequential, named);
     
     Pair<Long[], List<Path>> docFrequenciesFeatures = TFIDFConverter.calculateDF(tfVectors, 
@@ -143,6 +153,10 @@ public final class DictionaryVectorizerTest extends MahoutTestCase {
     
     
     validateVectors(conf, NUM_DOCS, tfidfVectors, sequential, named);
+
+    Integer secondTextBlockIdentifierDimensionId = validateDictionary(wordCount, conf);
+
+    validateVectorContainingSecondTextBlock(conf, tfVectors, secondTextBlockIdentifierDimensionId);
   }
   
   public static void validateVectors(Configuration conf,
@@ -166,9 +180,41 @@ public final class DictionaryVectorizerTest extends MahoutTestCase {
       } else {
         assertTrue("Expected RandomAccessSparseVector", v instanceof RandomAccessSparseVector);
       }
+    }
+
+    assertEquals("Expected " + numDocs + " documents", numDocs, count);
+  }
+
+  private Integer validateDictionary(Path dictionaryDirectoryPath, Configuration conf) {
+    PathFilter dictionaryChunkPathFilter = new PathFilter() {
+      @Override
+      public boolean accept(Path path) {
+        String name = path.getName();
+        return name.startsWith("dictionary.file");
+      }
+    };
+
+    Map<String, Integer> dictionary = new HashMap<String, Integer>();
 
+    for (Pair<Text, IntWritable> value :
+            new SequenceFileDirIterable<Text, IntWritable>(
+                    dictionaryDirectoryPath, PathType.LIST, dictionaryChunkPathFilter, null, true, conf)) {
+      dictionary.put(value.getFirst().toString(), value.getSecond().get());
     }
 
-  assertEquals("Expected " + numDocs + " documents", numDocs, count);
+    Integer secondTextBlockIdentifierDimensionId = dictionary.get(SECOND_TEXT_BLOCK_IDENTIFIER.toLowerCase());
+
+    assertNotNull("Token '" + SECOND_TEXT_BLOCK_IDENTIFIER + "' must be in dictionary ", secondTextBlockIdentifierDimensionId);
+    assertTrue("Dictionary must contain more than just 1 element!", dictionary.size() > 1);
+
+    return secondTextBlockIdentifierDimensionId;
+  }
+
+  public static void validateVectorContainingSecondTextBlock(Configuration conf, Path vectorPath, int dimensionId) {
+    for (VectorWritable value :
+            new SequenceFileDirValueIterable<VectorWritable>(
+                    vectorPath, PathType.LIST, PathFilters.partFilter(), null, true, conf)) {
+      assertTrue("The vector must contain the second text block", value.get().get(dimensionId) > 0);
+    }
   }
 }


[2/3] mahout git commit: Merge branch 'mahout-1598' of https://github.com/wobu/mahout

Posted by ak...@apache.org.
Merge branch 'mahout-1598' of https://github.com/wobu/mahout


Project: http://git-wip-us.apache.org/repos/asf/mahout/repo
Commit: http://git-wip-us.apache.org/repos/asf/mahout/commit/1f17d23f
Tree: http://git-wip-us.apache.org/repos/asf/mahout/tree/1f17d23f
Diff: http://git-wip-us.apache.org/repos/asf/mahout/diff/1f17d23f

Branch: refs/heads/master
Commit: 1f17d23f6095e67a2f8d188f4817e286cf916e98
Parents: 4b1c133 ccaec1b
Author: Andrew Musselman <ak...@apache.org>
Authored: Mon Mar 30 08:46:53 2015 -0700
Committer: Andrew Musselman <ak...@apache.org>
Committed: Mon Mar 30 08:46:53 2015 -0700

----------------------------------------------------------------------
 .../vectorizer/term/TFPartialVectorReducer.java | 16 ++++--
 .../vectorizer/DictionaryVectorizerTest.java    | 60 +++++++++++++++++---
 2 files changed, 65 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mahout/blob/1f17d23f/mrlegacy/src/main/java/org/apache/mahout/vectorizer/term/TFPartialVectorReducer.java
----------------------------------------------------------------------


[3/3] mahout git commit: This closes #34

Posted by ak...@apache.org.
This closes #34


Project: http://git-wip-us.apache.org/repos/asf/mahout/repo
Commit: http://git-wip-us.apache.org/repos/asf/mahout/commit/91c1626d
Tree: http://git-wip-us.apache.org/repos/asf/mahout/tree/91c1626d
Diff: http://git-wip-us.apache.org/repos/asf/mahout/diff/91c1626d

Branch: refs/heads/master
Commit: 91c1626df337900051ead7019f70bd15fe93f6a7
Parents: 1f17d23
Author: Andrew Musselman <ak...@apache.org>
Authored: Mon Mar 30 09:33:35 2015 -0700
Committer: Andrew Musselman <ak...@apache.org>
Committed: Mon Mar 30 09:33:35 2015 -0700

----------------------------------------------------------------------
 CHANGELOG | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mahout/blob/91c1626d/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 7b47a0f..2660638 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -2,6 +2,8 @@ Mahout Change Log
 
 Release 0.10.0 - unreleased
 
+  MAHOUT-1598: extend seq2sparse to handle multiple text blocks of same document (Wolfgang Buchnere via akm)
+
   MAHOUT-1659: Remove deprecated Lanczos solver from spectral clustering in mr-legacy (Shannon Quinn)
 
   MAHOUT-1612: NullPointerException happens during JSON output format for clusterdumper (smarthi, Manoj Awasthi)