You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mahout.apache.org by jm...@apache.org on 2010/01/21 15:20:02 UTC

svn commit: r901718 - in /lucene/mahout/trunk/math/src: main/java/org/apache/mahout/math/decomposer/ main/java/org/apache/mahout/math/decomposer/hebbian/ main/java/org/apache/mahout/math/decomposer/lanczos/ test/java/org/apache/mahout/math/decomposer/ ...

Author: jmannix
Date: Thu Jan 21 14:20:01 2010
New Revision: 901718

URL: http://svn.apache.org/viewvc?rev=901718&view=rev
Log:
Initial import for MAHOUT-180.  Brings in all the core of decomposer.  Lots more to do (to be described on the list [and wiki - need wiki!]), but has a couple unit tests to verify correctness, and my recent checkins in the rest of mahout-math help optimize for performance.

Added:
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/EigenStatus.java   (with props)
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/EigenUpdater.java   (with props)
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/HebbianSolver.java   (with props)
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/HebbianUpdater.java   (with props)
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/MultiThreadedEigenVerifier.java   (with props)
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/SimpleEigenVerifier.java   (with props)
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/SingularVectorVerifier.java   (with props)
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/TrainingState.java   (with props)
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/lanczos/
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/lanczos/LanczosSolver.java   (with props)
    lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/
    lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/SolverTest.java   (with props)
    lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/hebbian/
    lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/hebbian/TestHebbianSolver.java   (with props)
    lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/lanczos/
    lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/lanczos/TestLanczosSolver.java   (with props)

