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 2012/12/10 07:29:42 UTC
svn commit: r1419214 - in /hama/trunk: ./
core/src/main/java/org/apache/hama/bsp/
examples/src/main/java/org/apache/hama/examples/
examples/src/main/java/org/apache/hama/examples/util/
examples/src/test/java/org/apache/hama/examples/
Author: edwardyoon
Date: Mon Dec 10 06:29:41 2012
New Revision: 1419214
URL: http://svn.apache.org/viewvc?rev=1419214&view=rev
Log:
HAMA-659: Add random symmetric sparse matrix generator
Added:
hama/trunk/core/src/main/java/org/apache/hama/bsp/TextArrayWritable.java (with props)
hama/trunk/examples/src/main/java/org/apache/hama/examples/util/Generator.java
hama/trunk/examples/src/main/java/org/apache/hama/examples/util/SymmetricMatrixGen.java
hama/trunk/examples/src/test/java/org/apache/hama/examples/SymmetricMatrixGenTest.java
Modified:
hama/trunk/CHANGES.txt
hama/trunk/examples/src/main/java/org/apache/hama/examples/ExampleDriver.java
Modified: hama/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hama/trunk/CHANGES.txt?rev=1419214&r1=1419213&r2=1419214&view=diff
==============================================================================
--- hama/trunk/CHANGES.txt (original)
+++ hama/trunk/CHANGES.txt Mon Dec 10 06:29:41 2012
@@ -1,6 +1,16 @@
Hama Change Log
-Release 0.6 (unreleased changes)
+Release 0.7 (unreleased changes)
+
+ NEW FEATURES
+
+ HAMA-659: Add random symmetric sparse matrix generator (edwardyoon)
+
+ BUG FIXES
+
+ IMPROVEMENTS
+
+Release 0.6 - November 28, 2012
NEW FEATURES
Added: hama/trunk/core/src/main/java/org/apache/hama/bsp/TextArrayWritable.java
URL: http://svn.apache.org/viewvc/hama/trunk/core/src/main/java/org/apache/hama/bsp/TextArrayWritable.java?rev=1419214&view=auto
==============================================================================
--- hama/trunk/core/src/main/java/org/apache/hama/bsp/TextArrayWritable.java (added)
+++ hama/trunk/core/src/main/java/org/apache/hama/bsp/TextArrayWritable.java Mon Dec 10 06:29:41 2012
@@ -0,0 +1,29 @@
+/**
+ * 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.bsp;
+
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.Text;
+
+public class TextArrayWritable extends ArrayWritable {
+
+ public TextArrayWritable() {
+ super(Text.class);
+ }
+
+}
\ No newline at end of file
Propchange: hama/trunk/core/src/main/java/org/apache/hama/bsp/TextArrayWritable.java
------------------------------------------------------------------------------
svn:eol-style = native
Modified: hama/trunk/examples/src/main/java/org/apache/hama/examples/ExampleDriver.java
URL: http://svn.apache.org/viewvc/hama/trunk/examples/src/main/java/org/apache/hama/examples/ExampleDriver.java?rev=1419214&r1=1419213&r2=1419214&view=diff
==============================================================================
--- hama/trunk/examples/src/main/java/org/apache/hama/examples/ExampleDriver.java (original)
+++ hama/trunk/examples/src/main/java/org/apache/hama/examples/ExampleDriver.java Mon Dec 10 06:29:41 2012
@@ -20,6 +20,7 @@
package org.apache.hama.examples;
import org.apache.hadoop.util.ProgramDriver;
+import org.apache.hama.examples.util.Generator;
public class ExampleDriver {
@@ -37,6 +38,8 @@ public class ExampleDriver {
pgd.addClass("bipartite", BipartiteMatching.class, "Bipartite Matching");
pgd.addClass("kmeans", Kmeans.class, "K-Means Clustering");
pgd.addClass("gd", GradientDescentExample.class, "Gradient Descent");
+
+ pgd.addClass("gen", Generator.class, "Random Data Generator Util");
pgd.driver(args);
} catch (Throwable e) {
e.printStackTrace();
Added: hama/trunk/examples/src/main/java/org/apache/hama/examples/util/Generator.java
URL: http://svn.apache.org/viewvc/hama/trunk/examples/src/main/java/org/apache/hama/examples/util/Generator.java?rev=1419214&view=auto
==============================================================================
--- hama/trunk/examples/src/main/java/org/apache/hama/examples/util/Generator.java (added)
+++ hama/trunk/examples/src/main/java/org/apache/hama/examples/util/Generator.java Mon Dec 10 06:29:41 2012
@@ -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.hama.examples.util;
+
+public class Generator {
+
+ public static void main(String[] args) throws Exception {
+ if (args.length < 1) {
+ System.out.println("Valid command names are:");
+ System.out
+ .println(" symmetric: Generate random symmetric matrix, which can be used as a input of graph examples.");
+ System.exit(1);
+ }
+
+ if (args[0].equals("symmetric")) {
+ String[] newArgs = new String[args.length - 1];
+ System.arraycopy(args, 1, newArgs, 0, args.length - 1);
+ SymmetricMatrixGen.main(newArgs);
+ }
+ }
+}
Added: hama/trunk/examples/src/main/java/org/apache/hama/examples/util/SymmetricMatrixGen.java
URL: http://svn.apache.org/viewvc/hama/trunk/examples/src/main/java/org/apache/hama/examples/util/SymmetricMatrixGen.java?rev=1419214&view=auto
==============================================================================
--- hama/trunk/examples/src/main/java/org/apache/hama/examples/util/SymmetricMatrixGen.java (added)
+++ hama/trunk/examples/src/main/java/org/apache/hama/examples/util/SymmetricMatrixGen.java Mon Dec 10 06:29:41 2012
@@ -0,0 +1,155 @@
+/**
+ * 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.examples.util;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hama.HamaConfiguration;
+import org.apache.hama.bsp.BSP;
+import org.apache.hama.bsp.BSPJob;
+import org.apache.hama.bsp.BSPPeer;
+import org.apache.hama.bsp.FileOutputFormat;
+import org.apache.hama.bsp.NullInputFormat;
+import org.apache.hama.bsp.SequenceFileOutputFormat;
+import org.apache.hama.bsp.TextArrayWritable;
+import org.apache.hama.bsp.sync.SyncException;
+import org.apache.hama.examples.CombineExample;
+
+public class SymmetricMatrixGen {
+ protected static Log LOG = LogFactory.getLog(SymmetricMatrixGen.class);
+
+ private static String NUM_OF_VERTICES = "size.of.matrix";
+ private static String DENSITY = "density.of.matrix";
+
+ public static class SymmetricMatrixGenBSP extends
+ BSP<NullWritable, NullWritable, Text, TextArrayWritable, Text> {
+
+ private Configuration conf;
+ private int numberN;
+ private int density;
+ private Map<Integer, HashSet<Integer>> list = new HashMap<Integer, HashSet<Integer>>();
+
+ @Override
+ public void setup(
+ BSPPeer<NullWritable, NullWritable, Text, TextArrayWritable, Text> peer) {
+ this.conf = peer.getConfiguration();
+ numberN = conf.getInt(NUM_OF_VERTICES, 10);
+ density = conf.getInt(DENSITY, 1);
+ }
+
+ @Override
+ public void bsp(
+ BSPPeer<NullWritable, NullWritable, Text, TextArrayWritable, Text> peer)
+ throws IOException, SyncException, InterruptedException {
+ int interval = numberN / peer.getNumPeers();
+ int startID = peer.getPeerIndex() * interval;
+ int endID;
+ if (peer.getPeerIndex() == peer.getNumPeers() - 1)
+ endID = numberN;
+ else
+ endID = startID + interval;
+
+ // Generate N*(N+1) elements for lower triangular
+ for (int i = startID; i < endID; i++) {
+ HashSet<Integer> edges = new HashSet<Integer>();
+ for (int j = 0; j <= i; j++) {
+ boolean nonZero = new Random().nextInt(density) == 0;
+ if (nonZero && !edges.contains(j) && i != j) {
+ edges.add(j);
+ peer.send(peer.getPeerName(j / interval), new Text(j + "," + i));
+ }
+ }
+
+ list.put(i, edges);
+ }
+
+ // Synchronize the upper and lower
+ peer.sync();
+ Text received;
+ while ((received = peer.getCurrentMessage()) != null) {
+ String[] kv = received.toString().split(",");
+ HashSet<Integer> nList = list.get(Integer.parseInt(kv[0]));
+ nList.add(Integer.parseInt(kv[1]));
+ list.put(Integer.parseInt(kv[0]), nList);
+ }
+ }
+
+ @Override
+ public void cleanup(
+ BSPPeer<NullWritable, NullWritable, Text, TextArrayWritable, Text> peer)
+ throws IOException {
+ for (Map.Entry<Integer, HashSet<Integer>> e : list.entrySet()) {
+ Writable[] values = new Writable[e.getValue().size()];
+ if (values.length > 0) {
+ int i = 0;
+ for (Integer v : e.getValue()) {
+ values[i] = new Text(String.valueOf(v));
+ i++;
+ }
+
+ TextArrayWritable value = new TextArrayWritable();
+ value.set(values);
+ peer.write(new Text(String.valueOf(e.getKey())), value);
+ }
+ }
+ }
+ }
+
+ public static void main(String[] args) throws InterruptedException,
+ IOException, ClassNotFoundException {
+ if (args.length < 4) {
+ System.out
+ .println("Usage: <number n> <1/x density> <output path> <number of tasks>");
+ System.exit(1);
+ }
+
+ // BSP job configuration
+ HamaConfiguration conf = new HamaConfiguration();
+
+ conf.setInt(NUM_OF_VERTICES, Integer.parseInt(args[0]));
+ conf.setInt(DENSITY, Integer.parseInt(args[1]));
+
+ BSPJob bsp = new BSPJob(conf, CombineExample.class);
+ // Set the job name
+ bsp.setJobName("Random Symmetric Sparse Matrix Generator");
+ bsp.setBspClass(SymmetricMatrixGenBSP.class);
+ bsp.setInputFormat(NullInputFormat.class);
+ bsp.setOutputKeyClass(Text.class);
+ bsp.setOutputValueClass(TextArrayWritable.class);
+ bsp.setOutputFormat(SequenceFileOutputFormat.class);
+ FileOutputFormat.setOutputPath(bsp, new Path(args[2]));
+ bsp.setNumBspTask(Integer.parseInt(args[3]));
+
+ long startTime = System.currentTimeMillis();
+ if (bsp.waitForCompletion(true)) {
+ System.out.println("Job Finished in "
+ + (System.currentTimeMillis() - startTime) / 1000.0 + " seconds");
+ }
+ }
+}
Added: hama/trunk/examples/src/test/java/org/apache/hama/examples/SymmetricMatrixGenTest.java
URL: http://svn.apache.org/viewvc/hama/trunk/examples/src/test/java/org/apache/hama/examples/SymmetricMatrixGenTest.java?rev=1419214&view=auto
==============================================================================
--- hama/trunk/examples/src/test/java/org/apache/hama/examples/SymmetricMatrixGenTest.java (added)
+++ hama/trunk/examples/src/test/java/org/apache/hama/examples/SymmetricMatrixGenTest.java Mon Dec 10 06:29:41 2012
@@ -0,0 +1,65 @@
+/**
+ * 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.examples;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hama.bsp.TextArrayWritable;
+import org.apache.hama.examples.util.SymmetricMatrixGen;
+import org.junit.Test;
+
+public class SymmetricMatrixGenTest {
+ protected static Log LOG = LogFactory
+ .getLog(SymmetricMatrixGenTest.class);
+ private static String TEST_OUTPUT = "/tmp/test";
+
+ @Test
+ public void testGraphGenerator() throws Exception {
+ Configuration conf = new Configuration();
+
+ SymmetricMatrixGen
+ .main(new String[] { "20", "10", TEST_OUTPUT, "3" });
+ FileSystem fs = FileSystem.get(conf);
+
+ FileStatus[] globStatus = fs.globStatus(new Path(TEST_OUTPUT + "/part-*"));
+ for (FileStatus fts : globStatus) {
+ SequenceFile.Reader reader = new SequenceFile.Reader(fs, fts.getPath(),
+ conf);
+ Text key = new Text();
+ TextArrayWritable value = new TextArrayWritable();
+
+ while (reader.next(key, value)) {
+ String values = "";
+ for (Writable v : value.get()) {
+ values += v.toString() + " ";
+ }
+ LOG.info(fts.getPath() + ": " + key.toString() + " | " + values);
+ }
+ reader.close();
+ }
+
+ fs.delete(new Path(TEST_OUTPUT), true);
+ }
+}