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 2008/08/11 07:56:24 UTC

svn commit: r684641 - in /incubator/hama/trunk/src: java/org/apache/hama/ java/org/apache/hama/algebra/ test/org/apache/hama/ test/org/apache/hama/mapred/

Author: edwardyoon
Date: Sun Aug 10 22:56:23 2008
New Revision: 684641

URL: http://svn.apache.org/viewvc?rev=684641&view=rev
Log:
Matrix interface re-arrangement

Modified:
    incubator/hama/trunk/src/java/org/apache/hama/AbstractMatrix.java
    incubator/hama/trunk/src/java/org/apache/hama/Matrix.java
    incubator/hama/trunk/src/java/org/apache/hama/MatrixInterface.java
    incubator/hama/trunk/src/java/org/apache/hama/Vector.java
    incubator/hama/trunk/src/java/org/apache/hama/algebra/AdditionMap.java
    incubator/hama/trunk/src/test/org/apache/hama/TestMatrix.java
    incubator/hama/trunk/src/test/org/apache/hama/TestVector.java
    incubator/hama/trunk/src/test/org/apache/hama/mapred/TestMatrixMapReduce.java

Modified: incubator/hama/trunk/src/java/org/apache/hama/AbstractMatrix.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/java/org/apache/hama/AbstractMatrix.java?rev=684641&r1=684640&r2=684641&view=diff
==============================================================================
--- incubator/hama/trunk/src/java/org/apache/hama/AbstractMatrix.java (original)
+++ incubator/hama/trunk/src/java/org/apache/hama/AbstractMatrix.java Sun Aug 10 22:56:23 2008
@@ -30,7 +30,6 @@
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.io.BatchUpdate;
 import org.apache.hadoop.hbase.io.Cell;
-import org.apache.hadoop.hbase.io.RowResult;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
@@ -38,7 +37,8 @@
 /**
  * Methods of the matrix classes
  */