Added: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/EigenStatus.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/EigenStatus.java?rev=901718&view=auto
==============================================================================
--- lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/EigenStatus.java (added)
+++ lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/EigenStatus.java Thu Jan 21 14:20:01 2010
@@ -0,0 +1,36 @@
+/**
+ * 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.math.decomposer.hebbian;
+
+public class EigenStatus {
+  private final double eigenValue;
+  private final double cosAngle;
+
+  public EigenStatus(double eigenValue, double cosAngle) {
+    this.eigenValue = eigenValue;
+    this.cosAngle = cosAngle;
+  }
+
+  public double getCosAngle() {
+    return cosAngle;
+  }
+
+  public double getEigenValue() {
+    return eigenValue;
+  }
+}

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/EigenStatus.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/EigenStatus.java
------------------------------------------------------------------------------
    svn:keywords = "Date Rev Author URL Id"

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/EigenStatus.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/EigenUpdater.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/EigenUpdater.java?rev=901718&view=auto
==============================================================================
--- lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/EigenUpdater.java (added)
+++ lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/EigenUpdater.java Thu Jan 21 14:20:01 2010
@@ -0,0 +1,25 @@
+/**
+ * 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.math.decomposer.hebbian;
+
+import org.apache.mahout.math.Vector;
+
+
+public interface EigenUpdater {
+  void update(Vector pseudoEigen, Vector trainingVector, TrainingState currentState);
+}

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/EigenUpdater.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/EigenUpdater.java
------------------------------------------------------------------------------
    svn:keywords = "Date Rev Author URL Id"

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/EigenUpdater.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/HebbianSolver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/HebbianSolver.java?rev=901718&view=auto
==============================================================================
--- lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/HebbianSolver.java (added)
+++ lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/HebbianSolver.java Thu Jan 21 14:20:01 2010
@@ -0,0 +1,331 @@
+/**
+ * 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.math.decomposer.hebbian;
+
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import java.util.ArrayList;
+
+import org.apache.mahout.math.AbstractMatrix;
+import org.apache.mahout.math.DenseMatrix;
+import org.apache.mahout.math.DenseVector;
+import org.apache.mahout.math.Matrix;
+import org.apache.mahout.math.PlusWithScaleFunction;
+import org.apache.mahout.math.TimesFunction;
+import org.apache.mahout.math.Vector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The Hebbian solver is an iterative, sparse, singular value decomposition solver, based on the paper
+ * <a href="http://www.dcs.shef.ac.uk/~genevieve/gorrell_webb.pdf">Generalized Hebbian Algorithm for
+ * Latent Semantic Analysis</a> (2005) by Genevieve Gorrell and Brandyn Webb (a.k.a. Simon Funk).
+ * TODO: more description here!  For now: read the inline comments, and the comments for the constructors.
+ */
+public class HebbianSolver {
+
+  /**
+   * Logger for this class.
+   */
+  private static final Logger log = LoggerFactory.getLogger(HebbianSolver.class);
+
+  protected final EigenUpdater updater;
+  protected final SingularVectorVerifier verifier;
+  protected final double convergenceTarget;
+  protected final int maxPassesPerEigen;
+
+  protected int numPasses = 0;
+  protected boolean debug = false;
+
+  /**
+   * Creates a new HebbianSolver
+   *
+   * @param updater           {@link EigenUpdater} used to do the actual work of iteratively updating the current "best guess"
+   *                          singular vector one data-point presentation at a time.
+   * @param verifier          {@link SingularVectorVerifier } an object which perpetually tries to check how close to
+   *                          convergence the current singular vector is (typically is a {@link MultiThreadedEigenVerifier } which does this
+   *                          in the background in another thread, while the main thread continues to converge)
+   * @param convergenceTarget a small "epsilon" value which tells the solver how small you want the cosine of the
+   *                          angle between a proposed eigenvector and that same vector after being multiplied by the (square of the) input
+   *                          corpus
+   * @param maxPassesPerEigen a cutoff which tells the solver after how many times of checking for convergence (done
+   *                          by the verifier) should the solver stop trying, even if it has not reached the convergenceTarget.
+   */
+  public HebbianSolver(EigenUpdater updater,
+                       SingularVectorVerifier verifier,
+                       double convergenceTarget,
+                       int maxPassesPerEigen) {
+    this.updater = updater;
+    this.verifier = verifier;
+    this.convergenceTarget = convergenceTarget;
+    this.maxPassesPerEigen = maxPassesPerEigen;
+  }
+
+  /**
+   * Creates a new HebbianSolver with maxPassesPerEigen = Integer.MAX_VALUE (i.e. keep on iterating until
+   * convergenceTarget is reached).  <b>Not recommended</b> unless only looking for the first few (5, maybe 10?) singular
+   * vectors, as small errors which compound early on quickly put a minimum error on subsequent vectors.
+   *
+   * @param updater           {@link EigenUpdater} used to do the actual work of iteratively updating the current "best guess"
+   *                          singular vector one data-point presentation at a time.
+   * @param verifier          {@link SingularVectorVerifier } an object which perpetually tries to check how close to
+   *                          convergence the current singular vector is (typically is a {@link MultiThreadedEigenVerifier } which does this
+   *                          in the background in another thread, while the main thread continues to converge)
+   * @param convergenceTarget a small "epsilon" value which tells the solver how small you want the cosine of the
+   *                          angle between a proposed eigenvector and that same vector after being multiplied by the (square of the) input
+   *                          corpus
+   */
+  public HebbianSolver(EigenUpdater updater,
+                       SingularVectorVerifier verifier,
+                       double convergenceTarget) {
+    this(updater,
+        verifier,
+        convergenceTarget,
+        Integer.MAX_VALUE);
+  }
+
+  /**
+   * <b>This is the recommended constructor to use if you're not sure</b>
+   * Creates a new HebbianSolver with the default {@link HebbianUpdater } to do the updating work, and the default
+   * {@link MultiThreadedEigenVerifier } to check for convergence in a (single) background thread.
+   *
+   * @param convergenceTarget a small "epsilon" value which tells the solver how small you want the cosine of the
+   *                          angle between a proposed eigenvector and that same vector after being multiplied by the (square of the) input
+   *                          corpus
+   * @param maxPassesPerEigen a cutoff which tells the solver after how many times of checking for convergence (done
+   *                          by the verifier) should the solver stop trying, even if it has not reached the convergenceTarget.
+   */
+  public HebbianSolver(double convergenceTarget, int maxPassesPerEigen) {
+    this(new HebbianUpdater(),
+        new MultiThreadedEigenVerifier(),
+        convergenceTarget,
+        maxPassesPerEigen);
+  }
+
+  /**
+   * Creates a new HebbianSolver with the default {@link HebbianUpdater } to do the updating work, and the default
+   * {@link MultiThreadedEigenVerifier } to check for convergence in a (single) background thread, with
+   * maxPassesPerEigen set to Integer.MAX_VALUE.  <b>Not recommended</b> unless only looking for the first few (5, maybe 10?) singular
+   * vectors, as small errors which compound early on quickly put a minimum error on subsequent vectors.
+   *
+   * @param convergenceTarget a small "epsilon" value which tells the solver how small you want the cosine of the
+   *                          angle between a proposed eigenvector and that same vector after being multiplied by the (square of the) input
+   *                          corpus
+   */
+  public HebbianSolver(double convergenceTarget) {
+    this(convergenceTarget, Integer.MAX_VALUE);
+  }
+
+  /**
+   * Creates a new HebbianSolver with the default {@link HebbianUpdater } to do the updating work, and the default
+   * {@link MultiThreadedEigenVerifier } to check for convergence in a (single) background thread, with
+   * convergenceTarget set to 0, which means that the solver will not really care about convergence as a loop-exiting
+   * criterion (but will be checking for convergence anyways, so it will be logged and singular values will be
+   * saved).
+   *
+   * @param numPassesPerEigen the exact number of times the verifier will check convergence status in the background
+   *                          before the solver will move on to the next eigen-vector.
+   */
+  public HebbianSolver(int numPassesPerEigen) {
+    this(0d, numPassesPerEigen);
+  }
+
+  /**
+   * Primary singular vector solving method.
+   *
+   * @param corpus      input matrix to find singular vectors of.  Needs not be symmetric, should probably be sparse (in
+   *                    fact the input vectors are not mutated, and accessed only via dot-products and sums, so they should be
+   *                    {@link org.apache.mahout.math.SequentialAccessSparseVector }
+   * @param desiredRank the number of singular vectors to find (in roughly decreasing order by singular value)
+   * @return the final {@link TrainingState } of the solver, after desiredRank singular vectors (and approximate
+   *         singular values) have been found.
+   */
+  public TrainingState solve(Matrix corpus,
+                             int desiredRank) {
+    int cols = corpus.numCols();
+    Matrix eigens = new DenseMatrix(desiredRank, cols);
+    List<Double> eigenValues = new ArrayList<Double>();
+    log.info("Finding " + desiredRank + " singular vectors of matrix with " + corpus.numRows() + " rows, via Hebbian");
+    /**
+     * The corpusProjections matrix is a running cache of the residual projection of each corpus vector against all
+     * of the previously found singular vectors.  Without this, if multiple passes over the data is made (per
+     * singular vector), recalculating these projections eventually dominates the computational complexity of the
+     * solver.
+     */
+    Matrix corpusProjections = new DenseMatrix(corpus.numRows(), desiredRank);
+    TrainingState state = new TrainingState(eigens, corpusProjections);
+    for (int i = 0; i < desiredRank; i++) {
+      Vector currentEigen = new DenseVector(cols);
+      Vector previousEigen = null;
+      while (hasNotConverged(currentEigen, corpus, state)) {
+        int randomStartingIndex = getRandomStartingIndex(corpus, eigens);
+        Vector initialTrainingVector = corpus.getRow(randomStartingIndex);
+        state.setTrainingIndex(randomStartingIndex);
+        updater.update(currentEigen, initialTrainingVector, state);
+        for (int corpusRow = 0; corpusRow < corpus.numRows(); corpusRow++) {
+          state.setTrainingIndex(corpusRow);
+          if (corpusRow != randomStartingIndex)
+            updater.update(currentEigen, corpus.getRow(corpusRow), state);
+        }
+        state.setFirstPass(false);
+        if (debug) {
+          if (previousEigen == null) {
+            previousEigen = currentEigen.clone();
+          } else {
+            double dot = currentEigen.dot(previousEigen);
+            if (dot > 0) dot /= (currentEigen.norm(2) * previousEigen.norm(2));
+           // log.info("Current pass * previous pass = " + dot);
+          }
+        }
+      }
+      // converged!
+      double eigenValue = state.getStatusProgress().get(state.getStatusProgress().size() - 1).getEigenValue();
+      // it's actually more efficient to do this to normalize than to call currentEigen = currentEigen.normalize(),
+      // because the latter does a clone, which isn't necessary here.
+      currentEigen.assign(new TimesFunction(), 1 / currentEigen.norm(2));
+      eigens.assignRow(i, currentEigen);
+      eigenValues.add(eigenValue);
+      state.setCurrentEigenValues(eigenValues);
+      log.info("Found eigenvector " + i + ", eigenvalue: " + eigenValue);
+
+      /**
+       *  TODO: Persist intermediate output!
+       */
+      state.setFirstPass(true);
+      state.setNumEigensProcessed(state.getNumEigensProcessed() + 1);
+      state.setActivationDenominatorSquared(0);
+      state.setActivationNumerator(0);
+      state.getStatusProgress().clear();
+      numPasses = 0;
+    }
+    return state;
+  }
+
+  /**
+   * You have to start somewhere...
+   * TODO: start instead wherever you find a vector with maximum residual length after subtracting off the projection
+   * TODO: onto all previous eigenvectors.
+   *
+   * @param corpus the corpus matrix
+   * @param eigens not currently used, but should be (see above TODO)
+   * @return the index into the corpus where the "starting seed" input vector lies.
+   */
+  private int getRandomStartingIndex(Matrix corpus, Matrix eigens) {
+    int index;
+    Vector v = null;
+    do {
+      double r = new Random(System.nanoTime()).nextDouble();
+      index = (int) (r * corpus.numRows());
+      v = corpus.getRow(index);
+    }
+    while (v == null || v.norm(2) == 0 || v.getNumNondefaultElements() < 5);
+    return index;
+  }
+
+  /**
+   * Uses the {@link SingularVectorVerifier } to check for convergence
+   *
+   * @param currentPseudoEigen the purported singular vector whose convergence is being checked
+   * @param corpus             the corpus to check against
+   * @param state              contains the previous eigens, various other solving state {@see TrainingState }
+   * @return true if <em>either</em> we have converged, <em>or</em> maxPassesPerEigen has been exceeded.
+   */
+  protected boolean hasNotConverged(Vector currentPseudoEigen,
+                                    Matrix corpus,
+                                    TrainingState state) {
+    numPasses++;
+    if (state.isFirstPass()) {
+      log.info("First pass through the corpus, no need to check convergence...");
+      return true;
+    }
+    Matrix previousEigens = state.getCurrentEigens();
+    log.info("Have made " + numPasses + " passes through the corpus, checking convergence...");
+    /*
+     * Step 1: orthogonalize currentPseudoEigen by subtracting off eigen(i) * helper.get(i)
+     * Step 2: zero-out the helper vector because it has already helped.
+     */
+    for (int i = 0; i < state.getNumEigensProcessed(); i++) {
+      Vector previousEigen = previousEigens.getRow(i);
+      currentPseudoEigen.assign(previousEigen, new PlusWithScaleFunction(-state.getHelperVector().get(i)));
+      state.getHelperVector().set(i, 0);
+    }
+    if (debug && currentPseudoEigen.norm(2) > 0) {
+      for (int i = 0; i < state.getNumEigensProcessed(); i++) {
+        Vector previousEigen = previousEigens.getRow(i);
+        log.info("dot with previous: " + (previousEigen.dot(currentPseudoEigen)) / currentPseudoEigen.norm(2));
+      }
+    }
+    /*
+     * Step 3: verify how eigen-like the prospective eigen is.  This is potentially asynchronous.
+     */
+    EigenStatus status = verify(corpus, currentPseudoEigen);
+    /**
+     *  TODO: Having the cosAngle() be zero is not a good signal for an unfinished verification.
+     */
+    if (status.getCosAngle() == 0) {
+      log.info("Verifier not finished, making another pass...");
+    } else {
+      log.info("Has 1 - cosAngle: " + (1 - status.getCosAngle()) + ", convergence target is: " + convergenceTarget);
+      state.getStatusProgress().add(status);
+    }
+    return (state.getStatusProgress().size() <= maxPassesPerEigen && 1 - status.getCosAngle() > convergenceTarget);
+  }
+
+  protected EigenStatus verify(Matrix corpus, Vector currentPseudoEigen) {
+    return verifier.verify(corpus, currentPseudoEigen);
+  }
+
+  public static void main(String args[]) {
+    Properties props = new Properties();
+    String propertiesFile = args.length > 0 ? args[0] : "config/solver.properties";
+    //  props.load(new FileInputStream(propertiesFile));
+
+    String corpusDir = props.getProperty("solver.input.dir");
+    String outputDir = props.getProperty("solver.output.dir");
+    if (corpusDir == null || corpusDir.equals("") || outputDir == null || outputDir.equals("")) {
+      log.error(propertiesFile + " must contain values for solver.input.dir and solver.output.dir");
+      System.exit(1);
+    }
+    int inBufferSize = Integer.parseInt(props.getProperty("solver.input.bufferSize"));
+    int rank = Integer.parseInt(props.getProperty("solver.output.desiredRank"));
+    double convergence = Double.parseDouble(props.getProperty("solver.convergence"));
+    int maxPasses = Integer.parseInt(props.getProperty("solver.maxPasses"));
+    int numThreads = Integer.parseInt(props.getProperty("solver.verifier.numThreads"));
+
+    HebbianUpdater updater = new HebbianUpdater();
+    SingularVectorVerifier verifier = new MultiThreadedEigenVerifier();
+    HebbianSolver solver = new HebbianSolver(updater,
+        verifier,
+        convergence,
+        maxPasses);
+    Matrix corpus = null;
+    if (numThreads <= 1) {
+      //  corpus = new DiskBufferedDoubleMatrix(new File(corpusDir), inBufferSize);
+    } else {
+      //  corpus = new ParallelMultiplyingDiskBufferedDoubleMatrix(new File(corpusDir), inBufferSize, numThreads);
+    }
+    long now = System.currentTimeMillis();
+    TrainingState finalState = solver.solve(corpus, rank);
+    long time = (long) ((System.currentTimeMillis() - now) / 1000);
+    log.info("Solved " + finalState.getCurrentEigens().size()[AbstractMatrix.ROW] + " eigenVectors in " + time + " seconds.  Persisted to " + outputDir);
+    System.exit(0);
+  }
+
+}

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/HebbianSolver.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/HebbianSolver.java
------------------------------------------------------------------------------
    svn:keywords = "Date Rev Author URL Id"

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/HebbianSolver.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/HebbianUpdater.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/HebbianUpdater.java?rev=901718&view=auto
==============================================================================
--- lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/HebbianUpdater.java (added)
+++ lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/HebbianUpdater.java Thu Jan 21 14:20:01 2010
@@ -0,0 +1,76 @@
+/**
+ * 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.math.decomposer.hebbian;
+
+
+import org.apache.mahout.math.PlusFunction;
+import org.apache.mahout.math.PlusWithScaleFunction;
+import org.apache.mahout.math.Vector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class HebbianUpdater implements EigenUpdater {
+
+  /**
+   * Logger for this class.
+   */
+  private static final Logger log = LoggerFactory.getLogger(HebbianUpdater.class);
+
+  public void update(Vector pseudoEigen,
+                     Vector trainingVector,
+                     TrainingState currentState) {
+    double trainingVectorNorm = trainingVector.norm(2);
+    int numPreviousEigens = currentState.getNumEigensProcessed();
+    if (numPreviousEigens > 0) {
+      if (currentState.isFirstPass()) {
+        updateTrainingProjectionsVector(currentState,
+            trainingVector,
+            numPreviousEigens - 1);
+      }
+    }
+    if (currentState.getActivationDenominatorSquared() == 0 || trainingVectorNorm == 0) {
+      if (currentState.getActivationDenominatorSquared() == 0) {
+        pseudoEigen.assign(trainingVector, new PlusFunction());
+        currentState.setHelperVector(currentState.currentTrainingProjection().clone());
+        double helperNorm = currentState.getHelperVector().norm(2);
+        currentState.setActivationDenominatorSquared(trainingVectorNorm * trainingVectorNorm - helperNorm * helperNorm);
+      }
+      return;
+    }
+    currentState.setActivationNumerator(pseudoEigen.dot(trainingVector));
+    currentState.setActivationNumerator(currentState.getActivationNumerator() - currentState.getHelperVector().dot(currentState.currentTrainingProjection()));
+
+    double activation = currentState.getActivationNumerator() / Math.sqrt(currentState.getActivationDenominatorSquared());
+    currentState.setActivationDenominatorSquared(currentState.getActivationDenominatorSquared() + 2 * activation * currentState.getActivationNumerator()
+        + (activation * activation) * (trainingVector.getLengthSquared() - currentState.currentTrainingProjection().getLengthSquared()));
+    if (numPreviousEigens > 0)
+      currentState.getHelperVector().assign(currentState.currentTrainingProjection(), new PlusWithScaleFunction(activation));
+    pseudoEigen.assign(trainingVector, new PlusWithScaleFunction(activation));
+  }
+
+  private void updateTrainingProjectionsVector(TrainingState state,
+                                               Vector trainingVector,
+                                               int previousEigenIndex) {
+    Vector previousEigen = state.mostRecentEigen();
+    Vector currentTrainingVectorProjection = state.currentTrainingProjection();
+    double projection = previousEigen.dot(trainingVector);
+    currentTrainingVectorProjection.set(previousEigenIndex, projection);
+  }
+
+}

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/HebbianUpdater.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/HebbianUpdater.java
------------------------------------------------------------------------------
    svn:keywords = "Date Rev Author URL Id"

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/HebbianUpdater.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/MultiThreadedEigenVerifier.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/MultiThreadedEigenVerifier.java?rev=901718&view=auto
==============================================================================
--- lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/MultiThreadedEigenVerifier.java (added)
+++ lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/MultiThreadedEigenVerifier.java Thu Jan 21 14:20:01 2010
@@ -0,0 +1,75 @@
+/**
+ * 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.math.decomposer.hebbian;
+
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import org.apache.mahout.math.Matrix;
+import org.apache.mahout.math.Vector;
+
+
+public class MultiThreadedEigenVerifier extends SimpleEigenVerifier {
+  protected final Executor threadPool;
+  protected EigenStatus status = null;
+  protected boolean finished = false;
+  protected boolean started = false;
+
+  public MultiThreadedEigenVerifier() {
+    threadPool = Executors.newFixedThreadPool(1);
+    status = new EigenStatus(-1, 0);
+  }
+
+  @Override
+  public EigenStatus verify(Matrix eigenMatrix, Vector vector) {
+    synchronized (status) {
+      if (!finished && !started) // not yet started or finished, so start!
+      {
+        status = new EigenStatus(-1, 0);
+        Vector vectorCopy = vector.clone();
+        threadPool.execute(new VerifierRunnable(eigenMatrix, vectorCopy));
+        started = true;
+      }
+      if (finished) finished = false;
+      return status;
+    }
+  }
+
+  protected EigenStatus innerVerify(Matrix eigenMatrix, Vector vector) {
+    return super.verify(eigenMatrix, vector);
+  }
+
+  protected class VerifierRunnable implements Runnable {
+    Matrix eigenMatrix;
+    Vector vector;
+
+    public VerifierRunnable(Matrix eigenMatrix, Vector vector) {
+      this.eigenMatrix = eigenMatrix;
+      this.vector = vector;
+    }
+
+    public void run() {
+      EigenStatus status = innerVerify(eigenMatrix, vector);
+      synchronized (MultiThreadedEigenVerifier.this.status) {
+        MultiThreadedEigenVerifier.this.status = status;
+        finished = true;
+        started = false;
+      }
+    }
+  }
+}

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/MultiThreadedEigenVerifier.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/MultiThreadedEigenVerifier.java
------------------------------------------------------------------------------
    svn:keywords = "Date Rev Author URL Id"

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/MultiThreadedEigenVerifier.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/SimpleEigenVerifier.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/SimpleEigenVerifier.java?rev=901718&view=auto
==============================================================================
--- lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/SimpleEigenVerifier.java (added)
+++ lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/SimpleEigenVerifier.java Thu Jan 21 14:20:01 2010
@@ -0,0 +1,41 @@
+/**
+ * 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.math.decomposer.hebbian;
+
+
+import org.apache.mahout.math.Matrix;
+import org.apache.mahout.math.Vector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class SimpleEigenVerifier implements SingularVectorVerifier {
+  /**
+   * Logger for this class.
+   */
+  private static final Logger log = LoggerFactory.getLogger(SimpleEigenVerifier.class);
+
+  public EigenStatus verify(Matrix eigenMatrix, Vector vector) {
+    Vector resultantVector = eigenMatrix.timesSquared(vector);
+    double newNorm = resultantVector.norm(2);
+    double oldNorm = vector.norm(2);
+    double eigenValue = (newNorm > 0 && oldNorm > 0) ? newNorm / oldNorm : 1;
+    double cosAngle = (newNorm > 0 && oldNorm > 0) ? resultantVector.dot(vector) / (newNorm * oldNorm) : 0;
+    return new EigenStatus(eigenValue, cosAngle);
+  }
+
+}

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/SimpleEigenVerifier.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/SimpleEigenVerifier.java
------------------------------------------------------------------------------
    svn:keywords = "Date Rev Author URL Id"

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/SimpleEigenVerifier.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/SingularVectorVerifier.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/SingularVectorVerifier.java?rev=901718&view=auto
==============================================================================
--- lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/SingularVectorVerifier.java (added)
+++ lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/SingularVectorVerifier.java Thu Jan 21 14:20:01 2010
@@ -0,0 +1,25 @@
+/**
+ * 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.math.decomposer.hebbian;
+
+import org.apache.mahout.math.Matrix;
+import org.apache.mahout.math.Vector;
+
+public interface SingularVectorVerifier {
+  EigenStatus verify(Matrix eigenMatrix, Vector vector);
+}

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/SingularVectorVerifier.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/SingularVectorVerifier.java
------------------------------------------------------------------------------
    svn:keywords = "Date Rev Author URL Id"

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/SingularVectorVerifier.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/TrainingState.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/TrainingState.java?rev=901718&view=auto
==============================================================================
--- lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/TrainingState.java (added)
+++ lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/TrainingState.java Thu Jan 21 14:20:01 2010
@@ -0,0 +1,141 @@
+/**
+ * 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.math.decomposer.hebbian;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.mahout.math.DenseVector;
+import org.apache.mahout.math.Matrix;
+import org.apache.mahout.math.Vector;
+
+
+public class TrainingState {
+  TrainingState(Matrix eigens, Matrix projections) {
+    setCurrentEigens(eigens);
+    setTrainingProjections(projections);
+    setTrainingIndex(0);
+    setHelperVector(new DenseVector("helper", eigens.numRows()));
+    setFirstPass(true);
+    setStatusProgress(new ArrayList<EigenStatus>());
+    setActivationNumerator(0);
+    setActivationDenominatorSquared(0);
+    setNumEigensProcessed(0);
+  }
+
+  private Matrix currentEigens;
+  private int numEigensProcessed;
+  private List<Double> currentEigenValues;
+  private Matrix trainingProjections;
+  private int trainingIndex;
+  private Vector helperVector;
+  private boolean firstPass;
+  private List<EigenStatus> statusProgress;
+  private double activationNumerator;
+  private double activationDenominatorSquared;
+
+  public Vector mostRecentEigen() {
+    return getCurrentEigens().getRow(getNumEigensProcessed() - 1);
+  }
+
+  public Vector currentTrainingProjection() {
+    if (getTrainingProjections().getRow(getTrainingIndex()) == null)
+      getTrainingProjections().assignRow(getTrainingIndex(), new DenseVector(getCurrentEigens().numCols()));
+    return getTrainingProjections().getRow(getTrainingIndex());
+  }
+
+  public Matrix getCurrentEigens() {
+    return currentEigens;
+  }
+
+  public void setCurrentEigens(Matrix currentEigens) {
+    this.currentEigens = currentEigens;
+  }
+
+  public int getNumEigensProcessed() {
+    return numEigensProcessed;
+  }
+
+  public void setNumEigensProcessed(int numEigensProcessed) {
+    this.numEigensProcessed = numEigensProcessed;
+  }
+
+  public List<Double> getCurrentEigenValues() {
+    return currentEigenValues;
+  }
+
+  public void setCurrentEigenValues(List<Double> currentEigenValues) {
+    this.currentEigenValues = currentEigenValues;
+  }
+
+  public Matrix getTrainingProjections() {
+    return trainingProjections;
+  }
+
+  public void setTrainingProjections(Matrix trainingProjections) {
+    this.trainingProjections = trainingProjections;
+  }
+
+  public int getTrainingIndex() {
+    return trainingIndex;
+  }
+
+  public void setTrainingIndex(int trainingIndex) {
+    this.trainingIndex = trainingIndex;
+  }
+
+  public Vector getHelperVector() {
+    return helperVector;
+  }
+
+  public void setHelperVector(Vector helperVector) {
+    this.helperVector = helperVector;
+  }
+
+  public boolean isFirstPass() {
+    return firstPass;
+  }
+
+  public void setFirstPass(boolean firstPass) {
+    this.firstPass = firstPass;
+  }
+
+  public List<EigenStatus> getStatusProgress() {
+    return statusProgress;
+  }
+
+  public void setStatusProgress(List<EigenStatus> statusProgress) {
+    this.statusProgress = statusProgress;
+  }
+
+  public double getActivationNumerator() {
+    return activationNumerator;
+  }
+
+  public void setActivationNumerator(double activationNumerator) {
+    this.activationNumerator = activationNumerator;
+  }
+
+  public double getActivationDenominatorSquared() {
+    return activationDenominatorSquared;
+  }
+
+  public void setActivationDenominatorSquared(double activationDenominatorSquared) {
+    this.activationDenominatorSquared = activationDenominatorSquared;
+  }
+}

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/TrainingState.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/TrainingState.java
------------------------------------------------------------------------------
    svn:keywords = "Date Rev Author URL Id"

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/TrainingState.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/lanczos/LanczosSolver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/lanczos/LanczosSolver.java?rev=901718&view=auto
==============================================================================
--- lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/lanczos/LanczosSolver.java (added)
+++ lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/lanczos/LanczosSolver.java Thu Jan 21 14:20:01 2010
@@ -0,0 +1,207 @@
+/**
+ * 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.math.decomposer.lanczos;
+
+
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.mahout.math.DenseVector;
+import org.apache.mahout.math.Matrix;
+import org.apache.mahout.math.PlusWithScaleFunction;
+import org.apache.mahout.math.SparseRowMatrix;
+import org.apache.mahout.math.UnaryFunction;
+import org.apache.mahout.math.Vector;
+import org.apache.mahout.math.matrix.DoubleMatrix1D;
+import org.apache.mahout.math.matrix.DoubleMatrix2D;
+import org.apache.mahout.math.matrix.impl.DenseDoubleMatrix2D;
+import org.apache.mahout.math.matrix.linalg.EigenvalueDecomposition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <p>Simple implementation of the <a href="http://en.wikipedia.org/wiki/Lanczos_algorithm">Lanczos algorithm</a> for
+ * finding eigenvalues of a symmetric matrix, applied to non-symmetric matrices by applying Matrix.timesSquared(vector)
+ * as the "matrix-multiplication" method.</p>
+ * <p>
+ * To avoid floating point overflow problems which arise in power-methods like Lanczos, an initial pass is made
+ * through the input matrix to
+ * <ul>
+ *   <li>generate a good starting seed vector by summing all the rows of the input matrix, and</li>
+ *   <li>compute the trace(inputMatrix<sup>t</sup>*matrix)
+ * </ul>
+ * </p>
+ * <p>
+ * This latter value, being the sum of all of the singular values, is used to rescale the entire matrix, effectively
+ * forcing the largest singular value to be strictly less than one, and transforming floating point <em>overflow</em>
+ * problems into floating point <em>underflow</em> (ie, very small singular values will become invisible, as they
+ * will appear to be zero and the algorithm will terminate).
+ * </p>
+ * <p>This implementation uses {@link org.apache.mahout.math.matrix.linalg.EigenvalueDecomposition} to do the
+ * eigenvalue extraction from the small (desiredRank x desiredRank) tridiagonal matrix.  Numerical stability is
+ * achieved via brute-force: re-orthogonalization against all previous eigenvectors is computed after every pass.
+ * This can be made smarter if (when!) this proves to be a major bottleneck.  Of course, this step can be parallelized
+ * as well.
+ * </p>
+ * <p>TODO: The input corpus is only accessed in a semi-random-access fashion in the getInitialVector method, which
+ * if it were refactored to act as an iterator, this entire class would be parallelized by simply implementing
+ * Matrix.timesSquared(Vector) as a Map-Reduce call.  
+ */
+public class LanczosSolver {
+  /**
+   * Logger for this class.
+   */
+  private static final Logger log = LoggerFactory.getLogger(LanczosSolver.class);
+
+  public static double SAFE_MAX = 1e150;
+
+  private static final double NANOS_IN_MILLI = 1e6;
+
+  private double scaleFactor = 0;
+
+  private static final class Scale implements UnaryFunction {
+    private final double d;
+
+    public Scale(double d) {
+      this.d = d;
+    }
+
+    public double apply(double arg1) {
+      return arg1 * d;
+    }
+  }
+
+  public void solve(Matrix corpus,
+                    int desiredRank,
+                    Matrix eigenVectors,
+                    List<Double> eigenValues) {
+    log.info("Finding " + desiredRank + " singular vectors of matrix with " + corpus.numRows() + " rows, via Lanczos");
+    Vector currentVector = getInitialVector(corpus);
+    Vector previousVector = new DenseVector(currentVector.size());
+    Matrix basis = new SparseRowMatrix(new int[]{desiredRank, corpus.numCols()});
+    basis.assignRow(0, currentVector);
+    double alpha = 0;
+    double beta = 0;
+    DoubleMatrix2D triDiag = new DenseDoubleMatrix2D(desiredRank, desiredRank);
+    for (int i = 1; i < desiredRank; i++) {
+      startTime(TimingSection.ITERATE);
+      Vector nextVector = corpus.timesSquared(currentVector);
+      log.info(i + " passes through the corpus so far...");
+      nextVector.assign(new Scale(1 / scaleFactor));
+      nextVector.assign(previousVector, new PlusWithScaleFunction(-beta));
+      // now orthogonalize
+      alpha = currentVector.dot(nextVector);
+      nextVector.assign(currentVector, new PlusWithScaleFunction(-alpha));
+      endTime(TimingSection.ITERATE);
+      startTime(TimingSection.ORTHOGANLIZE);
+      orthoganalizeAgainstAllButLast(nextVector, basis);
+      endTime(TimingSection.ORTHOGANLIZE);
+      // and normalize
+      beta = nextVector.norm(2);
+      if (outOfRange(beta) || outOfRange(alpha)) {
+        log.warn("Lanczos parameters out of range: alpha = " + alpha + ", beta = " + beta + ".  Bailing out early!");
+        break;
+      }
+      final double b = beta;
+      nextVector.assign(new Scale(1 / b));
+      basis.assignRow(i, nextVector);
+      previousVector = currentVector;
+      currentVector = nextVector;
+      // save the projections and norms!
+      triDiag.set(i - 1, i - 1, alpha);
+      if (i < desiredRank - 1) {
+        triDiag.set(i - 1, i, beta);
+        triDiag.set(i, i - 1, beta);
+      }
+    }
+    startTime(TimingSection.TRIDIAG_DECOMP);
+
+    log.info("Lanczos iteration complete - now to diagonalize the tri-diagonal auxiliary matrix.");
+    // at this point, have tridiag all filled out, and basis is all filled out, and orthonormalized
+    EigenvalueDecomposition decomp = new EigenvalueDecomposition(triDiag);
+
+    DoubleMatrix2D eigenVects = decomp.getV();
+    DoubleMatrix1D eigenVals = decomp.getRealEigenvalues();
+    endTime(TimingSection.TRIDIAG_DECOMP);
+    startTime(TimingSection.FINAL_EIGEN_CREATE);
+
+    for (int i = 0; i < basis.numRows() - 1; i++) {
+      Vector realEigen = new DenseVector(corpus.numCols());
+      // the eigenvectors live as columns of V, in reverse order.  Weird but true.
+      DoubleMatrix1D ejCol = eigenVects.viewColumn(basis.numRows() - i - 1);
+      for (int j = 0; j < ejCol.size(); j++) {
+        double d = ejCol.getQuick(j);
+        realEigen.assign(basis.getRow(j), new PlusWithScaleFunction(d));
+      }
+      realEigen = realEigen.normalize();
+      eigenVectors.assignRow(i, realEigen);
+      log.info("Eigenvector " + i + " found with eigenvalue " + eigenVals.get(i));
+      eigenValues.add(eigenVals.get(i));
+    }
+    log.info("LanczosSolver finished.");
+    endTime(TimingSection.FINAL_EIGEN_CREATE);
+  }
+
+  private static boolean outOfRange(double d) {
+    return Double.isNaN(d) || d > SAFE_MAX || -d > SAFE_MAX;
+  }
+
+  private void orthoganalizeAgainstAllButLast(Vector nextVector, Matrix basis) {
+    for (int i = 0; i < basis.numRows() - 1; i++) {
+      double alpha = nextVector.dot(basis.getRow(i));
+      nextVector.assign(basis.getRow(i), new PlusWithScaleFunction(-alpha));
+    }
+  }
+
+  protected Vector getInitialVector(Matrix corpus) {
+    Vector v = null;
+    for (int i = 0; i < corpus.numRows(); i++) {
+      Vector vector = corpus.getRow(i);
+      if (vector == null || vector.getLengthSquared() == 0) continue;
+      scaleFactor += vector.dot(vector);
+      if (v == null) {
+        v = new DenseVector(vector.size()).plus(vector);
+      } else {
+        v.assign(vector, new PlusWithScaleFunction(1));
+      }
+    }
+    v.assign(new Scale(1d / v.norm(2)));
+    return v;
+  }
+
+  private void startTime(TimingSection section) {
+    startTimes.put(section, System.nanoTime());
+  }
+
+  private void endTime(TimingSection section) {
+    if (!times.containsKey(section)) times.put(section, 0L);
+    times.put(section, times.get(section) + (System.nanoTime() - startTimes.get(section)));
+  }
+
+  public double getTimeMillis(TimingSection section) {
+    return ((double) times.get(section)) / NANOS_IN_MILLI;
+  }
+
+  public static enum TimingSection {
+    ITERATE, ORTHOGANLIZE, TRIDIAG_DECOMP, FINAL_EIGEN_CREATE
+  }
+
+  private Map<TimingSection, Long> startTimes = new EnumMap<TimingSection, Long>(TimingSection.class);
+  private Map<TimingSection, Long> times = new EnumMap<TimingSection, Long>(TimingSection.class);
+}

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/lanczos/LanczosSolver.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/lanczos/LanczosSolver.java
------------------------------------------------------------------------------
    svn:keywords = "Date Rev Author URL Id"

