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/10/02 22:10:52 UTC

svn commit: r1003874 - in /mahout/trunk: math/src/main/java/org/apache/mahout/math/ math/src/test/java/org/apache/mahout/math/ utils/src/main/java/org/apache/mahout/text/ utils/src/main/java/org/apache/mahout/utils/vectors/common/ utils/src/main/java/o...

Author: robinanil
Date: Sat Oct  2 20:10:51 2010
New Revision: 1003874

URL: http://svn.apache.org/viewvc?rev=1003874&view=rev
Log:
MAHOUT-287 Adding lognormalize option to dictionary vectorizer

Modified:
    mahout/trunk/math/src/main/java/org/apache/mahout/math/AbstractVector.java
    mahout/trunk/math/src/main/java/org/apache/mahout/math/NamedVector.java
    mahout/trunk/math/src/main/java/org/apache/mahout/math/Vector.java
    mahout/trunk/math/src/test/java/org/apache/mahout/math/VectorTest.java
    mahout/trunk/utils/src/main/java/org/apache/mahout/text/SparseVectorsFromSequenceFiles.java
    mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/common/PartialVectorMergeReducer.java
    mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/common/PartialVectorMerger.java
    mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizer.java
    mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/tfidf/TFIDFConverter.java
    mahout/trunk/utils/src/test/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizerTest.java

Modified: mahout/trunk/math/src/main/java/org/apache/mahout/math/AbstractVector.java
URL: http://svn.apache.org/viewvc/mahout/trunk/math/src/main/java/org/apache/mahout/math/AbstractVector.java?rev=1003874&r1=1003873&r2=1003874&view=diff
==============================================================================
--- mahout/trunk/math/src/main/java/org/apache/mahout/math/AbstractVector.java (original)
+++ mahout/trunk/math/src/main/java/org/apache/mahout/math/AbstractVector.java Sat Oct  2 20:10:51 2010
@@ -161,6 +161,30 @@ public abstract class AbstractVector imp
   public Vector normalize(double power) {
     return divide(norm(power));
   }
