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 2013/06/02 16:57:18 UTC

svn commit: r1488725 - in /mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer: HighDFWordsPruner.java SparseVectorsFromSequenceFiles.java pruner/WordsPrunerReducer.java

Author: robinanil
Date: Sun Jun  2 14:57:18 2013
New Revision: 1488725

URL: http://svn.apache.org/r1488725
Log:
MAHOUT-962 minDF and maxDFPercent filtering doesnt get applied when output weight is tf in SpareVecorsFromSequenceFile (Dave Byrne)

Modified:
    mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/HighDFWordsPruner.java
    mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/SparseVectorsFromSequenceFiles.java
    mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/pruner/WordsPrunerReducer.java

Modified: mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/HighDFWordsPruner.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/HighDFWordsPruner.java?rev=1488725&r1=1488724&r2=1488725&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/HighDFWordsPruner.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/HighDFWordsPruner.java Sun Jun  2 14:57:18 2013
@@ -43,12 +43,13 @@ public final class HighDFWordsPruner {
 
   public static final String STD_CALC_DIR = "stdcalc";
   public static final String MAX_DF = "max.df";
+  public static final String MIN_DF = "min.df";
 
   private HighDFWordsPruner() {
   }
 
   public static void pruneVectors(Path tfDir, Path prunedTFDir, Path prunedPartialTFDir, long maxDF,
-                                  Configuration baseConf,
+                                  long minDF, Configuration baseConf,
                                   Pair<Long[], List<Path>> docFrequenciesFeatures,
                                   float normPower,
                                   boolean logNormalize,
@@ -59,7 +60,7 @@ public final class HighDFWordsPruner {
     for (Path path : docFrequenciesFeatures.getSecond()) {
       Path partialVectorOutputPath = new Path(prunedPartialTFDir, "partial-" + partialVectorIndex++);
       partialVectorPaths.add(partialVectorOutputPath);
-      pruneVectorsPartial(tfDir, partialVectorOutputPath, path, maxDF, baseConf);
+      pruneVectorsPartial(tfDir, partialVectorOutputPath, path, maxDF, minDF, baseConf);
     }
 
     mergePartialVectors(partialVectorPaths, prunedTFDir, baseConf, normPower, logNormalize, numReducers);
@@ -67,7 +68,7 @@ public final class HighDFWordsPruner {
   }
 
   private static void pruneVectorsPartial(Path input, Path output, Path dictionaryFilePath, long maxDF,
-                                          Configuration baseConf) throws IOException, InterruptedException,
+                                          long minDF, Configuration baseConf) throws IOException, InterruptedException,
           ClassNotFoundException {
 
     Configuration conf = new Configuration(baseConf);
@@ -77,6 +78,7 @@ public final class HighDFWordsPruner {
             "org.apache.hadoop.io.serializer.JavaSerialization,"
                     + "org.apache.hadoop.io.serializer.WritableSerialization");
     conf.setLong(MAX_DF, maxDF);
+    conf.setLong(MIN_DF, minDF);
     DistributedCache.setCacheFiles(
             new URI[]{dictionaryFilePath.toUri()}, conf);
 

Modified: mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/SparseVectorsFromSequenceFiles.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/SparseVectorsFromSequenceFiles.java?rev=1488725&r1=1488724&r2=1488725&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/SparseVectorsFromSequenceFiles.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/SparseVectorsFromSequenceFiles.java Sun Jun  2 14:57:18 2013
@@ -264,7 +264,7 @@ public final class SparseVectorsFromSequ
       if (cmdLine.hasOption(namedVectorOpt)) {
         namedVectors = true;
       }
-      boolean shouldPrune = maxDFSigma >= 0.0;
+      boolean shouldPrune = maxDFSigma >= 0.0  || maxDFPercent > 0.00;
       String tfDirName = shouldPrune
           ? DictionaryVectorizer.DOCUMENT_VECTOR_OUTPUT_FOLDER + "-toprune"
           : DictionaryVectorizer.DOCUMENT_VECTOR_OUTPUT_FOLDER;
@@ -308,13 +308,17 @@ public final class SparseVectorsFromSequ
 
       long maxDF = maxDFPercent; //if we are pruning by std dev, then this will get changed
       if (shouldPrune) {
-        Path dfDir = new Path(outputDir, TFIDFConverter.WORDCOUNT_OUTPUT_FOLDER);
-        Path stdCalcDir = new Path(outputDir, HighDFWordsPruner.STD_CALC_DIR);
+	long vectorCount = docFrequenciesFeatures.getFirst()[1];
+	if (maxDFSigma >= 0.0) {
+	  Path dfDir = new Path(outputDir, TFIDFConverter.WORDCOUNT_OUTPUT_FOLDER);
+	  Path stdCalcDir = new Path(outputDir, HighDFWordsPruner.STD_CALC_DIR);
+
+	  // Calculate the standard deviation
+	  double stdDev = BasicStats.stdDevForGivenMean(dfDir, stdCalcDir, 0.0, conf);
+	  maxDF = (int) (100.0 * maxDFSigma * stdDev / vectorCount);
+	}
 
-        // Calculate the standard deviation
-        double stdDev = BasicStats.stdDevForGivenMean(dfDir, stdCalcDir, 0.0, conf);
-        long vectorCount = docFrequenciesFeatures.getFirst()[1];
-        maxDF = (int) (100.0 * maxDFSigma * stdDev / vectorCount);
+	long maxDFThreshold = (long) (vectorCount * ((float) maxDF / 100f));
 
         // Prune the term frequency vectors
         Path tfDir = new Path(outputDir, tfDirName);
@@ -326,7 +330,8 @@ public final class SparseVectorsFromSequ
           HighDFWordsPruner.pruneVectors(tfDir,
                                          prunedTFDir,
                                          prunedPartialTFDir,
-                                         maxDF,
+                                         maxDFThreshold,
+					 minDf,
                                          conf,
                                          docFrequenciesFeatures,
                                          -1.0f,
@@ -336,7 +341,8 @@ public final class SparseVectorsFromSequ
           HighDFWordsPruner.pruneVectors(tfDir,
                                          prunedTFDir,
                                          prunedPartialTFDir,
-                                         maxDF,
+                                         maxDFThreshold,
+					 minDf,
                                          conf,
                                          docFrequenciesFeatures,
                                          norm,

Modified: mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/pruner/WordsPrunerReducer.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/pruner/WordsPrunerReducer.java?rev=1488725&r1=1488724&r2=1488725&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/pruner/WordsPrunerReducer.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/pruner/WordsPrunerReducer.java Sun Jun  2 14:57:18 2013
@@ -39,7 +39,8 @@ public class WordsPrunerReducer extends
         Reducer<WritableComparable<?>, VectorWritable, WritableComparable<?>, VectorWritable> {
 
   private final OpenIntLongHashMap dictionary = new OpenIntLongHashMap();
-  private long maxDf = -1;
+  private long maxDf = Long.MAX_VALUE;
+  private long minDf = -1;
 
   @Override
   protected void reduce(WritableComparable<?> key, Iterable<VectorWritable> values, Context context)
@@ -50,14 +51,14 @@ public class WordsPrunerReducer extends
     }
     Vector value = it.next().get();
     Vector vector = value.clone();
-    if (maxDf > -1) {
+    if (maxDf != Long.MAX_VALUE || minDf > -1) {
       for (Vector.Element e : value.nonZeroes()) {
         if (!dictionary.containsKey(e.index())) {
           vector.setQuick(e.index(), 0.0);
           continue;
         }
         long df = dictionary.get(e.index());
-        if (df > maxDf) {
+        if (df > maxDf || df < minDf) {
           vector.setQuick(e.index(), 0.0);
         }
       }
@@ -75,7 +76,8 @@ public class WordsPrunerReducer extends
     Preconditions.checkArgument(localFiles != null && localFiles.length >= 1,
             "missing paths from the DistributedCache");
 
-    maxDf = conf.getLong(HighDFWordsPruner.MAX_DF, -1);
+    maxDf = conf.getLong(HighDFWordsPruner.MAX_DF, Long.MAX_VALUE);
+    minDf = conf.getLong(HighDFWordsPruner.MIN_DF, -1);
 
     Path dictionaryFile = new Path(localFiles[0].getPath());
     // key is feature, value is the document frequency