Propchange: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/lanczos/LanczosSolver.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/SolverTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/SolverTest.java?rev=901718&view=auto
==============================================================================
--- lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/SolverTest.java (added)
+++ lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/SolverTest.java Thu Jan 21 14:20:01 2010
@@ -0,0 +1,104 @@
+/**
+ * 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.math.decomposer;
+
+import junit.framework.TestCase;
+import org.apache.mahout.math.Matrix;
+import org.apache.mahout.math.SequentialAccessSparseVector;
+import org.apache.mahout.math.SparseRowMatrix;
+import org.apache.mahout.math.Vector;
+
+import java.util.Random;
+
+
+public abstract class SolverTest extends TestCase {
+
+  public SolverTest(String name) {
+    super(name);
+  }
+
+  public static void assertOrthonormal(Matrix eigens) {
+    assertOrthonormal(eigens, 1e-6);
+  }
+
+  public static void assertOrthonormal(Matrix currentEigens, double errorMargin) {
+    for (int i = 0; i < currentEigens.numRows(); i++) {
+      Vector ei = currentEigens.getRow(i);
+      for (int j = 0; j <= i; j++) {
+        Vector ej = currentEigens.getRow(j);
+        if (ei.norm(2) == 0 || ej.norm(2) == 0) continue;
+        double dot = ei.dot(ej);
+        if (i == j) {
+          assertTrue("not norm 1 : " + dot + " (eigen #" + i + ")", (Math.abs(1 - dot) < errorMargin));
+        } else {
+          assertTrue("not orthogonal : " + dot + " (eigens " + i + ", " + j + ")", Math.abs(dot) < errorMargin);
+        }
+      }
+    }
+  }
+
+  public static void assertEigen(Matrix eigens, Matrix corpus, double errorMargin) {
+    for (int i = 0; i < eigens.numRows(); i++) {
+      Vector e = eigens.getRow(i);
+      if (e.norm(2) == 0) continue;
+      Vector afterMultiply = corpus.timesSquared(e);
+      double dot = afterMultiply.dot(e);
+      double error = 1 - dot / (afterMultiply.norm(2) * e.norm(2));
+      assertTrue("Error margin: " + error + " too high! (for eigen " + i + ")", Math.abs(error) < errorMargin);
+    }
+  }
+
+  /**
+   * Builds up a consistently random (same seed every time) sparse matrix, with sometimes
+   * repeated rows.
+   * @param numRows
+   * @param nonNullRows
+   * @param numCols
+   * @param entriesPerRow
+   * @param entryMean
+   * @return
+   */
+  public static Matrix randomSequentialAccessSparseMatrix(int numRows,
+                                                           int nonNullRows,
+                                                           int numCols,
+                                                           int entriesPerRow,
+                                                           double entryMean) {
+    SparseRowMatrix m = new SparseRowMatrix(new int[]{numRows, numCols});
+    double n = 0;
+    Random r = new Random(1234L);
+    for (int i = 0; i < nonNullRows; i++) {
+      SequentialAccessSparseVector v = new SequentialAccessSparseVector(numCols);
+      for (int j = 0; j < entriesPerRow; j++) {
+        int col = r.nextInt(numCols);
+        double val = r.nextGaussian();
+        v.set(col, val * entryMean);
+      }
+      int c = r.nextInt(numRows);
+      if (r.nextBoolean()) {
+        m.assignRow(c, v);
+      } else {
+        Vector other = m.getRow(r.nextInt(numRows));
+        if (other != null && other.getLengthSquared() > 0) {
+          m.assignRow(c, other.clone());
+        }
+      }
+      n += m.getRow(c).getLengthSquared();
+    }
+    return m;
+  }
+}