+  
+  public Vector logNormalize() {
+      return logNormalize(2, Math.sqrt(dotSelf()));
+  }
+  
+  public Vector logNormalize(double power) {
+    return logNormalize(power, norm(power));
+  }
+  
+  public Vector logNormalize(double power, double normLength) {   
+    // we can special case certain powers
+    if (Double.isInfinite(power) || power <= 1.0) {
+      throw new IllegalArgumentException("Power must be > 1 and < infinity");
+    } else {
+      double denominator = normLength * Math.log(power);
+      Vector result = like().assign(this);
+      Iterator<Element> iter = result.iterateNonZero();
+      while (iter.hasNext()) {
+        Element element = iter.next();
+        element.set(Math.log(1 + element.get()) / denominator);
+      }
+      return result;
+    }
+  }
 
   public double norm(double power) {
     if (power < 0.0) {

Modified: mahout/trunk/math/src/main/java/org/apache/mahout/math/NamedVector.java
URL: http://svn.apache.org/viewvc/mahout/trunk/math/src/main/java/org/apache/mahout/math/NamedVector.java?rev=1003874&r1=1003873&r2=1003874&view=diff
==============================================================================
--- mahout/trunk/math/src/main/java/org/apache/mahout/math/NamedVector.java (original)
+++ mahout/trunk/math/src/main/java/org/apache/mahout/math/NamedVector.java Sat Oct  2 20:10:51 2010
@@ -151,6 +151,14 @@ public class NamedVector implements Vect
     return delegate.normalize(power);
   }
 
+  public Vector logNormalize() {
+    return delegate.logNormalize();
+  }
+
+  public Vector logNormalize(double power) {
+    return delegate.logNormalize(power);
+  }
+
   public double norm(double power) {
     return delegate.norm(power);
   }

Modified: mahout/trunk/math/src/main/java/org/apache/mahout/math/Vector.java
URL: http://svn.apache.org/viewvc/mahout/trunk/math/src/main/java/org/apache/mahout/math/Vector.java?rev=1003874&r1=1003873&r2=1003874&view=diff
==============================================================================
--- mahout/trunk/math/src/main/java/org/apache/mahout/math/Vector.java (original)
+++ mahout/trunk/math/src/main/java/org/apache/mahout/math/Vector.java Sat Oct  2 20:10:51 2010
@@ -222,6 +222,21 @@ public interface Vector extends Cloneabl
    * @return a new Vector
    */
   Vector normalize(double power);
+  
+  /**
+   * Return a new vector containing the log(1 + entry)/ L_2 norm  values of the recipient
+   *
+   * @return a new Vector
+   */
+  Vector logNormalize();
+
+  /**
+   * Return a new Vector with a normalized value calculated as log_power(1 + entry)/ L_power norm. <p/>
+   *
+   * @param power The power to use. Must be > 1. Cannot be {@link Double#POSITIVE_INFINITY}.
+   * @return a new Vector
+   */
+  Vector logNormalize(double power);
 
   /**
    * Return the k-norm of the vector. <p/> See http://en.wikipedia.org/wiki/Lp_space <p/> Technically, when 0 &gt; power

Modified: mahout/trunk/math/src/test/java/org/apache/mahout/math/VectorTest.java
URL: http://svn.apache.org/viewvc/mahout/trunk/math/src/test/java/org/apache/mahout/math/VectorTest.java?rev=1003874&r1=1003873&r2=1003874&view=diff
==============================================================================
--- mahout/trunk/math/src/test/java/org/apache/mahout/math/VectorTest.java (original)
+++ mahout/trunk/math/src/test/java/org/apache/mahout/math/VectorTest.java Sat Oct  2 20:10:51 2010
@@ -460,6 +460,61 @@ public final class VectorTest extends Ma
       // expected
     }
   }
+  
+  @Test
+  public void testLogNormalize() {
+    Vector vec1 = new RandomAccessSparseVector(3);
+    
+    vec1.setQuick(0, 1);
+    vec1.setQuick(1, 2);
+    vec1.setQuick(2, 3);
+    Vector norm = vec1.logNormalize();
+    assertNotNull("norm1 is null and it shouldn't be", norm);
+    
+    Vector vec2 = new SequentialAccessSparseVector(3);
+    
+    vec2.setQuick(0, 1);
+    vec2.setQuick(1, 2);
+    vec2.setQuick(2, 3);
+    Vector norm2 = vec2.logNormalize();
+    assertNotNull("norm1 is null and it shouldn't be", norm2);
+    
+    Vector expected = new RandomAccessSparseVector(3);
+    
+    expected.setQuick(0, 0.2672612419124244);
+    expected.setQuick(1, 0.4235990463273581);
+    expected.setQuick(2, 0.5345224838248488);
+    
+    assertEquals(expected, norm);
+    
+    norm = vec1.logNormalize(2);
+    assertEquals(expected, norm);
+    
+    norm2 = vec2.logNormalize(2);
+    assertEquals(expected, norm2);
+    
+    try {
+      norm = vec1.logNormalize(1);
+      fail();
+    } catch (IllegalArgumentException e) {
+      // expected
+    }
+    norm = vec1.logNormalize(3);
+  
+    try {
+      vec1.logNormalize(-1);
+      fail();
+    } catch (IllegalArgumentException e) {
+      // expected
+    }
+    
+    try {
+      vec2.logNormalize(Double.POSITIVE_INFINITY);
+      fail();
+    } catch (IllegalArgumentException e) {
+      // expected
+    }  
+  }
 
   @Test
   public void testMax()  {

Modified: mahout/trunk/utils/src/main/java/org/apache/mahout/text/SparseVectorsFromSequenceFiles.java
URL: http://svn.apache.org/viewvc/mahout/trunk/utils/src/main/java/org/apache/mahout/text/SparseVectorsFromSequenceFiles.java?rev=1003874&r1=1003873&r2=1003874&view=diff
==============================================================================
--- mahout/trunk/utils/src/main/java/org/apache/mahout/text/SparseVectorsFromSequenceFiles.java (original)
+++ mahout/trunk/utils/src/main/java/org/apache/mahout/text/SparseVectorsFromSequenceFiles.java Sat Oct  2 20:10:51 2010
@@ -100,6 +100,11 @@ public final class SparseVectorsFromSequ
       "The norm to use, expressed as either a float or \"INF\" if you want to use the Infinite norm.  "
           + "Must be greater or equal to 0.  The default is not to normalize").withShortName("n").create();
     
+    Option logNormalizeOpt = obuilder.withLongName("logNormalize").withRequired(false)
+    .withDescription(
+      "(Optional) Whether output vectors should be logNormalize. If set true else false")
+    .withShortName("lnorm").create();
+    
     Option maxNGramSizeOpt = obuilder.withLongName("maxNGramSize").withRequired(false).withArgument(
       abuilder.withName("ngramSize").withMinimum(1).withMaximum(1).create())
         .withDescription(
@@ -124,8 +129,9 @@ public final class SparseVectorsFromSequ
     Group group = gbuilder.withName("Options").withOption(minSupportOpt).withOption(analyzerNameOpt)
         .withOption(chunkSizeOpt).withOption(outputDirOpt).withOption(inputDirOpt).withOption(minDFOpt)
         .withOption(maxDFPercentOpt).withOption(weightOpt).withOption(powerOpt).withOption(minLLROpt)
-        .withOption(numReduceTasksOpt).withOption(maxNGramSizeOpt).withOption(overwriteOutput).withOption(
-          helpOpt).withOption(sequentialAccessVectorOpt).withOption(namedVectorOpt).create();
+        .withOption(numReduceTasksOpt).withOption(maxNGramSizeOpt).withOption(overwriteOutput)
+        .withOption(helpOpt).withOption(sequentialAccessVectorOpt).withOption(namedVectorOpt).withOption(logNormalizeOpt)
+        .create();
     try {
       Parser parser = new Parser();
       parser.setGroup(group);
@@ -219,6 +225,12 @@ public final class SparseVectorsFromSequ
           norm = Float.parseFloat(power);
         }
       }
+      
+      boolean logNormalize = false;
+      if (cmdLine.hasOption(logNormalizeOpt)) {
+        logNormalize = true;
+      }
+      
       HadoopUtil.overwriteOutput(outputDir);
       Path tokenizedPath = new Path(outputDir, DocumentProcessor.TOKENIZED_DOCUMENT_OUTPUT_FOLDER);
       DocumentProcessor.tokenizeDocuments(inputDir, analyzerClass, tokenizedPath);
@@ -234,12 +246,16 @@ public final class SparseVectorsFromSequ
       }
       
       Configuration conf = new Configuration();
-      DictionaryVectorizer.createTermFrequencyVectors(tokenizedPath, outputDir, conf, minSupport, maxNGramSize,
-        minLLRValue, reduceTasks, chunkSize, sequentialAccessOutput, namedVectors);
-      if (processIdf) {
+      if (!processIdf) {
+        DictionaryVectorizer.createTermFrequencyVectors(tokenizedPath, outputDir, conf, minSupport, maxNGramSize,
+          minLLRValue, norm, logNormalize, reduceTasks, chunkSize, sequentialAccessOutput, namedVectors);
+      } else if (processIdf) {
+        DictionaryVectorizer.createTermFrequencyVectors(tokenizedPath, outputDir, conf, minSupport, maxNGramSize,
+          minLLRValue, -1.0f, false, reduceTasks, chunkSize, sequentialAccessOutput, namedVectors);
+      
         TFIDFConverter.processTfIdf(
           new Path(outputDir, DictionaryVectorizer.DOCUMENT_VECTOR_OUTPUT_FOLDER),
-          outputDir, chunkSize, minDf, maxDFPercent, norm,
+          outputDir, chunkSize, minDf, maxDFPercent, norm, logNormalize,
           sequentialAccessOutput, namedVectors, reduceTasks);
       }
     } catch (OptionException e) {

Modified: mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/common/PartialVectorMergeReducer.java
URL: http://svn.apache.org/viewvc/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/common/PartialVectorMergeReducer.java?rev=1003874&r1=1003873&r2=1003874&view=diff
==============================================================================
--- mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/common/PartialVectorMergeReducer.java (original)
+++ mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/common/PartialVectorMergeReducer.java Sat Oct  2 20:10:51 2010
@@ -42,6 +42,8 @@ public class PartialVectorMergeReducer e
 
   private boolean namedVector;
   
+  private boolean logNormalize;
+  
   @Override
   protected void reduce(WritableComparable<?> key, Iterable<VectorWritable> values, Context context) throws IOException,
       InterruptedException {
@@ -51,7 +53,11 @@ public class PartialVectorMergeReducer e
       value.get().addTo(vector);
     }
     if (normPower != PartialVectorMerger.NO_NORMALIZING) {
-      vector = vector.normalize(normPower);
+      if (logNormalize) {
+        vector = vector.logNormalize(normPower);
+      } else {
+        vector = vector.normalize(normPower);
+      }
     }
     if (sequentialAccess) {
       vector = new SequentialAccessSparseVector(vector);
@@ -73,6 +79,7 @@ public class PartialVectorMergeReducer e
     dimension = conf.getInt(PartialVectorMerger.DIMENSION, Integer.MAX_VALUE);
     sequentialAccess = conf.getBoolean(PartialVectorMerger.SEQUENTIAL_ACCESS, false);
     namedVector = conf.getBoolean(PartialVectorMerger.NAMED_VECTOR, false);
+    logNormalize = conf.getBoolean(PartialVectorMerger.LOG_NORMALIZE, false);
   }
 
 }

Modified: mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/common/PartialVectorMerger.java
URL: http://svn.apache.org/viewvc/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/common/PartialVectorMerger.java?rev=1003874&r1=1003873&r2=1003874&view=diff
==============================================================================
--- mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/common/PartialVectorMerger.java (original)
+++ mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/common/PartialVectorMerger.java Sat Oct  2 20:10:51 2010
@@ -50,6 +50,8 @@ public final class PartialVectorMerger {
   
   public static final String NAMED_VECTOR = "vector.named";
 
+  public static final String LOG_NORMALIZE = "vector.lognormalize";
+
   /**
    * Cannot be initialized. Use the static functions
    */
@@ -78,6 +80,7 @@ public final class PartialVectorMerger {
   public static void mergePartialVectors(Iterable<Path> partialVectorPaths,
                                          Path output,
                                          float normPower,
+                                         boolean logNormalize,
                                          int dimension,
                                          boolean sequentialAccess,
                                          boolean namedVector,
@@ -86,6 +89,9 @@ public final class PartialVectorMerger {
     if (normPower != NO_NORMALIZING && normPower < 0) {
       throw new IllegalArgumentException("normPower must either be -1 or >= 0");
     }
+    if (normPower != NO_NORMALIZING && (normPower <= 1 || Double.isInfinite(normPower)) && logNormalize) {
+      throw new IllegalArgumentException("normPower must be > 1 and not +infinity if log normalization is chosen");
+    }
 
     Configuration conf = new Configuration();
     // this conf parameter needs to be set enable serialisation of conf values
@@ -95,6 +101,7 @@ public final class PartialVectorMerger {
     conf.setBoolean(NAMED_VECTOR, namedVector);
     conf.setInt(DIMENSION, dimension);
     conf.setFloat(NORMALIZATION_POWER, normPower);
+    conf.setBoolean(LOG_NORMALIZE, logNormalize);
 
     Job job = new Job(conf);
     job.setJobName("PartialVectorMerger::MergePartialVectors");

Modified: mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizer.java
URL: http://svn.apache.org/viewvc/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizer.java?rev=1003874&r1=1003873&r2=1003874&view=diff
==============================================================================
--- mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizer.java (original)
+++ mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizer.java Sat Oct  2 20:10:51 2010
@@ -97,6 +97,10 @@ public final class DictionaryVectorizer 
    * @param output
    *          output directory where {@link org.apache.mahout.math.RandomAccessSparseVector}'s of the document
    *          are generated
+   * @param normPower
+   *          L_p norm to be computed
+   * @param logNormalize
+   *          whether to use log normalization         
    * @param minSupport
    *          the minimum frequency of the feature in the entire corpus to be considered for inclusion in the
    *          sparse vector
@@ -117,11 +121,19 @@ public final class DictionaryVectorizer 
                                                 int minSupport,
                                                 int maxNGramSize,
                                                 float minLLRValue,
+                                                float normPower,
+                                                boolean logNormalize,
                                                 int numReducers,
                                                 int chunkSizeInMegabytes,
                                                 boolean sequentialAccess,
                                                 boolean namedVectors)
     throws IOException, InterruptedException, ClassNotFoundException {
+    if (normPower != PartialVectorMerger.NO_NORMALIZING && normPower < 0) {
+      throw new IllegalArgumentException("normPower must either be -1 or >= 0");
+    }
+    if (normPower != PartialVectorMerger.NO_NORMALIZING && (normPower <= 1 || Double.isInfinite(normPower)) && logNormalize) {
+      throw new IllegalArgumentException("normPower must be > 1 and not +infinity if log normalization is chosen");
+    }
     if (chunkSizeInMegabytes < MIN_CHUNKSIZE) {
       chunkSizeInMegabytes = MIN_CHUNKSIZE;
     } else if (chunkSizeInMegabytes > MAX_CHUNKSIZE) { // 10GB
@@ -160,15 +172,9 @@ public final class DictionaryVectorizer 
     FileSystem fs = FileSystem.get(partialVectorPaths.get(0).toUri(), conf);
     
     Path outputDir = new Path(output, DOCUMENT_VECTOR_OUTPUT_FOLDER);
-    if (dictionaryChunks.size() > 1) {
-      PartialVectorMerger.mergePartialVectors(partialVectorPaths, outputDir, -1, maxTermDimension[0],
-        sequentialAccess, namedVectors, numReducers);
-      HadoopUtil.deletePaths(partialVectorPaths, fs);
-    } else {
-      Path singlePartialVectorOutputPath = partialVectorPaths.get(0);
-      fs.delete(outputDir, true);
-      fs.rename(singlePartialVectorOutputPath, outputDir);
-    }
+    PartialVectorMerger.mergePartialVectors(partialVectorPaths, outputDir, normPower, logNormalize,
+      maxTermDimension[0], sequentialAccess, namedVectors, numReducers);
+    HadoopUtil.deletePaths(partialVectorPaths, fs);  
   }
   
   /**

Modified: mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/tfidf/TFIDFConverter.java
URL: http://svn.apache.org/viewvc/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/tfidf/TFIDFConverter.java?rev=1003874&r1=1003873&r2=1003874&view=diff
==============================================================================
--- mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/tfidf/TFIDFConverter.java (original)
+++ mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/tfidf/TFIDFConverter.java Sat Oct  2 20:10:51 2010
@@ -119,6 +119,7 @@ public final class TFIDFConverter {
                                   int minDf,
                                   int maxDFPercent,
                                   float normPower,
+                                  boolean logNormalize,
                                   boolean sequentialAccessOutput,
                                   boolean namedVector,
                                   int numReducers) throws IOException, InterruptedException, ClassNotFoundException {
@@ -131,6 +132,9 @@ public final class TFIDFConverter {
     if (normPower != PartialVectorMerger.NO_NORMALIZING && normPower < 0) {
       throw new IllegalArgumentException("normPower must either be -1 or >= 0");
     }
+    if (normPower != PartialVectorMerger.NO_NORMALIZING && (normPower <= 1 || Double.isInfinite(normPower)) && logNormalize) {
+      throw new IllegalArgumentException("normPower must be > 1 and not +infinity if log normalization is chosen");
+    }
 
     if (minDf < 1) {
       minDf = 1;
@@ -165,20 +169,17 @@ public final class TFIDFConverter {
     FileSystem fs = FileSystem.get(partialVectorPaths.get(0).toUri(), conf);
 
     Path outputDir = new Path(output, DOCUMENT_VECTOR_OUTPUT_FOLDER);
-    if (dictionaryChunks.size() > 1) {
-      PartialVectorMerger.mergePartialVectors(partialVectorPaths,
-                                              outputDir,
-                                              normPower,
-                                              datasetFeatures.getFirst()[0].intValue(),
-                                              sequentialAccessOutput,
-                                              namedVector,
-                                              numReducers);
-      HadoopUtil.deletePaths(partialVectorPaths, fs);
-    } else {
-      Path singlePartialVectorOutputPath = partialVectorPaths.get(0);
-      fs.delete(outputDir, true);
-      fs.rename(singlePartialVectorOutputPath, outputDir);
-    }
+    
+    PartialVectorMerger.mergePartialVectors(partialVectorPaths,
+                                            outputDir,
+                                            normPower,
+                                            logNormalize,
+                                            datasetFeatures.getFirst()[0].intValue(),
+                                            sequentialAccessOutput,
+                                            namedVector,
+                                            numReducers);
+    HadoopUtil.deletePaths(partialVectorPaths, fs);
+
   }
 
   /**

Modified: mahout/trunk/utils/src/test/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizerTest.java
URL: http://svn.apache.org/viewvc/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizerTest.java?rev=1003874&r1=1003873&r2=1003874&view=diff
==============================================================================
--- mahout/trunk/utils/src/test/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizerTest.java (original)
+++ mahout/trunk/utils/src/test/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizerTest.java Sat Oct  2 20:10:51 2010
@@ -103,6 +103,8 @@ public final class DictionaryVectorizerT
                                                     2,
                                                     1,
                                                     0.0f,
+                                                    -1.0f,
+                                                    true,
                                                     1,
                                                     100,
                                                     sequential,
@@ -115,7 +117,8 @@ public final class DictionaryVectorizerT
                                 100,
                                 1,
                                 99,
-                                1.0f,
+                                2.0f,
+                                false,
                                 sequential,
                                 named,
                                 1);