-public abstract class AbstractMatrix extends AbstractBase implements MatrixInterface {
+public abstract class AbstractMatrix extends AbstractBase implements
+    MatrixInterface {
   static final Logger LOG = Logger.getLogger(AbstractMatrix.class);
 
   /** Hbase Configuration */
@@ -77,8 +77,7 @@
    */
   protected void create() {
     try {
-      tableDesc.addFamily(new HColumnDescriptor(Constants.METADATA
-          .toString()));
+      tableDesc.addFamily(new HColumnDescriptor(Constants.METADATA.toString()));
       LOG.info("Initializaing.");
       admin.createTable(tableDesc);
     } catch (IOException e) {
@@ -87,30 +86,6 @@
   }
 
   /** {@inheritDoc} */
-  public int getRowDimension() {
-    Cell rows = null;
-    try {
-      rows = table.get(Constants.METADATA, Constants.METADATA_ROWS);
-    } catch (IOException e) {
-      LOG.error(e, e);
-    }
-
-    return Bytes.toInt(rows.getValue());
-  }
-
-  /** {@inheritDoc} */
-  public int getColumnDimension() {
-    Cell columns = null;
-    try {
-      columns = table.get(Constants.METADATA,
-          Constants.METADATA_COLUMNS);
-    } catch (IOException e) {
-      LOG.error(e, e);
-    }
-    return Bytes.toInt(columns.getValue());
-  }
-
-  /** {@inheritDoc} */
   public double get(int i, int j) {
     Text row = new Text(String.valueOf(i));
     Text column = new Text(Constants.COLUMN + String.valueOf(j));
@@ -128,18 +103,19 @@
   }
 
   /** {@inheritDoc} */
-  public RowResult getRowResult(byte[] row) {
+  public Vector getRow(int row) {
     try {
-      return table.getRow(row);
+      return new Vector(row, table.getRow(String.valueOf(row).getBytes()));
     } catch (IOException e) {
       e.printStackTrace();
     }
     return null;
   }
 
-  public RowResult getRowResult(int row) {
+  /** {@inheritDoc} */
+  public Vector getRow(byte[] row) {
     try {
-      return table.getRow(String.valueOf(row).getBytes());
+      return new Vector(bytesToInt(row), table.getRow(row));
     } catch (IOException e) {
       e.printStackTrace();
     }
@@ -147,33 +123,41 @@
   }
   
   /** {@inheritDoc} */
-  public void set(int i, int j, double d) {
-    BatchUpdate b = new BatchUpdate(new Text(String.valueOf(i)));
-    b.put(new Text(Constants.COLUMN + String.valueOf(j)), doubleToBytes(d));
+  public int getRows() {
+    Cell rows = null;
     try {
-      table.commit(b);
+      rows = table.get(Constants.METADATA, Constants.METADATA_ROWS);
     } catch (IOException e) {
       LOG.error(e, e);
     }
-  }
 
-  /** {@inheritDoc} */
-  public void add(int i, int j, double d) {
-    // TODO Auto-generated method stub
+    return Bytes.toInt(rows.getValue());
   }
 
   /** {@inheritDoc} */
-  public void deleteColumnEquals(int j) {
-    // TODO Auto-generated method stub
+  public int getColumns() {
+    Cell columns = null;
+    try {
+      columns = table.get(Constants.METADATA, Constants.METADATA_COLUMNS);
+    } catch (IOException e) {
+      LOG.error(e, e);
+    }
+    return Bytes.toInt(columns.getValue());
   }
 
   /** {@inheritDoc} */
-  public void deleteRowEquals(int i) {
-    // TODO Auto-generated method stub
+  public void set(int i, int j, double value) {
+    BatchUpdate b = new BatchUpdate(new Text(String.valueOf(i)));
+    b.put(new Text(Constants.COLUMN + String.valueOf(j)), doubleToBytes(value));
+    try {
+      table.commit(b);
+    } catch (IOException e) {
+      LOG.error(e, e);
+    }
   }
 
   /** {@inheritDoc} */
-  public void reset(int m, int n) {
+  public void add(int i, int j, double d) {
     // TODO Auto-generated method stub
   }
 
@@ -194,47 +178,4 @@
   public String getName() {
     return (matrixName != null) ? matrixName.toString() : null;
   }
-
-  /**
-   * Return the value of determinant
-   * 
-   * @return the value of determinant
-   */
-  public double getDeterminant() {
-    try {
-      return bytesToDouble(table.get(
-          new Text(String.valueOf(Constants.DETERMINANT)),
-          new Text(Constants.COLUMN)).getValue());
-    } catch (IOException e) {
-      LOG.error(e, e);
-      return -1;
-    }
-  }
-
-  /** {@inheritDoc} */
-  public Matrix copy() {
-    // TODO
-    return null;
-  }
-
-  /** {@inheritDoc} */
-  public void save(String matrixName) {
-    // TODO
-  }
-
-  /** {@inheritDoc} */
-  public void close() {
-    admin = null;
-    matrixName = null;
-    tableDesc = null;
-  }
-
-  /** {@inheritDoc} */
-  public void clear() {
-    try {
-      admin.deleteTable(matrixName);
-    } catch (IOException e) {
-      LOG.error(e, e);
-    }
-  }
 }

Modified: incubator/hama/trunk/src/java/org/apache/hama/Matrix.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/java/org/apache/hama/Matrix.java?rev=684641&r1=684640&r2=684641&view=diff
==============================================================================
--- incubator/hama/trunk/src/java/org/apache/hama/Matrix.java (original)
+++ incubator/hama/trunk/src/java/org/apache/hama/Matrix.java Sun Aug 10 22:56:23 2008
@@ -55,8 +55,7 @@
 
       if (!admin.tableExists(matrixName)) {
         tableDesc = new HTableDescriptor(matrixName.toString());
-        tableDesc.addFamily(new HColumnDescriptor(Constants.COLUMN
-            .toString()));
+        tableDesc.addFamily(new HColumnDescriptor(Constants.COLUMN.toString()));
         create();
       }
 
@@ -82,8 +81,7 @@
 
       if (!admin.tableExists(matrixName)) {
         tableDesc = new HTableDescriptor(matrixName.toString());
-        tableDesc.addFamily(new HColumnDescriptor(Constants.COLUMN
-            .toString()));
+        tableDesc.addFamily(new HColumnDescriptor(Constants.COLUMN.toString()));
         create();
       }
 
@@ -121,91 +119,40 @@
     return rand;
   }
 
-  /**
-   * Generate matrix with identity elements
-   * 
-   * @param conf configuration object
-   * @param m the number of rows.
-   * @param n the number of columns.
-   * @return an m-by-n matrix with ones on the diagonal and zeros elsewhere.
-   */
-  public static Matrix identity(HBaseConfiguration conf, int m, int n) {
-    // TODO
+  public Matrix add(Matrix B) {
+    
+    
+    // TODO Auto-generated method stub
     return null;
   }
 
-  /** {@inheritDoc} */
-  public Matrix multiply(Matrix b) {
-    /*
-     * String output = Constants.RESULT + System.currentTimeMillis(); Matrix c =
-     * new Matrix(config, new Text(output)); JobConf jobConf = new
-     * JobConf(config); jobConf.setJobName("parallel matrix multiplication of " +
-     * getName() + " and " + b.getName());
-     * jobConf.setInputFormat(MatrixInputFormat.class);
-     * jobConf.setOutputFormat(MatrixOutputFormat.class);
-     * MultiplicationMap.initJob(getName(), b.getName(),
-     * MultiplicationMap.class, jobConf); MultiplicationReduce.initJob(output,
-     * MultiplicationReduce.class, jobConf); jobConf.setNumMapTasks(mapper);
-     * jobConf.setNumReduceTasks(reducer); try { JobClient.runJob(jobConf); }
-     * catch (IOException e) { LOG.info(e); } return c;
-     */
+  public Matrix add(double alpha, Matrix B) {
+    // TODO Auto-generated method stub
     return null;
   }
 
-  /** {@inheritDoc} */
-  public Matrix addition(Matrix b) {
-    return additionSubtraction(b, Constants.PLUS);
-  }
-
-  /** {@inheritDoc} */
-  public Matrix subtraction(Matrix b) {
-    return additionSubtraction(b, Constants.PLUS);
+  public Matrix mult(Matrix B) {
+    // TODO Auto-generated method stub
+    return null;
   }
 
-  /**
-   * Method for add or subtract operation
-   * 
-   * @param target
-   * @param operator
-   * @return matrix
-   */
-  public Matrix additionSubtraction(Matrix target, String operator) {
-    /*
-     * String b = target.getName(); String output = Constants.RESULT +
-     * System.currentTimeMillis(); Matrix c = new Matrix(config, new
-     * Text(output)); String jobName = "parallel matrix " + operator + " of " +
-     * getName() + " and " + b; LOG.info(jobName); JobConf jobConf = new
-     * JobConf(config); jobConf.setJobName(jobName);
-     * jobConf.setInputFormat(MatrixInputFormat.class);
-     * jobConf.setOutputFormat(MatrixOutputFormat.class);
-     * AdditionSubtractionMap.initJob(getName(), b, operator,
-     * AdditionSubtractionMap.class, jobConf);
-     * AdditionSubtractionReduce.initJob(output,
-     * AdditionSubtractionReduce.class, jobConf);
-     * jobConf.setNumMapTasks(mapper); jobConf.setNumReduceTasks(reducer); try {
-     * JobClient.runJob(jobConf); } catch (IOException e) { LOG.info(e); }
-     * return c;
-     */
+  public Matrix multAdd(double alpha, Matrix B, Matrix C) {
+    // TODO Auto-generated method stub
     return null;
   }
 
-  /** {@inheritDoc} */
-  public double determinant() {
-    /*
-     * JobConf jobConf = new JobConf(config); jobConf.setJobName("matrix
-     * determinant"); String check = Constants.RESULT +
-     * System.currentTimeMillis(); Matrix c = new Matrix(config, new
-     * Text(check)); for (int i = 0; i < getRowDimension(); i++) { c.set(i, 0,
-     * 1.0); } c.setDimension(getRowDimension(), 0);
-     * jobConf.setInputFormat(MatrixInputFormat.class);
-     * jobConf.setOutputFormat(MatrixOutputFormat.class);
-     * DeterminantMap.initJob(getName(), check, DeterminantMap.class, jobConf);
-     * DeterminantReduce.initJob(getName(), DeterminantReduce.class, jobConf);
-     * jobConf.setNumMapTasks(mapper); jobConf.setNumReduceTasks(reducer); try {
-     * JobClient.runJob(jobConf); } catch (IOException e) { LOG.info(e); }
-     * c.clear(); return getDeterminant();
-     */
+  public double norm(Norm type) {
+    // TODO Auto-generated method stub
     return 0;
   }
 
+  public Matrix set(double alpha, Matrix B) {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  public Matrix set(Matrix B) {
+    // TODO Auto-generated method stub
+    return null;
+  }
 }

Modified: incubator/hama/trunk/src/java/org/apache/hama/MatrixInterface.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/java/org/apache/hama/MatrixInterface.java?rev=684641&r1=684640&r2=684641&view=diff
==============================================================================
--- incubator/hama/trunk/src/java/org/apache/hama/MatrixInterface.java (original)
+++ incubator/hama/trunk/src/java/org/apache/hama/MatrixInterface.java Sun Aug 10 22:56:23 2008
@@ -19,12 +19,8 @@
  */
 package org.apache.hama;
 
-import org.apache.hadoop.hbase.io.RowResult;
-
 /**
- * Basic matrix interface. It holds <code>double</code>s in a rectangular 2D
- * array, and it is used alongside <code>Vector</code> in numerical
- * computations. Implementing classes decides on the actual storage.
+ * Basic matrix interface.
  */
 public interface MatrixInterface {
 
@@ -43,53 +39,47 @@
    * @param row the row index of the matrix
    * @return the feature vector of row
    */
-  public RowResult getRowResult(byte[] row);
+  public Vector getRow(int row);
 
   /**
-   * Sets the double value of (i, j)
-   * 
-   * @param i ith row of the matrix
-   * @param j jth column of the matrix
-   * @param d the value of entry
-   */
-  public void set(int i, int j, double d);
-
-  /**
-   * Adds value to (i, j)
+   * Get a number of row of the matrix from the meta-data column
    * 
-   * @param i i th row of the matrix
-   * @param j j th column of the matrix
-   * @param d the value of entry
+   * @return a number of rows of the matrix
    */
-  public void add(int i, int j, double d);
+  public int getRows();
 
   /**
-   * Delete a Row to Matrix.
+   * Get a number of column of the matrix from the meta-data column
    * 
-   * @param i row number to delete
+   * @return a number of columns of the matrix
    */
-  public void deleteRowEquals(int i);
+  public int getColumns();
 
   /**
-   * Delete a Column to Matrix.
+   * Sets the double value of (i, j)
    * 
-   * @param j column number to delete
+   * @param i ith row of the matrix
+   * @param j jth column of the matrix
+   * @param value the value of entry
    */
-  public void deleteColumnEquals(int j);
+  public void set(int i, int j, double value);
 
   /**
-   * Get a number of row of the matrix from the meta-data column
+   * A=alpha*B
    * 
-   * @return a number of rows of the matrix
+   * @param alpha
+   * @param B
+   * @return A
    */
-  public int getRowDimension();
+  public Matrix set(double alpha, Matrix B);
 
   /**
-   * Get a number of column of the matrix from the meta-data column
+   * A=B
    * 
-   * @return a number of columns of the matrix
+   * @param B
+   * @return A
    */
-  public int getColumnDimension();
+  public Matrix set(Matrix B);
 
   /**
    * Sets the dimension of matrix
@@ -100,79 +90,68 @@
   public void setDimension(int rows, int columns);
 
   /**
-   * Modify dimensions of matrix
+   * A(i, j) += value
    * 
-   * @param m number of rows
-   * @param n number of columns
+   * @param i
+   * @param j
+   * @param value
    */
-  public void reset(int m, int n);
+  public void add(int i, int j, double value);
 
   /**
-   * Return the matrix name
+   * A = B + A
    * 
-   * @return the name of the matrix
+   * @param B
+   * @return
    */
-  public String getName();
+  public Matrix add(Matrix B);
 
   /**
-   * Make a deep copy of a matrix
+   * A = alpha*B + A
    * 
-   * @return clone matrix
+   * @param alpha
+   * @param B
+   * @return A
    */
-  public Matrix copy();
+  public Matrix add(double alpha, Matrix B);
 
   /**
-   * Multiply two matrices
+   * C = A*B
    * 
-   * @param b matrix b
-   * @return the result of the multiplication of matrix a and matrix b
+   * @param B
+   * @return C
    */
-  public Matrix multiply(Matrix b);
+  public Matrix mult(Matrix B);
 
   /**
-   * Add two matrices
+   * C = alpha*A*B + C
    * 
-   * @param b matrix b
-   * @return the result of the addition of matrix a and matrix b
+   * @param alpha
+   * @param B
+   * @param C
+   * @return C
    */
-  public Matrix addition(Matrix b);
+  public Matrix multAdd(double alpha, Matrix B, Matrix C);
 
   /**
-   * Subtract two matrices
+   * Computes the given norm of the matrix
    * 
-   * @param b matrix b
-   * @return the result of the substraction of matrix a and matrix b
+   * @param type
+   * @return norm of the matrix
    */
-  public Matrix subtraction(Matrix b);
+  public double norm(Norm type);
 
   /**
-   * Calculates determinant of a matrix
-   * 
-   * @return the value of determinant
-   */
-  public double determinant();
-
-  /**
-   * Save the matrix to table
-   * 
-   * @param matrixName the name of the matrix
+   * Supported matrix-norms.
    */
-  public void save(String matrixName);
+  enum Norm {
+    // TODO
+  }
 
   /**
-   * Decomposition
+   * Return the matrix name
    * 
-   * @return the decomposed result
-   */
-  // public TriangularMatrix decompose(Decomposition technique);
-  /**
-   * Clear object
-   */
-  public void clear();
-
-  /**
-   * Close object
+   * @return the name of the matrix
    */
-  public void close();
-
+  public String getName();
 }

Modified: incubator/hama/trunk/src/java/org/apache/hama/Vector.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/java/org/apache/hama/Vector.java?rev=684641&r1=684640&r2=684641&view=diff
==============================================================================
--- incubator/hama/trunk/src/java/org/apache/hama/Vector.java (original)
+++ incubator/hama/trunk/src/java/org/apache/hama/Vector.java Sun Aug 10 22:56:23 2008
@@ -21,6 +21,7 @@
 
 import org.apache.hadoop.hbase.io.Cell;
 import org.apache.hadoop.hbase.io.HbaseMapWritable;
+import org.apache.hadoop.hbase.io.RowResult;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hama.io.VectorWritable;
 import org.apache.log4j.Logger;
@@ -37,15 +38,11 @@
     this.cells = m;
   }
 
-  /**
-   * @param row
-   * @param matrix
-   */
-  public Vector(byte[] row, Matrix matrix) {
-    this.row = row;
-    parse(matrix.getRowResult(this.row).entrySet());
+  public Vector(int row, RowResult rowResult) {
+    this.row = intToBytes(row);
+    parse(rowResult.entrySet());
   }
-
+  
   public void add(int index, double value) {
     // TODO Auto-generated method stub
 

Modified: incubator/hama/trunk/src/java/org/apache/hama/algebra/AdditionMap.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/java/org/apache/hama/algebra/AdditionMap.java?rev=684641&r1=684640&r2=684641&view=diff
==============================================================================
--- incubator/hama/trunk/src/java/org/apache/hama/algebra/AdditionMap.java (original)
+++ incubator/hama/trunk/src/java/org/apache/hama/algebra/AdditionMap.java Sun Aug 10 22:56:23 2008
@@ -13,8 +13,8 @@
   public void map(ImmutableBytesWritable key, Vector value,
       OutputCollector<ImmutableBytesWritable, Vector> output,
       Reporter reporter) throws IOException {
-
-    Vector v1 = new Vector(key.get(), B);
+    
+    Vector v1 = B.getRow(key.get());
     output.collect(key, v1.add(value));
   }
 

Modified: incubator/hama/trunk/src/test/org/apache/hama/TestMatrix.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/test/org/apache/hama/TestMatrix.java?rev=684641&r1=684640&r2=684641&view=diff
==============================================================================
--- incubator/hama/trunk/src/test/org/apache/hama/TestMatrix.java (original)
+++ incubator/hama/trunk/src/test/org/apache/hama/TestMatrix.java Sun Aug 10 22:56:23 2008
@@ -19,19 +19,6 @@
  */
 package org.apache.hama;
 
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hama.algebra.AdditionMap;
-import org.apache.hama.algebra.AdditionReduce;
-import org.apache.hama.io.VectorWritable;
-import org.apache.hama.mapred.MatrixMap;
-import org.apache.hama.mapred.MatrixReduce;
-import org.apache.hama.mapred.TestMatrixMapReduce;
-
 /**
  * Matrix test
  */
@@ -42,7 +29,6 @@
    */
   public void testRandomMatrix() {
     Matrix rand = Matrix.random(conf, SIZE, SIZE);
-    assertTrue(rand.getRowDimension() == SIZE);
-    rand.close();
+    assertTrue(rand.getRows() == SIZE);
   }
 }

Modified: incubator/hama/trunk/src/test/org/apache/hama/TestVector.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/test/org/apache/hama/TestVector.java?rev=684641&r1=684640&r2=684641&view=diff
==============================================================================
--- incubator/hama/trunk/src/test/org/apache/hama/TestVector.java (original)
+++ incubator/hama/trunk/src/test/org/apache/hama/TestVector.java Sun Aug 10 22:56:23 2008
@@ -19,7 +19,6 @@
  */
 package org.apache.hama;
 
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.Text;
 
 public class TestVector extends HamaTestCase {
@@ -31,10 +30,6 @@
     final double result = 0.6978227007909176;
     Matrix m1 = new Matrix(conf, new Text("cosine"));
 
-    // TODO : We need setArray(int row, double[] value) to matrix
-    // e.g. matrixA.setArray(0, new double[] {2,5,1,4});
-    // -- Edward
-
     m1.set(0, 0, 2);
     m1.set(0, 1, 5);
     m1.set(0, 2, 1);
@@ -48,11 +43,10 @@
     LOG.info("get test : " + m1.get(0, 0));
     LOG.info("get test : " + m1.get(0, 1));
 
-    Vector v1 = new Vector(Bytes.toBytes(String.valueOf(0)), m1);
-    Vector v2 = new Vector(Bytes.toBytes(String.valueOf(1)), m1);
+    Vector v1 = m1.getRow(0);
+    Vector v2 = m1.getRow(1);
 
     double cos = v1.dot(v2);
     assertEquals(cos, result);
-    m1.close();
   }
 }

Modified: incubator/hama/trunk/src/test/org/apache/hama/mapred/TestMatrixMapReduce.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/test/org/apache/hama/mapred/TestMatrixMapReduce.java?rev=684641&r1=684640&r2=684641&view=diff
==============================================================================
--- incubator/hama/trunk/src/test/org/apache/hama/mapred/TestMatrixMapReduce.java (original)
+++ incubator/hama/trunk/src/test/org/apache/hama/mapred/TestMatrixMapReduce.java Sun Aug 10 22:56:23 2008
@@ -58,7 +58,6 @@
 
   public void miniMRJob() throws IOException {
     Matrix c = new Matrix(conf, new Text("xanadu"));
-    c.close();
 
     JobConf jobConf = new JobConf(conf, TestMatrixMapReduce.class);
     jobConf.setJobName("test MR job");