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/09/22 16:47:02 UTC

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

Author: edwardyoon
Date: Mon Sep 22 07:47:01 2008
New Revision: 697858

URL: http://svn.apache.org/viewvc?rev=697858&view=rev
Log:
Renaming 1d layout map/reduce classes

Added:
    incubator/hama/trunk/src/java/org/apache/hama/algebra/Add1DLayoutMap.java
    incubator/hama/trunk/src/java/org/apache/hama/algebra/Add1DLayoutReduce.java
    incubator/hama/trunk/src/java/org/apache/hama/algebra/Mult1DLayoutMap.java
    incubator/hama/trunk/src/java/org/apache/hama/algebra/Mult1DLayoutReduce.java
Removed:
    incubator/hama/trunk/src/java/org/apache/hama/algebra/AdditionMap.java
    incubator/hama/trunk/src/java/org/apache/hama/algebra/AdditionReduce.java
    incubator/hama/trunk/src/java/org/apache/hama/algebra/MultiplicationMap.java
    incubator/hama/trunk/src/java/org/apache/hama/algebra/MultiplicationReduce.java
Modified:
    incubator/hama/trunk/src/java/org/apache/hama/DenseMatrix.java
    incubator/hama/trunk/src/java/org/apache/hama/shell/execution/AlgebraOperator.java
    incubator/hama/trunk/src/test/org/apache/hama/mapred/TestMatrixMapReduce.java