Propchange: lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/SolverTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/SolverTest.java
------------------------------------------------------------------------------
    svn:keywords = "Date Rev Author URL Id"

Propchange: lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/SolverTest.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/hebbian/TestHebbianSolver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/hebbian/TestHebbianSolver.java?rev=901718&view=auto
==============================================================================
--- lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/hebbian/TestHebbianSolver.java (added)
+++ lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/hebbian/TestHebbianSolver.java Thu Jan 21 14:20:01 2010
@@ -0,0 +1,214 @@
+/**
+ * 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.math.decomposer.hebbian;
+
+import org.apache.mahout.math.DenseMatrix;
+import org.apache.mahout.math.Matrix;
+import org.apache.mahout.math.Vector;
+
+import junit.framework.TestCase;
+import org.apache.mahout.math.decomposer.SolverTest;
+import org.apache.mahout.math.decomposer.lanczos.LanczosSolver;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This test is woefully inadequate, and also requires tons of memory, because it's part
+ * unit test, part performance test, and part comparison test (between the Hebbian and Lanczos
+ * approaches).
+ * TODO: make better.
+ */
+public class TestHebbianSolver extends SolverTest {
+
+  public TestHebbianSolver(String name) {
+    super(name);
+  }
+
+  public static long timeSolver(Matrix corpus,
+                                double convergence,
+                                int maxNumPasses,
+                                TrainingState state) throws Exception {
+    return timeSolver(corpus,
+                      convergence,
+                      maxNumPasses,
+                      10,
+                      state);
+  }
+
+  public static long timeSolver(Matrix corpus,
+                                double convergence,
+                                int maxNumPasses,
+                                int desiredRank,
+                                TrainingState state) throws Exception {
+    long time = 0;
+    HebbianUpdater updater = new HebbianUpdater();
+    SingularVectorVerifier verifier = new MultiThreadedEigenVerifier();
+    HebbianSolver solver;
+    solver = new HebbianSolver(updater,
+        verifier,
+        convergence,
+        maxNumPasses);
+    long start = System.nanoTime();
+    TrainingState finalState = solver.solve(corpus, desiredRank);
+    assertNotNull(finalState);
+    state.setCurrentEigens(finalState.getCurrentEigens());
+    state.setCurrentEigenValues(finalState.getCurrentEigenValues());
+    time += System.nanoTime() - start;
+    assertEquals(state.getCurrentEigens().numRows(), desiredRank);
+    return (long) (time / 1e6);
+  }
+
+
+
+  public static long timeSolver(Matrix corpus, TrainingState state) throws Exception {
+    return timeSolver(corpus, state, 10);
+  }
+
+  public static long timeSolver(Matrix corpus, TrainingState state, int rank) throws Exception {
+    return timeSolver(corpus, 0.01, 20, rank, state);
+  }
+
+  public void testHebbianSolver() throws Exception {
+    int numColumns = 800;
+    Matrix corpus = randomSequentialAccessSparseMatrix(1000, 900, numColumns, 30, 1.0);
+    int rank = 50;
+    Matrix eigens = new DenseMatrix(rank, numColumns);
+    TrainingState state = new TrainingState(eigens, null);
+    long optimizedTime = timeSolver(corpus,
+                                    0.00001,
+                                    5,
+                                    rank,
+                                    state);
+    eigens = state.getCurrentEigens();
+    assertEigen(eigens, corpus, 0.05);
+    assertOrthonormal(eigens, 1e-6);
+    System.out.println("Avg solving (Hebbian) time in ms: " + optimizedTime);
+  }
+
+  /*
+  public void testSolverWithSerialization() throws Exception
+  {
+    _corpusProjectionsVectorFactory = new DenseMapVectorFactory();
+    _eigensVectorFactory = new DenseMapVectorFactory();
+    
+    timeSolver(TMP_EIGEN_DIR,
+               0.001, 
+               5, 
+               new TrainingState(null, null));
+    
+    File eigenDir = new File(TMP_EIGEN_DIR + File.separator + HebbianSolver.EIGEN_VECT_DIR);
+    DiskBufferedDoubleMatrix eigens = new DiskBufferedDoubleMatrix(eigenDir, 10);
+    
+    DoubleMatrix inMemoryMatrix = new HashMapDoubleMatrix(_corpusProjectionsVectorFactory, eigens);
+    
+    for(Entry<Integer, MapVector> diskEntry : eigens)
+    {
+      for(Entry<Integer, MapVector> inMemoryEntry : inMemoryMatrix)
+      {
+        if(diskEntry.getKey() - inMemoryEntry.getKey() == 0)
+        {
+          assertTrue("vector with index : " + diskEntry.getKey() + " is not the same on disk as in memory", 
+                     Math.abs(1 - diskEntry.getValue().dot(inMemoryEntry.getValue())) < 1e-6);
+        }
+        else
+        {
+          assertTrue("vector with index : " + diskEntry.getKey() 
+                     + " is not orthogonal to memory vect with index : " + inMemoryEntry.getKey(),
+                     Math.abs(diskEntry.getValue().dot(inMemoryEntry.getValue())) < 1e-6);
+        }
+      }
+    }
+    File corpusDir = new File(TMP_EIGEN_DIR + File.separator + "corpus");
+    corpusDir.mkdir();
+    // TODO: persist to disk?
+   // DiskBufferedDoubleMatrix.persistChunk(corpusDir, corpus, true);
+   // eigens.delete();
+    
+   // DiskBufferedDoubleMatrix.delete(new File(TMP_EIGEN_DIR));
+  }
+  */
+/*
+  public void testHebbianVersusLanczos() throws Exception
+  {
+    _corpusProjectionsVectorFactory = new DenseMapVectorFactory();
+    _eigensVectorFactory = new DenseMapVectorFactory();
+    int desiredRank = 200;
+    long time = timeSolver(TMP_EIGEN_DIR,
+                           0.00001, 
+                           5, 
+                           desiredRank,
+                           new TrainingState());
+
+    System.out.println("Hebbian time: " + time + "ms");
+    File eigenDir = new File(TMP_EIGEN_DIR + File.separator + HebbianSolver.EIGEN_VECT_DIR);
+    DiskBufferedDoubleMatrix eigens = new DiskBufferedDoubleMatrix(eigenDir, 10);
+    
+    DoubleMatrix2D srm = asSparseDoubleMatrix2D(corpus);
+    long timeA = System.nanoTime();
+    EigenvalueDecomposition asSparseRealDecomp = new EigenvalueDecomposition(srm);
+    for(int i=0; i<desiredRank; i++)
+      asSparseRealDecomp.getEigenvector(i);
+    System.out.println("CommonsMath time: " + (System.nanoTime() - timeA)/TimingConstants.NANOS_IN_MILLI + "ms");
+    
+   // System.out.println("Hebbian results:");
+   // printEigenVerify(eigens, corpus);
+    
+    DoubleMatrix lanczosEigenVectors = new HashMapDoubleMatrix(new HashMapVectorFactory());
+    List<Double> lanczosEigenValues = new ArrayList<Double>();
+ 
+    LanczosSolver solver = new LanczosSolver();
+    solver.solve(corpus, desiredRank*5, lanczosEigenVectors, lanczosEigenValues);
+    
+    for(TimingSection section : LanczosSolver.TimingSection.values())
+    {
+      System.out.println("Lanczos " + section.toString() + " = " + (int)(solver.getTimeMillis(section)/1000) + " seconds");
+    }
+    
+   // System.out.println("\nLanczos results:");
+   // printEigenVerify(lanczosEigenVectors, corpus);
+  }
+  
+  private DoubleMatrix2D asSparseDoubleMatrix2D(Matrix corpus)
+  {
+    DoubleMatrix2D result = new DenseDoubleMatrix2D(corpus.numRows(), corpus.numRows());
+    for(int i=0; i<corpus.numRows(); i++) {
+      for(int j=i; j<corpus.numRows(); j++) {
+        double v = corpus.getRow(i).dot(corpus.getRow(j));
+        result.set(i, j, v);
+        result.set(j, i, v);
+      }
+    }
+    return result;
+  }
+
+
+  public static void printEigenVerify(DoubleMatrix eigens, DoubleMatrix corpus)
+  {
+    for(Map.Entry<Integer, MapVector> entry : eigens)
+    {
+      MapVector eigen = entry.getValue();
+      MapVector afterMultiply = corpus.timesSquared(eigen);
+      double norm = afterMultiply.norm();
+      double error = 1 - eigen.dot(afterMultiply) / (eigen.norm() * afterMultiply.norm());
+      System.out.println(entry.getKey() + ": error = " + error + ", eVal = " + (norm / eigen.norm()));
+    }
+  }
+    */
+
+}

