Return-Path: X-Original-To: apmail-hama-commits-archive@www.apache.org Delivered-To: apmail-hama-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 0328C10DEB for ; Wed, 5 Feb 2014 02:16:01 +0000 (UTC) Received: (qmail 23502 invoked by uid 500); 5 Feb 2014 02:16:00 -0000 Delivered-To: apmail-hama-commits-archive@hama.apache.org Received: (qmail 23469 invoked by uid 500); 5 Feb 2014 02:16:00 -0000 Mailing-List: contact commits-help@hama.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hama.apache.org Delivered-To: mailing list commits@hama.apache.org Received: (qmail 23461 invoked by uid 99); 5 Feb 2014 02:15:59 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 05 Feb 2014 02:15:59 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 05 Feb 2014 02:15:58 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id 2F8032388860; Wed, 5 Feb 2014 02:15:38 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1564610 - in /hama/trunk/examples/src: main/java/org/apache/hama/examples/BipartiteMatching.java test/java/org/apache/hama/examples/BipartiteMatchingTest.java Date: Wed, 05 Feb 2014 02:15:38 -0000 To: commits@hama.apache.org From: edwardyoon@apache.org X-Mailer: svnmailer-1.0.9 Message-Id: <20140205021538.2F8032388860@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: edwardyoon Date: Wed Feb 5 02:15:37 2014 New Revision: 1564610 URL: http://svn.apache.org/r1564610 Log: Fix a bug in BipartiteMatching Modified: hama/trunk/examples/src/main/java/org/apache/hama/examples/BipartiteMatching.java hama/trunk/examples/src/test/java/org/apache/hama/examples/BipartiteMatchingTest.java Modified: hama/trunk/examples/src/main/java/org/apache/hama/examples/BipartiteMatching.java URL: http://svn.apache.org/viewvc/hama/trunk/examples/src/main/java/org/apache/hama/examples/BipartiteMatching.java?rev=1564610&r1=1564609&r2=1564610&view=diff ============================================================================== --- hama/trunk/examples/src/main/java/org/apache/hama/examples/BipartiteMatching.java (original) +++ hama/trunk/examples/src/main/java/org/apache/hama/examples/BipartiteMatching.java Wed Feb 5 02:15:37 2014 @@ -64,93 +64,69 @@ public final class BipartiteMatching { @Override public void compute(Iterable msgs) throws IOException { - Random random = new Random(Long.parseLong(getConf().get( - SEED_CONFIGURATION_KEY))); - if (isMatched()) { voteToHalt(); - } else { - switch ((int) getSuperstepCount() % 4) { - case 0: - if (Objects.equal(getComponent(), LEFT)) { - sendMessageToNeighbors(getNewMessage()); - } - break; + return; + } - case 1: - if (Objects.equal(getComponent(), RIGHT)) { - List buffer = new ArrayList(); - for (TextPair next : msgs) { - buffer.add(new TextPair(next.getFirst(), next.getSecond())); - } - if (buffer.size() > 0) { - int rand = RandomUtils.nextInt(random, buffer.size()); - TextPair luckyMsg = buffer.get(rand); - - Text sourceVertex = getSourceVertex(luckyMsg); - sendMessage(sourceVertex, getNewMessage()); - } + switch ((int) getSuperstepCount() % 4) { + case 0: + if (Objects.equal(getValue().getSecond(), LEFT)) { + sendMessageToNeighbors(getNewMessage()); + } + break; + + case 1: + if (Objects.equal(getValue().getSecond(), RIGHT)) { + List buffer = new ArrayList(); + for (TextPair next : msgs) { + buffer.add(new TextPair(next.getFirst(), next.getSecond())); } - break; + if (buffer.size() > 0) { + Random random = new Random(System.currentTimeMillis()); + TextPair luckyMsg = buffer.get(RandomUtils.nextInt(random, + buffer.size())); - case 2: - if (Objects.equal(getComponent(), LEFT)) { - List buffer = new ArrayList(); - for (TextPair next : msgs) { - buffer.add(new TextPair(next.getFirst(), next.getSecond())); - } - if (buffer.size() > 0) { - int rand = RandomUtils.nextInt(random, buffer.size()); - TextPair luckyMsg = buffer.get(rand); - - Text sourceVertex = getSourceVertex(luckyMsg); - setMatchVertex(sourceVertex); - sendMessage(sourceVertex, getNewMessage()); - } + sendMessage(luckyMsg.getFirst(), getNewMessage()); } - break; + } + break; - case 3: - if (Objects.equal(getComponent(), RIGHT)) { - Iterator messages = msgs.iterator(); - if (messages.hasNext()) { - TextPair next = messages.next(); - Text sourceVertex = getSourceVertex(next); - setMatchVertex(sourceVertex); - } + case 2: + if (Objects.equal(getValue().getSecond(), LEFT)) { + List buffer = new ArrayList(); + for (TextPair next : msgs) { + buffer.add(new TextPair(next.getFirst(), next.getSecond())); } - break; - } - } - } + if (buffer.size() > 0) { + Random random = new Random(System.currentTimeMillis()); + TextPair luckyMsg = buffer.get(RandomUtils.nextInt(random, + buffer.size())); - /** - * Finds the vertex from which "msg" came. - */ - private static Text getSourceVertex(TextPair msg) { - return msg.getFirst(); - } + getValue().setFirst(luckyMsg.getFirst()); + sendMessage(luckyMsg.getFirst(), getNewMessage()); + } + } + break; - /** - * Pairs "this" vertex with the "matchVertex" - */ - private void setMatchVertex(Text matchVertex) { - getValue().setFirst(matchVertex); + case 3: + if (Objects.equal(getValue().getSecond(), RIGHT)) { + Iterator messages = msgs.iterator(); + if (messages.hasNext()) { + TextPair next = messages.next(); + getValue().setFirst(next.getFirst()); + } + } + break; + } } private TextPair getNewMessage() { return new TextPair(new Text(getVertexID()), new Text("1")); } - /** - * Returns the component{LEFT/RIGHT} to which this vertex belongs. - */ - private Text getComponent() { - return getValue().getSecond(); - } - private boolean isMatched() { - return !this.getValue().getFirst().equals(UNMATCHED); + return !getValue().getFirst().equals(UNMATCHED); } } @@ -158,10 +134,10 @@ public final class BipartiteMatching { /** * * Input graph is given as
- * : ..
+ * : ..
* A L:B D
- * B R:A C
- * C L:B D
+ * B R:A
+ * C L:B
* D R:A C
*/ public static class BipartiteMatchingVertexReader extends @@ -197,12 +173,9 @@ public final class BipartiteMatching { GraphJob job = new GraphJob(conf, BipartiteMatching.class); // set the defaults - job.setMaxIteration(Integer.MAX_VALUE); + job.setMaxIteration(30); job.setNumBspTask(2); - conf.set(SEED_CONFIGURATION_KEY, System.currentTimeMillis() + ""); - if (args.length == 5) - conf.set(SEED_CONFIGURATION_KEY, args[4]); if (args.length >= 4) job.setNumBspTask(Integer.parseInt(args[3])); if (args.length >= 3) Modified: hama/trunk/examples/src/test/java/org/apache/hama/examples/BipartiteMatchingTest.java URL: http://svn.apache.org/viewvc/hama/trunk/examples/src/test/java/org/apache/hama/examples/BipartiteMatchingTest.java?rev=1564610&r1=1564609&r2=1564610&view=diff ============================================================================== --- hama/trunk/examples/src/test/java/org/apache/hama/examples/BipartiteMatchingTest.java (original) +++ hama/trunk/examples/src/test/java/org/apache/hama/examples/BipartiteMatchingTest.java Wed Feb 5 02:15:37 2014 @@ -42,16 +42,16 @@ import org.junit.Test; public class BipartiteMatchingTest extends TestCase { - private String[] input = { "A L:B D", "B R:A C", "C L:B D", "D R:A C" }; + private String[] input = { "A L:B", "B R:A", "C L:B D", "D R:A C" }; private final static String DELIMETER = "\t"; private Map output1 = new HashMap(); { - output1.put("A", "D L"); - output1.put("B", "C R"); - output1.put("C", "B L"); - output1.put("D", "A R"); + output1.put("A", "B L"); + output1.put("B", "A R"); + output1.put("C", "D L"); + output1.put("D", "C R"); } public static class CustomTextPartitioner implements @@ -122,7 +122,7 @@ public class BipartiteMatchingTest exten assertNotNull(expValue); System.out.println(lineA[0] + " -> " + lineA[1] + " expvalue = " + expValue); - // assertEquals(expValue, lineA[1]); + assertEquals(expValue, lineA[1]); } in.close(); } @@ -148,10 +148,9 @@ public class BipartiteMatchingTest exten deleteTempDirs(); generateTestData(); try { - String seed = "2"; HamaConfiguration conf = new HamaConfiguration(); GraphJob job = BipartiteMatching.createJob(new String[] { INPUT, OUTPUT, - "60", "2", seed }, conf); + "30", "2" }, conf); job.setPartitioner(CustomTextPartitioner.class); long startTime = System.currentTimeMillis();