Modified: incubator/hama/trunk/src/java/org/apache/hama/DenseMatrix.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/java/org/apache/hama/DenseMatrix.java?rev=697858&r1=697857&r2=697858&view=diff
==============================================================================
--- incubator/hama/trunk/src/java/org/apache/hama/DenseMatrix.java (original)
+++ incubator/hama/trunk/src/java/org/apache/hama/DenseMatrix.java Mon Sep 22 07:47:01 2008
@@ -1,222 +1,222 @@
-/**
- * Copyright 2007 The Apache Software Foundation
- *
- * 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.hama;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Scanner;
-import org.apache.hadoop.hbase.io.RowResult;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hama.algebra.AdditionMap;
-import org.apache.hama.algebra.AdditionReduce;
-import org.apache.hama.algebra.MultiplicationMap;
-import org.apache.hama.algebra.MultiplicationReduce;
-import org.apache.hama.io.VectorEntry;
-import org.apache.hama.io.VectorMapWritable;
-import org.apache.hama.io.VectorUpdate;
-import org.apache.hama.mapred.MatrixReduce;
-import org.apache.hama.util.Numeric;
-import org.apache.hama.util.RandomVariable;
-
-public class DenseMatrix extends AbstractMatrix implements Matrix {
-
-  /**
-   * Construct
-   * 
-   * @param conf configuration object
-   */
-  public DenseMatrix(HamaConfiguration conf) {
-    setConfiguration(conf);
-  }
-
-  /**
-   * Construct an matrix
-   * 
-   * @param conf configuration object
-   * @param matrixName the name of the matrix
-   */
-  public DenseMatrix(HamaConfiguration conf, String matrixName) {
-    try {
-      setConfiguration(conf);
-      this.matrixName = matrixName;
-
-      if (!admin.tableExists(matrixName)) {
-        tableDesc = new HTableDescriptor(matrixName);
-        create();
-      }
-
-      table = new HTable(config, matrixName);
-    } catch (Exception e) {
-      e.printStackTrace();
-    }
-  }
-
-  /**
-   * Construct an m-by-n constant matrix.
-   * 
-   * @param conf configuration object
-   * @param m the number of rows.
-   * @param n the number of columns.
-   * @param s fill the matrix with this scalar value.
-   */
-  public DenseMatrix(HamaConfiguration conf, int m, int n, double s) {
-    try {
-      setConfiguration(conf);
-      matrixName = RandomVariable.randMatrixName();
-
-      if (!admin.tableExists(matrixName)) {
-        tableDesc = new HTableDescriptor(matrixName);
-        tableDesc.addFamily(new HColumnDescriptor(Constants.COLUMN));
-        create();
-      }
-
-      table = new HTable(config, matrixName);
-
-      for (int i = 0; i < m; i++) {
-        for (int j = 0; j < n; j++) {
-          set(i, j, s);
-        }
-      }
-
-      setDimension(m, n);
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-  }
-
-  /**
-   * Generate matrix with random elements
-   * 
-   * @param conf configuration object
-   * @param m the number of rows.
-   * @param n the number of columns.
-   * @return an m-by-n matrix with uniformly distributed random elements.
-   * @throws IOException
-   */
-  public static Matrix random(HamaConfiguration conf, int m, int n)
-      throws IOException {
-    String name = RandomVariable.randMatrixName();
-    Matrix rand = new DenseMatrix(conf, name);
-    for (int i = 0; i < m; i++) {
-      DenseVector vector = new DenseVector();
-      for (int j = 0; j < n; j++) {
-        vector.set(j, RandomVariable.rand());
-      }
-      rand.setRow(i, vector);
-    }
-
-    rand.setDimension(m, n);
-    LOG.info("Create the " + m + " * " + n + " random matrix : " + name);
-    return rand;
-  }
-
-  public Matrix add(Matrix B) throws IOException {
-    String output = RandomVariable.randMatrixName();
-    Matrix result = new DenseMatrix(config, output);
-
-    JobConf jobConf = new JobConf(config);
-    jobConf.setJobName("addition MR job" + result.getName());
-
-    AdditionMap.initJob(this.getName(), B.getName(), AdditionMap.class,
-        IntWritable.class, DenseVector.class, jobConf);
-    MatrixReduce.initJob(result.getName(), AdditionReduce.class, jobConf);
-
-    execute(jobConf, result);
-    return result;
-  }
-
-  public Matrix add(double alpha, Matrix B) throws IOException {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  public DenseVector getRow(int row) throws IOException {
-    return new DenseVector(row, table.getRow(String.valueOf(row)));
-  }
-
-  public Vector getColumn(int column) throws IOException {
-    byte[] columnKey = Numeric.getColumnIndex(column);
-    byte[][] c = { columnKey };
-    Scanner scan = table.getScanner(c, HConstants.EMPTY_START_ROW);
-
-    VectorMapWritable<Integer, VectorEntry> trunk = new VectorMapWritable<Integer, VectorEntry>();
-
-    for (RowResult row : scan) {
-      trunk.put(Numeric.bytesToInt(row.getRow()), new VectorEntry(row
-          .get(columnKey)));
-    }
-
-    return new DenseVector(trunk);
-  }
-
-  public Matrix mult(Matrix B) throws IOException {
-    String output = RandomVariable.randMatrixName();
-    Matrix result = new DenseMatrix(config, output);
-
-    JobConf jobConf = new JobConf(config);
-    jobConf.setJobName("multiplication MR job : " + result.getName());
-
-    MultiplicationMap.initJob(this.getName(), B.getName(),
-        MultiplicationMap.class, IntWritable.class, DenseVector.class, jobConf);
-    MatrixReduce.initJob(result.getName(), MultiplicationReduce.class, jobConf);
-    execute(jobConf, result);
-    return result;
-  }
-
-  public Matrix multAdd(double alpha, Matrix B, Matrix C) throws IOException {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  public double norm(Norm type) throws IOException {
-    // TODO Auto-generated method stub
-    return 0;
-  }
-
-  public Matrix set(double alpha, Matrix B) throws IOException {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  public Matrix set(Matrix B) throws IOException {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  public void setRow(int row, Vector vector) throws IOException {
-    VectorUpdate update = new VectorUpdate(row);
-    update.putAll(((DenseVector) vector).getEntries().entrySet());
-    table.commit(update.getBatchUpdate());
-  }
-
-  public void setColumn(int column, Vector vector) throws IOException {
-    // TODO Auto-generated method stub
-  }
-
-  public void load(String path) throws IOException {
-    matrixName = hAdmin.get(path);
-    table = new HTable(matrixName);
-  }
-}
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * 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.hama;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Scanner;
+import org.apache.hadoop.hbase.io.RowResult;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hama.algebra.Add1DLayoutMap;
+import org.apache.hama.algebra.Add1DLayoutReduce;
+import org.apache.hama.algebra.Mult1DLayoutMap;
+import org.apache.hama.algebra.Mult1DLayoutReduce;
+import org.apache.hama.io.VectorEntry;
+import org.apache.hama.io.VectorMapWritable;
+import org.apache.hama.io.VectorUpdate;
+import org.apache.hama.mapred.MatrixReduce;
+import org.apache.hama.util.Numeric;
+import org.apache.hama.util.RandomVariable;
+
+public class DenseMatrix extends AbstractMatrix implements Matrix {
+
+  /**
+   * Construct
+   * 
+   * @param conf configuration object
+   */
+  public DenseMatrix(HamaConfiguration conf) {
+    setConfiguration(conf);
+  }
+
+  /**
+   * Construct an matrix
+   * 
+   * @param conf configuration object
+   * @param matrixName the name of the matrix
+   */
+  public DenseMatrix(HamaConfiguration conf, String matrixName) {
+    try {
+      setConfiguration(conf);
+      this.matrixName = matrixName;
+
+      if (!admin.tableExists(matrixName)) {
+        tableDesc = new HTableDescriptor(matrixName);
+        create();
+      }
+
+      table = new HTable(config, matrixName);
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+  }
+
+  /**
+   * Construct an m-by-n constant matrix.
+   * 
+   * @param conf configuration object
+   * @param m the number of rows.
+   * @param n the number of columns.
+   * @param s fill the matrix with this scalar value.
+   */
+  public DenseMatrix(HamaConfiguration conf, int m, int n, double s) {
+    try {
+      setConfiguration(conf);
+      matrixName = RandomVariable.randMatrixName();
+
+      if (!admin.tableExists(matrixName)) {
+        tableDesc = new HTableDescriptor(matrixName);
+        tableDesc.addFamily(new HColumnDescriptor(Constants.COLUMN));
+        create();
+      }
+
+      table = new HTable(config, matrixName);
+
+      for (int i = 0; i < m; i++) {
+        for (int j = 0; j < n; j++) {
+          set(i, j, s);
+        }
+      }
+
+      setDimension(m, n);
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+  }
+
+  /**
+   * Generate matrix with random elements
+   * 
+   * @param conf configuration object
+   * @param m the number of rows.
+   * @param n the number of columns.
+   * @return an m-by-n matrix with uniformly distributed random elements.
+   * @throws IOException
+   */
+  public static Matrix random(HamaConfiguration conf, int m, int n)
+      throws IOException {
+    String name = RandomVariable.randMatrixName();
+    Matrix rand = new DenseMatrix(conf, name);
+    for (int i = 0; i < m; i++) {
+      DenseVector vector = new DenseVector();
+      for (int j = 0; j < n; j++) {
+        vector.set(j, RandomVariable.rand());
+      }
+      rand.setRow(i, vector);
+    }
+
+    rand.setDimension(m, n);
+    LOG.info("Create the " + m + " * " + n + " random matrix : " + name);
+    return rand;
+  }
+
+  public Matrix add(Matrix B) throws IOException {
+    String output = RandomVariable.randMatrixName();
+    Matrix result = new DenseMatrix(config, output);
+
+    JobConf jobConf = new JobConf(config);
+    jobConf.setJobName("addition MR job" + result.getName());
+
+    Add1DLayoutMap.initJob(this.getName(), B.getName(), Add1DLayoutMap.class,
+        IntWritable.class, DenseVector.class, jobConf);
+    MatrixReduce.initJob(result.getName(), Add1DLayoutReduce.class, jobConf);
+
+    execute(jobConf, result);
+    return result;
+  }
+
+  public Matrix add(double alpha, Matrix B) throws IOException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  public DenseVector getRow(int row) throws IOException {
+    return new DenseVector(row, table.getRow(String.valueOf(row)));
+  }
+
+  public Vector getColumn(int column) throws IOException {
+    byte[] columnKey = Numeric.getColumnIndex(column);
+    byte[][] c = { columnKey };
+    Scanner scan = table.getScanner(c, HConstants.EMPTY_START_ROW);
+
+    VectorMapWritable<Integer, VectorEntry> trunk = new VectorMapWritable<Integer, VectorEntry>();
+
+    for (RowResult row : scan) {
+      trunk.put(Numeric.bytesToInt(row.getRow()), new VectorEntry(row
+          .get(columnKey)));
+    }
+
+    return new DenseVector(trunk);
+  }
+
+  public Matrix mult(Matrix B) throws IOException {
+    String output = RandomVariable.randMatrixName();
+    Matrix result = new DenseMatrix(config, output);
+
+    JobConf jobConf = new JobConf(config);
+    jobConf.setJobName("multiplication MR job : " + result.getName());
+
+    Mult1DLayoutMap.initJob(this.getName(), B.getName(),
+        Mult1DLayoutMap.class, IntWritable.class, DenseVector.class, jobConf);
+    MatrixReduce.initJob(result.getName(), Mult1DLayoutReduce.class, jobConf);
+    execute(jobConf, result);
+    return result;
+  }
+
+  public Matrix multAdd(double alpha, Matrix B, Matrix C) throws IOException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  public double norm(Norm type) throws IOException {
+    // TODO Auto-generated method stub
+    return 0;
+  }
+
+  public Matrix set(double alpha, Matrix B) throws IOException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  public Matrix set(Matrix B) throws IOException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  public void setRow(int row, Vector vector) throws IOException {
+    VectorUpdate update = new VectorUpdate(row);
+    update.putAll(((DenseVector) vector).getEntries().entrySet());
+    table.commit(update.getBatchUpdate());
+  }
+
+  public void setColumn(int column, Vector vector) throws IOException {
+    // TODO Auto-generated method stub
+  }
+
+  public void load(String path) throws IOException {
+    matrixName = hAdmin.get(path);
+    table = new HTable(matrixName);
+  }
+}

Added: incubator/hama/trunk/src/java/org/apache/hama/algebra/Add1DLayoutMap.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/java/org/apache/hama/algebra/Add1DLayoutMap.java?rev=697858&view=auto
==============================================================================
--- incubator/hama/trunk/src/java/org/apache/hama/algebra/Add1DLayoutMap.java (added)
+++ incubator/hama/trunk/src/java/org/apache/hama/algebra/Add1DLayoutMap.java Mon Sep 22 07:47:01 2008
@@ -0,0 +1,69 @@
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * 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.hama.algebra;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hama.DenseMatrix;
+import org.apache.hama.DenseVector;
+import org.apache.hama.HamaConfiguration;
+import org.apache.hama.Matrix;
+import org.apache.hama.Vector;
+import org.apache.hama.mapred.DenseMap;
+import org.apache.log4j.Logger;
+
+public class Add1DLayoutMap extends DenseMap<IntWritable, DenseVector> {
+  static final Logger LOG = Logger.getLogger(Add1DLayoutMap.class);
+  protected Matrix matrix_b;
+  public static final String MATRIX_B = "hama.addition.matrix.b";
+  
+  public void configure(JobConf job) {
+    matrix_b = new DenseMatrix(new HamaConfiguration(), job.get(MATRIX_B, ""));
+  }
+
+  public static void initJob(String matrix_a, String matrix_b,
+      Class<Add1DLayoutMap> map, 
+      Class<IntWritable> outputKeyClass, 
+      Class<DenseVector> outputValueClass, 
+      JobConf jobConf) {
+    
+    jobConf.setMapOutputValueClass(outputValueClass);
+    jobConf.setMapOutputKeyClass(outputKeyClass);
+    jobConf.setMapperClass(map);
+    jobConf.set(MATRIX_B, matrix_b);
+    
+    initJob(matrix_a, map, jobConf);
+  }
+
+  @Override
+  public void map(IntWritable key, DenseVector value,
+      OutputCollector<IntWritable, DenseVector> output, Reporter reporter)
+      throws IOException {
+
+    Vector v1 = matrix_b.getRow(key.get());
+    output.collect(key, (DenseVector) v1.add(value));
+
+  }
+
+}

Added: incubator/hama/trunk/src/java/org/apache/hama/algebra/Add1DLayoutReduce.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/java/org/apache/hama/algebra/Add1DLayoutReduce.java?rev=697858&view=auto
==============================================================================
--- incubator/hama/trunk/src/java/org/apache/hama/algebra/Add1DLayoutReduce.java (added)
+++ incubator/hama/trunk/src/java/org/apache/hama/algebra/Add1DLayoutReduce.java Mon Sep 22 07:47:01 2008
@@ -0,0 +1,51 @@
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * 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.hama.algebra;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hama.DenseVector;
+import org.apache.hama.io.VectorEntry;
+import org.apache.hama.io.VectorUpdate;
+import org.apache.hama.mapred.MatrixReduce;
+
+public class Add1DLayoutReduce extends MatrixReduce<IntWritable, DenseVector> {
+
+  @Override
+  public void reduce(IntWritable key, Iterator<DenseVector> values,
+      OutputCollector<IntWritable, VectorUpdate> output, Reporter reporter)
+      throws IOException {
+
+    VectorUpdate update = new VectorUpdate(key.get());
+    DenseVector vector = values.next();
+    
+    for (Map.Entry<Integer, VectorEntry> f : vector.entrySet()) {
+      update.put(f.getKey(), f.getValue().getValue());
+    }
+
+    output.collect(key, update);
+  }
+
+}

Added: incubator/hama/trunk/src/java/org/apache/hama/algebra/Mult1DLayoutMap.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/java/org/apache/hama/algebra/Mult1DLayoutMap.java?rev=697858&view=auto
==============================================================================
--- incubator/hama/trunk/src/java/org/apache/hama/algebra/Mult1DLayoutMap.java (added)
+++ incubator/hama/trunk/src/java/org/apache/hama/algebra/Mult1DLayoutMap.java Mon Sep 22 07:47:01 2008
@@ -0,0 +1,77 @@
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * 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.hama.algebra;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hama.DenseMatrix;
+import org.apache.hama.DenseVector;
+import org.apache.hama.HamaConfiguration;
+import org.apache.hama.Matrix;
+import org.apache.hama.Vector;
+import org.apache.hama.io.VectorEntry;
+import org.apache.hama.mapred.DenseMap;
+import org.apache.log4j.Logger;
+
+/**
+ * 1D Block Layout version 
+ */
+public class Mult1DLayoutMap extends DenseMap<IntWritable, DenseVector> {
+  static final Logger LOG = Logger.getLogger(Mult1DLayoutMap.class);
+  protected Matrix matrix_b;
+  public static final String MATRIX_B = "hama.multiplication.matrix.b";
+  
+  public void configure(JobConf job) {
+    matrix_b = new DenseMatrix(new HamaConfiguration(), job.get(MATRIX_B, ""));
+  }
+  
+  public static void initJob(String matrix_a, String matrix_b,
+      Class<Mult1DLayoutMap> map, 
+      Class<IntWritable> outputKeyClass, 
+      Class<DenseVector> outputValueClass, 
+      JobConf jobConf) {
+    
+    jobConf.setMapOutputValueClass(outputValueClass);
+    jobConf.setMapOutputKeyClass(outputKeyClass);
+    jobConf.setMapperClass(map);
+    jobConf.set(MATRIX_B, matrix_b);
+    
+    initJob(matrix_a, map, jobConf);
+  }
+  
+  @Override
+  public void map(IntWritable key, DenseVector value,
+      OutputCollector<IntWritable, DenseVector> output, Reporter reporter)
+      throws IOException {
+
+    Iterator<VectorEntry> it = value.iterator();
+    int i = 0;
+    while (it.hasNext()) {
+      Vector v = matrix_b.getRow(i);
+      output.collect(key, (DenseVector) v.scale(it.next().getValue()));
+      i++;
+    }
+  }
+}

Added: incubator/hama/trunk/src/java/org/apache/hama/algebra/Mult1DLayoutReduce.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/java/org/apache/hama/algebra/Mult1DLayoutReduce.java?rev=697858&view=auto
==============================================================================
--- incubator/hama/trunk/src/java/org/apache/hama/algebra/Mult1DLayoutReduce.java (added)
+++ incubator/hama/trunk/src/java/org/apache/hama/algebra/Mult1DLayoutReduce.java Mon Sep 22 07:47:01 2008
@@ -0,0 +1,64 @@
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * 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.hama.algebra;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hama.DenseVector;
+import org.apache.hama.io.VectorUpdate;
+import org.apache.hama.mapred.MatrixReduce;
+import org.apache.log4j.Logger;
+
+public class Mult1DLayoutReduce extends
+    MatrixReduce<IntWritable, DenseVector> {
+  static final Logger LOG = Logger.getLogger(Mult1DLayoutReduce.class);
+
+  @Override
+  public void reduce(IntWritable key, Iterator<DenseVector> values,
+      OutputCollector<IntWritable, VectorUpdate> output, Reporter reporter)
+      throws IOException {
+
+    VectorUpdate update = new VectorUpdate(key.get());
+
+    DenseVector sum;
+    Map<Integer, Double> buffer = new HashMap<Integer, Double>();
+
+    // Summation
+    while (values.hasNext()) {
+      sum = values.next();
+      for (int i = 0; i < sum.size(); i++) {
+        if (buffer.containsKey(i)) {
+          buffer.put(i, sum.get(i) + buffer.get(i));
+        } else {
+          buffer.put(i, sum.get(i));
+        }
+      }
+    }
+    update.putAll(buffer);
+    output.collect(key, update);
+  }
+
+}

Modified: incubator/hama/trunk/src/java/org/apache/hama/shell/execution/AlgebraOperator.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/java/org/apache/hama/shell/execution/AlgebraOperator.java?rev=697858&r1=697857&r2=697858&view=diff
==============================================================================
--- incubator/hama/trunk/src/java/org/apache/hama/shell/execution/AlgebraOperator.java (original)
+++ incubator/hama/trunk/src/java/org/apache/hama/shell/execution/AlgebraOperator.java Mon Sep 22 07:47:01 2008
@@ -72,11 +72,11 @@
    * 
    * if it is a
    * 
-   * @see {@link Matrix} operator, return it's value (matrix). if it is a
-   * @see {@link Matrix#add(Matrix)} operator, do addition and return it's value
+   * @see Matrix operator, return it's value (matrix). if it is a
+   * @see Matrix#add(Matrix) operator, do addition and return it's value
    *      (matrix). if it is a
-   * @see {@link Matrix#mult(Matrix)} operator, do multiplication and return
-   *      it's value (matrix).
+   * @see Matrix#mult(Matrix) operator, do multiplication and return it's value
+   *      (matrix).
    * 
    * @return Matrix ( or AlgebraOperator(used internally) )
    * @throws AlgebraOpException

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=697858&r1=697857&r2=697858&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 Mon Sep 22 07:47:01 2008
@@ -28,8 +28,8 @@
 import org.apache.hama.DenseVector;
 import org.apache.hama.HCluster;
 import org.apache.hama.Matrix;
-import org.apache.hama.algebra.AdditionMap;
-import org.apache.hama.algebra.AdditionReduce;
+import org.apache.hama.algebra.Add1DLayoutMap;
+import org.apache.hama.algebra.Add1DLayoutReduce;
 import org.apache.log4j.Logger;
 
 /**
@@ -66,9 +66,9 @@
     JobConf jobConf = new JobConf(conf, TestMatrixMapReduce.class);
     jobConf.setJobName("test MR job");
 
-    AdditionMap.initJob(A, B, AdditionMap.class, IntWritable.class,
+    Add1DLayoutMap.initJob(A, B, Add1DLayoutMap.class, IntWritable.class,
         DenseVector.class, jobConf);
-    MatrixReduce.initJob(output, AdditionReduce.class, jobConf);
+    MatrixReduce.initJob(output, Add1DLayoutReduce.class, jobConf);
 
     jobConf.setNumMapTasks(1);
     jobConf.setNumReduceTasks(1);