Propchange: lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/hebbian/TestHebbianSolver.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/hebbian/TestHebbianSolver.java
------------------------------------------------------------------------------
    svn:keywords = "Date Rev Author URL Id"

Propchange: lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/hebbian/TestHebbianSolver.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/lanczos/TestLanczosSolver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/lanczos/TestLanczosSolver.java?rev=901718&view=auto
==============================================================================
--- lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/lanczos/TestLanczosSolver.java (added)
+++ lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/lanczos/TestLanczosSolver.java Thu Jan 21 14:20:01 2010
@@ -0,0 +1,55 @@
+/**
+ * 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.math.decomposer.lanczos;
+
+import org.apache.mahout.math.DenseMatrix;
+import org.apache.mahout.math.Matrix;
+import org.apache.mahout.math.decomposer.SolverTest;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class TestLanczosSolver extends SolverTest {
+
+  public TestLanczosSolver(String name) {
+    super(name);
+  }
+
+  public void testLanczosSolver() throws Exception {
+    int numColumns = 800;
+    Matrix corpus = randomSequentialAccessSparseMatrix(1000, 900, numColumns, 30, 1.0);
+    int rank = 50;
+    Matrix eigens = new DenseMatrix(rank, numColumns);
+    long time = timeLanczos(corpus, eigens, rank);
+    assertTrue("Lanczos taking too long!  Are you in the debugger? :)", time < 10000);
+    assertOrthonormal(eigens);
+    assertEigen(eigens, corpus, 0.1);
+  }
+
+  public static long timeLanczos(Matrix corpus, Matrix eigens, int rank) throws Exception {
+    long start = System.currentTimeMillis();
+
+    LanczosSolver solver = new LanczosSolver();
+    List<Double> eVals = new ArrayList<Double>();
+    solver.solve(corpus, rank, eigens, eVals);
+    
+    long end = System.currentTimeMillis();
+    return end - start;
+  }
+
+}

Propchange: lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/lanczos/TestLanczosSolver.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/lanczos/TestLanczosSolver.java
------------------------------------------------------------------------------
    svn:keywords = "Date Rev Author URL Id"

Propchange: lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/decomposer/lanczos/TestLanczosSolver.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain