You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hama.apache.org by ed...@apache.org on 2010/06/07 12:14:48 UTC

svn commit: r952177 - in /incubator/hama/trunk/src: examples/org/apache/hama/examples/ java/org/apache/hama/matrix/ java/org/apache/hama/matrix/algebra/ test/org/apache/hama/matrix/

Author: edwardyoon
Date: Mon Jun  7 10:14:47 2010
New Revision: 952177

URL: http://svn.apache.org/viewvc?rev=952177&view=rev
Log:
Refactor matrix package

Removed:
    incubator/hama/trunk/src/examples/org/apache/hama/examples/MatrixAddition.java
    incubator/hama/trunk/src/java/org/apache/hama/matrix/algebra/MatrixAdditionMap.java
    incubator/hama/trunk/src/java/org/apache/hama/matrix/algebra/MatrixAdditionReduce.java
    incubator/hama/trunk/src/java/org/apache/hama/matrix/algebra/SparseMatrixVectorMultMap.java
    incubator/hama/trunk/src/java/org/apache/hama/matrix/algebra/SparseMatrixVectorMultReduce.java
    incubator/hama/trunk/src/java/org/apache/hama/matrix/algebra/TransposeMap.java
    incubator/hama/trunk/src/java/org/apache/hama/matrix/algebra/TransposeReduce.java
    incubator/hama/trunk/src/java/org/apache/hama/matrix/algebra/package.html
Modified:
    incubator/hama/trunk/src/examples/org/apache/hama/examples/ExampleDriver.java
    incubator/hama/trunk/src/examples/org/apache/hama/examples/MatrixMultiplication.java
    incubator/hama/trunk/src/java/org/apache/hama/matrix/AbstractMatrix.java
    incubator/hama/trunk/src/java/org/apache/hama/matrix/DenseMatrix.java
    incubator/hama/trunk/src/java/org/apache/hama/matrix/Matrix.java
    incubator/hama/trunk/src/java/org/apache/hama/matrix/SparseMatrix.java
    incubator/hama/trunk/src/test/org/apache/hama/matrix/TestAbstractMatrix.java
    incubator/hama/trunk/src/test/org/apache/hama/matrix/TestDenseMatrix.java
    incubator/hama/trunk/src/test/org/apache/hama/matrix/TestSparseMatrix.java

