You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@giraph.apache.org by ac...@apache.org on 2012/07/10 11:31:35 UTC

svn commit: r1359536 - in /giraph/trunk: CHANGELOG src/main/java/org/apache/giraph/graph/IntIntNullIntVertex.java src/test/java/org/apache/giraph/graph/TestIntIntNullIntVertex.java

Author: aching
Date: Tue Jul 10 09:31:35 2012
New Revision: 1359536

URL: http://svn.apache.org/viewvc?rev=1359536&view=rev
Log:
GIRAPH-239: IntIntNullIntVertex doesn't save halted state (apresta
via aching)


Added:
    giraph/trunk/src/test/java/org/apache/giraph/graph/TestIntIntNullIntVertex.java
Modified:
    giraph/trunk/CHANGELOG
    giraph/trunk/src/main/java/org/apache/giraph/graph/IntIntNullIntVertex.java

Modified: giraph/trunk/CHANGELOG
URL: http://svn.apache.org/viewvc/giraph/trunk/CHANGELOG?rev=1359536&r1=1359535&r2=1359536&view=diff
==============================================================================
--- giraph/trunk/CHANGELOG (original)
+++ giraph/trunk/CHANGELOG Tue Jul 10 09:31:35 2012
@@ -2,6 +2,9 @@ Giraph Change Log
 
 Release 0.2.0 - unreleased
 
+  GIRAPH-239: IntIntNullIntVertex doesn't save halted state (apresta
+  via aching)
+
   GIRAPH-238: BasicVertex should have default Writable implementation
   (apresta via aching)
 

Modified: giraph/trunk/src/main/java/org/apache/giraph/graph/IntIntNullIntVertex.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/graph/IntIntNullIntVertex.java?rev=1359536&r1=1359535&r2=1359536&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/graph/IntIntNullIntVertex.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/graph/IntIntNullIntVertex.java Tue Jul 10 09:31:35 2012
@@ -18,12 +18,11 @@
 
 package org.apache.giraph.graph;
 
+import com.google.common.collect.Iterables;
 import org.apache.giraph.utils.UnmodifiableIntArrayIterator;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.NullWritable;
 
-import com.google.common.collect.Iterables;
-
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
@@ -156,6 +155,7 @@ public abstract class IntIntNullIntVerte
     for (int n = 0; n < messages.length; n++) {
       out.writeInt(messages[n]);
     }
+    out.writeBoolean(halt);
   }
 
   @Override
@@ -172,6 +172,7 @@ public abstract class IntIntNullIntVerte
     for (int n = 0; n < numMessages; n++) {
       messages[n] = in.readInt();
     }
+    halt = in.readBoolean();
   }
 
 }

Added: giraph/trunk/src/test/java/org/apache/giraph/graph/TestIntIntNullIntVertex.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/test/java/org/apache/giraph/graph/TestIntIntNullIntVertex.java?rev=1359536&view=auto
==============================================================================
--- giraph/trunk/src/test/java/org/apache/giraph/graph/TestIntIntNullIntVertex.java (added)
+++ giraph/trunk/src/test/java/org/apache/giraph/graph/TestIntIntNullIntVertex.java Tue Jul 10 09:31:35 2012
@@ -0,0 +1,86 @@
+/*
+ * 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.giraph.graph;
+
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+import com.google.common.collect.Lists;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Tests {@link IntIntNullIntVertex}.
+ */
+public class TestIntIntNullIntVertex {
+  /**
+   * Simple instantiable class that extends {@link IntIntNullIntVertex}.
+   */
+  private static class MyIntIntNullVertex extends IntIntNullIntVertex {
+    @Override
+    public void compute(Iterator<IntWritable> msgIterator) throws IOException {
+    }
+  }
+
+  @Test
+  public void testSerialize() throws IOException {
+    IntIntNullIntVertex vertex = new MyIntIntNullVertex();
+
+    Map<IntWritable, NullWritable> edges =
+        new HashMap<IntWritable, NullWritable>();
+    edges.put(new IntWritable(3), NullWritable.get());
+    edges.put(new IntWritable(47), NullWritable.get());
+
+    List<IntWritable> messages = Lists.newArrayList(new IntWritable(943),
+        new IntWritable(571));
+
+    vertex.initialize(new IntWritable(23), new IntWritable(7), edges,
+        messages);
+    vertex.voteToHalt();
+
+    ByteArrayOutputStream outStream = new ByteArrayOutputStream();
+    DataOutput out = new DataOutputStream(outStream);
+    vertex.write(out);
+
+    IntIntNullIntVertex vertex1 = new MyIntIntNullVertex();
+
+    ByteArrayInputStream inStream = new ByteArrayInputStream(
+        outStream.toByteArray());
+    DataInput in = new DataInputStream(inStream);
+    vertex1.readFields(in);
+
+    assertEquals(2, vertex1.getNumOutEdges());
+    assertEquals(Lists.newArrayList(new IntWritable(943),
+        new IntWritable(571)), Lists.newArrayList(vertex1.getMessages()));
+    assertEquals(true, vertex1.isHalted());
+  }
+}