hama-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From edwardy...@apache.org
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 GMT
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<TextPair> 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<TextPair> buffer = new ArrayList<TextPair>();
-              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<TextPair> buffer = new ArrayList<TextPair>();
+            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<TextPair> buffer = new ArrayList<TextPair>();
-              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<TextPair> 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<TextPair> buffer = new ArrayList<TextPair>();
+            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<TextPair> 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<br/>
-   * <Vertex> <component value>: <adjacent_vertex_1> <adjacent_vertex_2>
..<br/>
+   * <Vertex> <component value>:<edge 1> <edge 2> ..<br/>
    * A L:B D<br/>
-   * B R:A C<br/>
-   * C L:B D<br/>
+   * B R:A<br/>
+   * C L:B<br/>
    * D R:A C<br/>
    */
   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<String, String> output1 = new HashMap<String, String>();
   {
-    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();



Mime
View raw message