Modified: incubator/hama/trunk/src/examples/org/apache/hama/examples/ExampleDriver.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/examples/org/apache/hama/examples/ExampleDriver.java?rev=952177&r1=952176&r2=952177&view=diff
==============================================================================
--- incubator/hama/trunk/src/examples/org/apache/hama/examples/ExampleDriver.java (original)
+++ incubator/hama/trunk/src/examples/org/apache/hama/examples/ExampleDriver.java Mon Jun  7 10:14:47 2010
@@ -27,7 +27,6 @@ public class ExampleDriver {
     ProgramDriver pgd = new ProgramDriver();
     try {
       pgd.addClass("rand", RandomMatrix.class, "Generate matrix with random elements.");
-      pgd.addClass("add", MatrixAddition.class, "Mat-Mat Addition.");
       pgd.addClass("mult", MatrixMultiplication.class, "Mat-Mat Multiplication.");
       pgd.addClass("similarity", CosineSimilarityMatrix.class, "Cosine Similarity Matrix.");
       pgd.addClass("norms", MatrixNorm.class, "Matrix Norms.");

Modified: incubator/hama/trunk/src/examples/org/apache/hama/examples/MatrixMultiplication.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/examples/org/apache/hama/examples/MatrixMultiplication.java?rev=952177&r1=952176&r2=952177&view=diff
==============================================================================
--- incubator/hama/trunk/src/examples/org/apache/hama/examples/MatrixMultiplication.java (original)
+++ incubator/hama/trunk/src/examples/org/apache/hama/examples/MatrixMultiplication.java Mon Jun  7 10:14:47 2010
@@ -37,11 +37,14 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hama.Constants;
 import org.apache.hama.HamaAdmin;
 import org.apache.hama.HamaAdminImpl;
-import org.apache.hama.examples.mapreduce.*;
+import org.apache.hama.examples.mapreduce.BlockMultMap;
+import org.apache.hama.examples.mapreduce.BlockMultReduce;
+import org.apache.hama.examples.mapreduce.CollectBlocksMapper;
+import org.apache.hama.examples.mapreduce.DenseMatrixVectorMultMap;
+import org.apache.hama.examples.mapreduce.DenseMatrixVectorMultReduce;
 import org.apache.hama.io.BlockID;
 import org.apache.hama.matrix.DenseMatrix;
 import org.apache.hama.matrix.Matrix;
-import org.apache.hama.matrix.SparseMatrix;
 import org.apache.hama.util.RandomVariable;
 
 public class MatrixMultiplication extends AbstractExample {
@@ -66,13 +69,11 @@ public class MatrixMultiplication extend
       System.exit(-1);
     }
 
-    Matrix c;
+    Matrix c = null;
     if (a.getType().equals("SparseMatrix")) {
-      if (ARGS.size() > 2) {
         System.out
-            .println("NOTE: You can't use the block algorithm for sparse matrix multiplication.");
-      }
-      c = ((SparseMatrix) a).mult(b);
+            .println("NOTE: Not implemented M/R based sparse matrix multiplication.");
+        System.exit(-1);
     } else {
       if (ARGS.size() > 2) {
         c = mult(a, b, Integer.parseInt(ARGS.get(2)));

Modified: incubator/hama/trunk/src/java/org/apache/hama/matrix/AbstractMatrix.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/java/org/apache/hama/matrix/AbstractMatrix.java?rev=952177&r1=952176&r2=952177&view=diff
==============================================================================
--- incubator/hama/trunk/src/java/org/apache/hama/matrix/AbstractMatrix.java (original)
+++ incubator/hama/trunk/src/java/org/apache/hama/matrix/AbstractMatrix.java Mon Jun  7 10:14:47 2010
@@ -40,15 +40,11 @@ import org.apache.hadoop.hbase.client.Sc
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.mapreduce.TableMapper;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.MapWritable;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hama.Constants;
 import org.apache.hama.HamaAdmin;
 import org.apache.hama.HamaAdminImpl;
 import org.apache.hama.HamaConfiguration;
-import org.apache.hama.matrix.algebra.TransposeMap;
-import org.apache.hama.matrix.algebra.TransposeReduce;
 import org.apache.hama.util.BytesUtil;
 import org.apache.hama.util.RandomVariable;
 import org.apache.log4j.Logger;
@@ -424,34 +420,6 @@ public abstract class AbstractMatrix imp
     closed = true;
   }
 
-  public Matrix transpose() throws IOException {
-    Matrix result;
-    if (this.getType().equals("SparseMatrix")) {
-      result = new SparseMatrix(config, this.getRows(), this.getColumns());
-    } else {
-      result = new DenseMatrix(config, this.getRows(), this.getColumns());
-    }
-
-    Job job = new Job(config, "set MR job : " + this.getPath());
-
-    Scan scan = new Scan();
-    scan.addFamily(Constants.COLUMNFAMILY);
-    org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil.initTableMapperJob(
-        this.getPath(), scan, TransposeMap.class, IntWritable.class,
-        MapWritable.class, job);
-    org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil.initTableReducerJob(
-        result.getPath(), TransposeReduce.class, job);
-    try {
-      job.waitForCompletion(true);
-    } catch (InterruptedException e) {
-      e.printStackTrace();
-    } catch (ClassNotFoundException e) {
-      e.printStackTrace();
-    }
-
-    return result;
-  }
-
   public boolean checkAllJobs(List<Job> jobId) throws IOException {
     Iterator<Job> it = jobId.iterator();
     boolean allTrue = true;

Modified: incubator/hama/trunk/src/java/org/apache/hama/matrix/DenseMatrix.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/java/org/apache/hama/matrix/DenseMatrix.java?rev=952177&r1=952176&r2=952177&view=diff
==============================================================================
--- incubator/hama/trunk/src/java/org/apache/hama/matrix/DenseMatrix.java (original)
+++ incubator/hama/trunk/src/java/org/apache/hama/matrix/DenseMatrix.java Mon Jun  7 10:14:47 2010
@@ -29,17 +29,13 @@ import org.apache.hadoop.hbase.client.Pu
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.MapWritable;
 import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.Job;
 import org.apache.hama.Constants;
 import org.apache.hama.HamaConfiguration;
-import org.apache.hama.matrix.algebra.MatrixAdditionMap;
-import org.apache.hama.matrix.algebra.MatrixAdditionReduce;
 import org.apache.hama.util.BytesUtil;
 import org.apache.hama.util.RandomVariable;
 
@@ -333,104 +329,6 @@ public class DenseMatrix extends Abstrac
   }
 
   /**
-   * C = alpha*B + A
-   * 
-   * @param alpha
-   * @param B
-   * @return C
-   * @throws IOException
-   */
-  public DenseMatrix add(double alpha, Matrix B) throws IOException {
-    ensureForAddition(B);
-
-    DenseMatrix result = new DenseMatrix(config, this.getRows(), this
-        .getColumns());
-    Job job = new Job(config, "addition MR job" + result.getPath());
-
-    Scan scan = new Scan();
-    scan.addFamily(Constants.COLUMNFAMILY);
-    job.getConfiguration().set(MatrixAdditionMap.MATRIX_SUMMANDS, B.getPath());
-    job.getConfiguration().set(MatrixAdditionMap.MATRIX_ALPHAS,
-        Double.toString(alpha));
-
-    TableMapReduceUtil.initTableMapperJob(this.getPath(), scan,
-        MatrixAdditionMap.class, IntWritable.class, MapWritable.class, job);
-    TableMapReduceUtil.initTableReducerJob(result.getPath(),
-        MatrixAdditionReduce.class, job);
-    try {
-      job.waitForCompletion(true);
-    } catch (InterruptedException e) {
-      e.printStackTrace();
-    } catch (ClassNotFoundException e) {
-      e.printStackTrace();
-    }
-
-    return result;
-  }
-
-  /**
-   * C = B + A
-   * 
-   * @param B
-   * @return C
-   * @throws IOException
-   */
-  public DenseMatrix add(Matrix B) throws IOException {
-    return add(1.0, B);
-  }
-
-  public DenseMatrix add(Matrix... matrices) throws IOException {
-    // ensure all the matrices are suitable for addition.
-    for (Matrix m : matrices) {
-      ensureForAddition(m);
-    }
-
-    DenseMatrix result = new DenseMatrix(config, this.getRows(), this
-        .getColumns());
-
-    StringBuilder summandList = new StringBuilder();
-    StringBuilder alphaList = new StringBuilder();
-    for (Matrix m : matrices) {
-      summandList.append(m.getPath());
-      summandList.append(",");
-      alphaList.append("1");
-      alphaList.append(",");
-    }
-    summandList.deleteCharAt(summandList.length() - 1);
-    alphaList.deleteCharAt(alphaList.length() - 1);
-
-    Job job = new Job(config, "addition MR job" + result.getPath());
-
-    Scan scan = new Scan();
-    scan.addFamily(Constants.COLUMNFAMILY);
-    job.getConfiguration().set(MatrixAdditionMap.MATRIX_SUMMANDS,
-        summandList.toString());
-    job.getConfiguration().set(MatrixAdditionMap.MATRIX_ALPHAS,
-        alphaList.toString());
-
-    TableMapReduceUtil.initTableMapperJob(this.getPath(), scan,
-        MatrixAdditionMap.class, IntWritable.class, MapWritable.class, job);
-    TableMapReduceUtil.initTableReducerJob(result.getPath(),
-        MatrixAdditionReduce.class, job);
-    try {
-      job.waitForCompletion(true);
-    } catch (InterruptedException e) {
-      e.printStackTrace();
-    } catch (ClassNotFoundException e) {
-      e.printStackTrace();
-    }
-
-    return result;
-  }
-
-  private void ensureForAddition(Matrix m) throws IOException {
-    if (getRows() != m.getRows() || getColumns() != m.getColumns()) {
-      throw new IOException(
-          "Matrices' rows and columns should be same while A+B.");
-    }
-  }
-
-  /**
    * C = alpha*A*B + C
    * 
    * @param alpha

Modified: incubator/hama/trunk/src/java/org/apache/hama/matrix/Matrix.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/java/org/apache/hama/matrix/Matrix.java?rev=952177&r1=952176&r2=952177&view=diff
==============================================================================
--- incubator/hama/trunk/src/java/org/apache/hama/matrix/Matrix.java (original)
+++ incubator/hama/trunk/src/java/org/apache/hama/matrix/Matrix.java Mon Jun  7 10:14:47 2010
@@ -177,25 +177,6 @@ public interface Matrix {
   public void add(int i, int j, double value) throws IOException;
 
   /**
-   * A = B + A
-   * 
-   * @param B
-   * @return A
-   * @throws IOException
-   */
-  public Matrix add(Matrix B) throws IOException;
-
-  /**
-   * A = alpha*B + A
-   * 
-   * @param alpha
-   * @param B
-   * @return A
-   * @throws IOException
-   */
-  public Matrix add(double alpha, Matrix B) throws IOException;
-
-  /**
    * C = alpha*A*B + C
    * 
    * @param alpha
@@ -222,14 +203,6 @@ public interface Matrix {
     /** Largest entry in absolute value.  */
     Maxvalue
   }
-
-  /**
-   * Transposes the matrix. In most cases, the matrix must be square
-   * for this to work.
-   * 
-   * @return the transposed matrix
-   */
-  public Matrix transpose() throws IOException;
   
   /**
    * Save to a table or file

Modified: incubator/hama/trunk/src/java/org/apache/hama/matrix/SparseMatrix.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/java/org/apache/hama/matrix/SparseMatrix.java?rev=952177&r1=952176&r2=952177&view=diff
==============================================================================
--- incubator/hama/trunk/src/java/org/apache/hama/matrix/SparseMatrix.java (original)
+++ incubator/hama/trunk/src/java/org/apache/hama/matrix/SparseMatrix.java Mon Jun  7 10:14:47 2010
@@ -34,13 +34,9 @@ import org.apache.hadoop.hbase.mapreduce
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.MapWritable;
 import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.Job;
 import org.apache.hama.Constants;
 import org.apache.hama.HamaConfiguration;
-import org.apache.hama.matrix.algebra.SparseMatrixVectorMultMap;
-import org.apache.hama.matrix.algebra.SparseMatrixVectorMultReduce;
 import org.apache.hama.util.BytesUtil;
 import org.apache.hama.util.RandomVariable;
 
@@ -107,18 +103,6 @@ public class SparseMatrix extends Abstra
   }
 
   @Override
-  public Matrix add(Matrix B) throws IOException {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  @Override
-  public Matrix add(double alpha, Matrix B) throws IOException {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  @Override
   public double get(int i, int j) throws IOException {
     if (this.getRows() < i || this.getColumns() < j)
       throw new ArrayIndexOutOfBoundsException(i + ", " + j);
@@ -168,56 +152,6 @@ public class SparseMatrix extends Abstra
     return this.getClass().getSimpleName();
   }
 
-  /**
-   * C = A*B using iterative method
-   * 
-   * @param B
-   * @return C
-   * @throws IOException
-   */
-  public SparseMatrix mult(Matrix B) throws IOException {
-    SparseMatrix result = new SparseMatrix(config, this.getRows(), this
-        .getColumns());
-
-    List<Job> jobId = new ArrayList<Job>();
-
-    for (int i = 0; i < this.getRows(); i++) {
-      Job job = new Job(config, "multiplication MR job : " + result.getPath()
-          + " " + i);
-
-      Scan scan = new Scan();
-      scan.addFamily(Constants.COLUMNFAMILY);
-      job.getConfiguration().set(SparseMatrixVectorMultMap.MATRIX_A,
-          this.getPath());
-      job.getConfiguration().setInt(SparseMatrixVectorMultMap.ITH_ROW, i);
-
-      TableMapReduceUtil.initTableMapperJob(B.getPath(), scan,
-          SparseMatrixVectorMultMap.class, IntWritable.class,
-          MapWritable.class, job);
-      TableMapReduceUtil.initTableReducerJob(result.getPath(),
-          SparseMatrixVectorMultReduce.class, job);
-      try {
-        job.waitForCompletion(false);
-        jobId.add(job);
-      } catch (InterruptedException e) {
-        e.printStackTrace();
-      } catch (ClassNotFoundException e) {
-        e.printStackTrace();
-      }
-    }
-
-    while (checkAllJobs(jobId) == false) {
-      try {
-        Thread.sleep(1000);
-      } catch (InterruptedException e) {
-        // TODO Auto-generated catch block
-        e.printStackTrace();
-      }
-    }
-
-    return result;
-  }
-
   @Override
   public Matrix multAdd(double alpha, Matrix B, Matrix C) throws IOException {
     // TODO Auto-generated method stub

Modified: incubator/hama/trunk/src/test/org/apache/hama/matrix/TestAbstractMatrix.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/test/org/apache/hama/matrix/TestAbstractMatrix.java?rev=952177&r1=952176&r2=952177&view=diff
==============================================================================
--- incubator/hama/trunk/src/test/org/apache/hama/matrix/TestAbstractMatrix.java (original)
+++ incubator/hama/trunk/src/test/org/apache/hama/matrix/TestAbstractMatrix.java Mon Jun  7 10:14:47 2010
@@ -33,20 +33,9 @@ public class TestAbstractMatrix extends 
   }
 
   public void testTransposeAndNorm() throws IOException {
-    testTrans(m1);
-    testTrans(m2);
     normTest(m1);
     normTest(m2);
   }
-  
-  public void testTrans(Matrix matrix) throws IOException {
-    Matrix trans1 = matrix.transpose();
-    for (int i = 0; i < trans1.getRows(); i++) {
-      for (int j = 0; j < trans1.getColumns(); j++) {
-        assertEquals(trans1.get(i, j), matrix.get(j, i));
-      }
-    }
-  }
 
   public void normTest(Matrix matrix) throws IOException {
     double norm1 = MatrixNorm.norm(matrix, Norm.One);

Modified: incubator/hama/trunk/src/test/org/apache/hama/matrix/TestDenseMatrix.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/test/org/apache/hama/matrix/TestDenseMatrix.java?rev=952177&r1=952176&r2=952177&view=diff
==============================================================================
--- incubator/hama/trunk/src/test/org/apache/hama/matrix/TestDenseMatrix.java (original)
+++ incubator/hama/trunk/src/test/org/apache/hama/matrix/TestDenseMatrix.java Mon Jun  7 10:14:47 2010
@@ -37,7 +37,6 @@ public class TestDenseMatrix extends Ham
   static final Logger LOG = Logger.getLogger(TestDenseMatrix.class);
   private int SIZE = 10;
   private Matrix m1;
-  private Matrix m2;
   private HamaConfiguration conf;
 
   /**
@@ -53,28 +52,13 @@ public class TestDenseMatrix extends Ham
     conf = getConf();
 
     m1 = DenseMatrix.random(conf, SIZE, SIZE);
-    m2 = DenseMatrix.random(conf, SIZE, SIZE);
   }
 
   public void testAddMult() throws IOException {
-
-    Matrix m3 = DenseMatrix.random(conf, SIZE, SIZE);
-    Matrix m4 = DenseMatrix.random(conf, SIZE - 2, SIZE - 2);
-    try {
-      m1.add(m4);
-      fail("Matrix-Addition should be failed while rows and columns aren't same.");
-    } catch (IOException e) {
-      LOG.info(e.toString());
-    }
-    
     double origin = m1.get(1, 1);
     m1.add(1, 1, 0.5);
     assertEquals(m1.get(1, 1), origin + 0.5);
     
-    matrixAdd(m1, m2);
-    multMatrixAdd(m1, m2, m3);
-    addAlphaMatrix(m1, m2);
-
     getRowColumnVector();
     setRowColumnVector();
 
@@ -82,53 +66,6 @@ public class TestDenseMatrix extends Ham
     setAlphaMatrix(m1);
   }
 
-  /**
-   * Test matrices addition
-   * 
-   * @throws IOException
-   */
-  public void matrixAdd(Matrix m1, Matrix m2) throws IOException {
-    Matrix result = m1.add(m2);
-
-    assertEquals(result.getRows(), SIZE);
-    assertEquals(result.getColumns(), SIZE);
-
-    for (int i = 0; i < SIZE; i++) {
-      for (int j = 0; j < SIZE; j++) {
-        assertEquals(result.get(i, j), m1.get(i, j) + m2.get(i, j));
-      }
-    }
-
-    Matrix subtract = result.add(-1.0, m2);
-
-    for (int i = 0; i < SIZE; i++) {
-      for (int j = 0; j < SIZE; j++) {
-        double gap = (subtract.get(i, j) - m1.get(i, j));
-        assertTrue(-0.00001 < gap && gap < 0.00001);
-      }
-    }
-  }
-
-  public void multMatrixAdd(Matrix m1, Matrix m2, Matrix m3) throws IOException {
-    Matrix result = ((DenseMatrix) m1).add(m2, m3);
-
-    assertEquals(result.getRows(), SIZE);
-    assertEquals(result.getColumns(), SIZE);
-
-    for (int i = 0; i < SIZE; i++) {
-      for (int j = 0; j < SIZE; j++) {
-        assertEquals(result.get(i, j), m1.get(i, j) + m2.get(i, j)
-            + m3.get(i, j));
-      }
-    }
-  }
-
-  public void addAlphaMatrix(Matrix m1, Matrix m2) throws IOException {
-    double value = m1.get(0, 0) + (m2.get(0, 0) * 0.1);
-    Matrix result = m1.add(0.1, m2);
-    assertEquals(value, result.get(0, 0));
-  }
-
   public void setMatrix(Matrix m1) throws IOException {
     Matrix a = new DenseMatrix(conf, m1.getRows(), m1.getColumns());
     a.set(m1);

Modified: incubator/hama/trunk/src/test/org/apache/hama/matrix/TestSparseMatrix.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/test/org/apache/hama/matrix/TestSparseMatrix.java?rev=952177&r1=952176&r2=952177&view=diff
==============================================================================
--- incubator/hama/trunk/src/test/org/apache/hama/matrix/TestSparseMatrix.java (original)
+++ incubator/hama/trunk/src/test/org/apache/hama/matrix/TestSparseMatrix.java Mon Jun  7 10:14:47 2010
@@ -29,7 +29,6 @@ public class TestSparseMatrix extends Ha
   static final Logger LOG = Logger.getLogger(TestSparseMatrix.class);
   private int SIZE = 10;
   private SparseMatrix m1;
-  private SparseMatrix m2;
 
   /**
    * @throws UnsupportedEncodingException
@@ -41,7 +40,6 @@ public class TestSparseMatrix extends Ha
   public void setUp() throws Exception {
     super.setUp();
     m1 = SparseMatrix.random(getConf(), SIZE, SIZE);
-    m2 = SparseMatrix.random(getConf(), SIZE, SIZE);
   }
 
   public void testMult() throws IOException {
@@ -50,9 +48,6 @@ public class TestSparseMatrix extends Ha
     m1.set(0, 0, -8);
     assertEquals(m1.get(0, 0), -8.0);
 
-    SparseMatrix result = m1.mult(m2);
-    verifyMultResult(m1, m2, result);
-
     SparseVector vector = new SparseVector();
     vector.set(0, 3);
     vector.set(1, -8);
@@ -75,32 +70,4 @@ public class TestSparseMatrix extends Ha
 
     assertTrue(appeared);
   }
-
-  /**
-   * Verifying multiplication result
-   * 
-   * @param m1
-   * @param m2
-   * @param result
-   * @throws IOException
-   */
-  private void verifyMultResult(SparseMatrix m1, SparseMatrix m2,
-      SparseMatrix result) throws IOException {
-    double[][] c = new double[SIZE][SIZE];
-
-    for (int i = 0; i < SIZE; i++) {
-      for (int j = 0; j < SIZE; j++) {
-        for (int k = 0; k < SIZE; k++) {
-          c[i][k] += m1.get(i, j) * m2.get(j, k);
-        }
-      }
-    }
-
-    for (int i = 0; i < SIZE; i++) {
-      for (int j = 0; j < SIZE; j++) {
-        double gap = (c[i][j] - result.get(i, j));
-        assertTrue(gap < 0.000001 && gap > -0.000001);
-      }
-    }
-  }
 }