You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@giraph.apache.org by jg...@apache.org on 2012/08/09 23:43:37 UTC

svn commit: r1371498 [2/2] - in /giraph/trunk: ./ src/main/java/org/apache/giraph/benchmark/ src/main/java/org/apache/giraph/examples/ src/main/java/org/apache/giraph/io/ src/main/java/org/apache/giraph/lib/ src/test/java/org/apache/giraph/ src/test/ja...

Modified: giraph/trunk/src/test/java/org/apache/giraph/TestBspBasic.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/test/java/org/apache/giraph/TestBspBasic.java?rev=1371498&r1=1371497&r2=1371498&view=diff
==============================================================================
--- giraph/trunk/src/test/java/org/apache/giraph/TestBspBasic.java (original)
+++ giraph/trunk/src/test/java/org/apache/giraph/TestBspBasic.java Thu Aug  9 21:43:36 2012
@@ -18,9 +18,6 @@
 
 package org.apache.giraph;
 
-import org.apache.giraph.aggregators.DoubleMaxAggregator;
-import org.apache.giraph.aggregators.DoubleMinAggregator;
-import org.apache.giraph.aggregators.LongSumAggregator;
 import org.apache.giraph.examples.GeneratedVertexReader;
 import org.apache.giraph.examples.SimpleCombinerVertex;
 import org.apache.giraph.examples.SimpleFailVertex;
@@ -39,7 +36,7 @@ import org.apache.giraph.graph.GraphStat
 import org.apache.giraph.graph.TextAggregatorWriter;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexInputFormat;
-import org.apache.giraph.lib.JsonLongDoubleFloatDoubleVertexOutputFormat;
+import org.apache.giraph.io.JsonLongDoubleFloatDoubleVertexOutputFormat;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;

Modified: giraph/trunk/src/test/java/org/apache/giraph/TestJsonBase64Format.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/test/java/org/apache/giraph/TestJsonBase64Format.java?rev=1371498&r1=1371497&r2=1371498&view=diff
==============================================================================
--- giraph/trunk/src/test/java/org/apache/giraph/TestJsonBase64Format.java (original)
+++ giraph/trunk/src/test/java/org/apache/giraph/TestJsonBase64Format.java Thu Aug  9 21:43:36 2012
@@ -24,12 +24,11 @@ import java.io.IOException;
 
 import org.apache.giraph.benchmark.PageRankBenchmark;
 import org.apache.giraph.benchmark.PageRankComputation;
-import org.apache.giraph.benchmark.PseudoRandomVertexInputFormat;
+import org.apache.giraph.io.PseudoRandomVertexInputFormat;
 import org.apache.giraph.graph.GiraphJob;
-import org.apache.giraph.lib.JsonBase64VertexInputFormat;
-import org.apache.giraph.lib.JsonBase64VertexOutputFormat;
+import org.apache.giraph.io.JsonBase64VertexInputFormat;
+import org.apache.giraph.io.JsonBase64VertexOutputFormat;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.junit.Test;

Modified: giraph/trunk/src/test/java/org/apache/giraph/TestVertexTypes.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/test/java/org/apache/giraph/TestVertexTypes.java?rev=1371498&r1=1371497&r2=1371498&view=diff
==============================================================================
--- giraph/trunk/src/test/java/org/apache/giraph/TestVertexTypes.java (original)
+++ giraph/trunk/src/test/java/org/apache/giraph/TestVertexTypes.java Thu Aug  9 21:43:36 2012
@@ -18,7 +18,7 @@
 
 package org.apache.giraph;
 
-import org.apache.giraph.examples.GeneratedVertexInputFormat;
+import org.apache.giraph.io.GeneratedVertexInputFormat;
 import org.apache.giraph.examples.SimpleSuperstepVertex.SimpleSuperstepVertexInputFormat;
 import org.apache.giraph.graph.EdgeListVertex;
 import org.apache.giraph.graph.GiraphJob;
@@ -27,8 +27,8 @@ import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexCombiner;
 import org.apache.giraph.graph.VertexInputFormat;
 import org.apache.giraph.graph.VertexOutputFormat;
-import org.apache.giraph.lib.JsonBase64VertexInputFormat;
-import org.apache.giraph.lib.JsonBase64VertexOutputFormat;
+import org.apache.giraph.io.JsonBase64VertexInputFormat;
+import org.apache.giraph.io.JsonBase64VertexOutputFormat;
 import org.apache.giraph.utils.EmptyIterable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.DoubleWritable;
@@ -166,7 +166,7 @@ public class TestVertexTypes {
         } catch (IllegalArgumentException e) {
         }
     }
-    
+
     @Test
     public void testMismatchingCombiner() throws SecurityException,
             NoSuchMethodException, NoSuchFieldException {

Modified: giraph/trunk/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTest.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTest.java?rev=1371498&r1=1371497&r2=1371498&view=diff
==============================================================================
--- giraph/trunk/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTest.java (original)
+++ giraph/trunk/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTest.java Thu Aug  9 21:43:36 2012
@@ -23,6 +23,8 @@ import static org.junit.Assert.assertTru
 
 import java.util.Set;
 
+import org.apache.giraph.io.IntIntNullIntTextInputFormat;
+import org.apache.giraph.io.IdWithValueTextOutputFormat;
 import org.apache.giraph.utils.InternalVertexRunner;
 import org.junit.Test;
 
@@ -66,7 +68,7 @@ public class ConnectedComponentsVertexTe
                 ConnectedComponentsVertex.class,
                 MinimumIntCombiner.class,
                 IntIntNullIntTextInputFormat.class,
-                VertexWithComponentTextOutputFormat.class,
+                IdWithValueTextOutputFormat.class,
                 Maps.<String,String>newHashMap(), graph);
 
         SetMultimap<Integer,Integer> components = parseResults(results);

Modified: giraph/trunk/src/test/java/org/apache/giraph/examples/SimpleShortestPathsVertexTest.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/test/java/org/apache/giraph/examples/SimpleShortestPathsVertexTest.java?rev=1371498&r1=1371497&r2=1371498&view=diff
==============================================================================
--- giraph/trunk/src/test/java/org/apache/giraph/examples/SimpleShortestPathsVertexTest.java (original)
+++ giraph/trunk/src/test/java/org/apache/giraph/examples/SimpleShortestPathsVertexTest.java Thu Aug  9 21:43:36 2012
@@ -18,8 +18,8 @@
 
 package org.apache.giraph.examples;
 
-import org.apache.giraph.lib.JsonLongDoubleFloatDoubleVertexInputFormat;
-import org.apache.giraph.lib.JsonLongDoubleFloatDoubleVertexOutputFormat;
+import org.apache.giraph.io.JsonLongDoubleFloatDoubleVertexInputFormat;
+import org.apache.giraph.io.JsonLongDoubleFloatDoubleVertexOutputFormat;
 import org.apache.giraph.utils.InternalVertexRunner;
 import org.apache.giraph.utils.MockUtils;
 import org.apache.hadoop.io.DoubleWritable;

Modified: giraph/trunk/src/test/java/org/apache/giraph/examples/SimpleTriangleClosingVertexTest.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/test/java/org/apache/giraph/examples/SimpleTriangleClosingVertexTest.java?rev=1371498&r1=1371497&r2=1371498&view=diff
==============================================================================
--- giraph/trunk/src/test/java/org/apache/giraph/examples/SimpleTriangleClosingVertexTest.java (original)
+++ giraph/trunk/src/test/java/org/apache/giraph/examples/SimpleTriangleClosingVertexTest.java Thu Aug  9 21:43:36 2012
@@ -24,21 +24,11 @@ import static org.junit.Assert.assertNot
 import org.junit.Test;
 
 import org.apache.giraph.utils.MockUtils;
-import org.apache.giraph.lib.IdWithValueTextOutputFormat;
-import org.apache.giraph.examples.IntIntNullIntTextInputFormat;
-import org.apache.giraph.examples.SimpleTriangleClosingVertex.IntArrayListWritable;
 
-import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.NullWritable;
 
-import java.util.Map;
-import java.util.List;
-
-import org.mockito.Mockito;
-
 import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 
 /**
  * Contains a simple unit test for {@link SimpleTriangleClosingVertex}
@@ -58,7 +48,7 @@ public class SimpleTriangleClosingVertex
       new SimpleTriangleClosingVertex.IntArrayListWritable();
     vertex.initialize(null, null, null, null);
     vertex.addEdge(new IntWritable(5), NullWritable.get());
-    vertex.addEdge(new IntWritable(7), NullWritable.get()); 
+    vertex.addEdge(new IntWritable(7), NullWritable.get());
 
     MockUtils.MockedEnvironment<IntWritable,
       SimpleTriangleClosingVertex.IntArrayListWritable,

Modified: giraph/trunk/src/test/java/org/apache/giraph/examples/TryMultiRpcBindingPortsTest.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/test/java/org/apache/giraph/examples/TryMultiRpcBindingPortsTest.java?rev=1371498&r1=1371497&r2=1371498&view=diff
==============================================================================
--- giraph/trunk/src/test/java/org/apache/giraph/examples/TryMultiRpcBindingPortsTest.java (original)
+++ giraph/trunk/src/test/java/org/apache/giraph/examples/TryMultiRpcBindingPortsTest.java Thu Aug  9 21:43:36 2012
@@ -25,6 +25,8 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.giraph.graph.GiraphJob;
+import org.apache.giraph.io.IntIntNullIntTextInputFormat;
+import org.apache.giraph.io.IdWithValueTextOutputFormat;
 import org.apache.giraph.utils.InternalVertexRunner;
 import org.junit.Test;
 
@@ -71,8 +73,8 @@ public class TryMultiRpcBindingPortsTest
                 ConnectedComponentsVertex.class,
                 MinimumIntCombiner.class,
                 IntIntNullIntTextInputFormat.class,
-                VertexWithComponentTextOutputFormat.class,
-                params, 
+                IdWithValueTextOutputFormat.class,
+                params,
                 graph);
 
         SetMultimap<Integer,Integer> components = parseResults(results);

Added: giraph/trunk/src/test/java/org/apache/giraph/io/TestAdjacencyListTextVertexOutputFormat.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/test/java/org/apache/giraph/io/TestAdjacencyListTextVertexOutputFormat.java?rev=1371498&view=auto
==============================================================================
--- giraph/trunk/src/test/java/org/apache/giraph/io/TestAdjacencyListTextVertexOutputFormat.java (added)
+++ giraph/trunk/src/test/java/org/apache/giraph/io/TestAdjacencyListTextVertexOutputFormat.java Thu Aug  9 21:43:36 2012
@@ -0,0 +1,128 @@
+/*
+ * 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.io;
+
+import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.io.AdjacencyListTextVertexOutputFormat.AdjacencyListVertexWriter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.junit.Test;
+import org.mockito.Matchers;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+
+public class TestAdjacencyListTextVertexOutputFormat {
+
+  @Test
+  public void testVertexWithNoEdges() throws IOException, InterruptedException {
+    Configuration conf = new Configuration();
+    TaskAttemptContext tac = mock(TaskAttemptContext.class);
+    when(tac.getConfiguration()).thenReturn(conf);
+
+    Vertex vertex = mock(Vertex.class);
+    when(vertex.getId()).thenReturn(new Text("The Beautiful South"));
+    when(vertex.getValue()).thenReturn(new DoubleWritable(32.2d));
+    // Create empty iterable == no edges
+    when(vertex.getEdges()).thenReturn(new ArrayList<Text>());
+
+    RecordWriter<Text, Text> tw = mock(RecordWriter.class);
+    AdjacencyListVertexWriter writer = new AdjacencyListVertexWriter(tw);
+    writer.initialize(tac);
+    writer.writeVertex(vertex);
+
+    Text expected = new Text("The Beautiful South\t32.2");
+    verify(tw).write(expected, null);
+    verify(vertex, times(1)).getEdges();
+    verify(vertex, times(0)).getEdgeValue(Matchers.<WritableComparable>any());
+  }
+
+  @Test
+  public void testVertexWithEdges() throws IOException, InterruptedException {
+    Configuration conf = new Configuration();
+    TaskAttemptContext tac = mock(TaskAttemptContext.class);
+    when(tac.getConfiguration()).thenReturn(conf);
+
+    Vertex vertex = mock(Vertex.class);
+    when(vertex.getId()).thenReturn(new Text("San Francisco"));
+    when(vertex.getValue()).thenReturn(new DoubleWritable(0d));
+    when(vertex.getTotalNumEdges()).thenReturn(2l);
+    ArrayList<Edge<Text, DoubleWritable>> cities = new ArrayList<Edge<Text,
+        DoubleWritable>>();
+    Collections.addAll(cities,
+        new Edge<Text, DoubleWritable>(
+            new Text("Los Angeles"), new DoubleWritable(347.16)),
+        new Edge<Text, DoubleWritable>(
+            new Text("Phoenix"), new DoubleWritable(652.48)));
+
+    when(vertex.getEdges()).thenReturn(cities);
+
+    RecordWriter<Text,Text> tw = mock(RecordWriter.class);
+    AdjacencyListVertexWriter writer = new AdjacencyListVertexWriter(tw);
+    writer.initialize(tac);
+    writer.writeVertex(vertex);
+
+    Text expected = new Text("San Francisco\t0.0\tLos Angeles\t347.16\t" +
+            "Phoenix\t652.48");
+    verify(tw).write(expected, null);
+    verify(vertex, times(1)).getEdges();
+  }
+
+  @Test
+  public void testWithDifferentDelimiter() throws IOException, InterruptedException {
+    Configuration conf = new Configuration();
+    conf.set(AdjacencyListVertexWriter.LINE_TOKENIZE_VALUE, ":::");
+    TaskAttemptContext tac = mock(TaskAttemptContext.class);
+    when(tac.getConfiguration()).thenReturn(conf);
+
+    Vertex vertex = mock(Vertex.class);
+    when(vertex.getId()).thenReturn(new Text("San Francisco"));
+    when(vertex.getValue()).thenReturn(new DoubleWritable(0d));
+    when(vertex.getTotalNumEdges()).thenReturn(2l);
+    ArrayList<Edge<Text, DoubleWritable>> cities = new ArrayList<Edge<Text,
+        DoubleWritable>>();
+    Collections.addAll(cities,
+        new Edge<Text, DoubleWritable>(
+            new Text("Los Angeles"), new DoubleWritable(347.16)),
+        new Edge<Text, DoubleWritable>(
+            new Text("Phoenix"), new DoubleWritable(652.48)));
+
+    when(vertex.getEdges()).thenReturn(cities);
+
+    RecordWriter<Text,Text> tw = mock(RecordWriter.class);
+    AdjacencyListVertexWriter writer = new AdjacencyListVertexWriter(tw);
+    writer.initialize(tac);
+    writer.writeVertex(vertex);
+
+    Text expected = new Text("San Francisco:::0.0:::Los Angeles:::347.16:::" +
+            "Phoenix:::652.48");
+    verify(tw).write(expected, null);
+    verify(vertex, times(1)).getEdges();
+  }
+}

Added: giraph/trunk/src/test/java/org/apache/giraph/io/TestIdWithValueTextOutputFormat.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/test/java/org/apache/giraph/io/TestIdWithValueTextOutputFormat.java?rev=1371498&view=auto
==============================================================================
--- giraph/trunk/src/test/java/org/apache/giraph/io/TestIdWithValueTextOutputFormat.java (added)
+++ giraph/trunk/src/test/java/org/apache/giraph/io/TestIdWithValueTextOutputFormat.java Thu Aug  9 21:43:36 2012
@@ -0,0 +1,91 @@
+/*
+ * 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.io;
+
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.io.IdWithValueTextOutputFormat.IdWithValueVertexWriter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.junit.Test;
+import org.mockito.Matchers;
+
+import static org.apache.giraph.io.IdWithValueTextOutputFormat.IdWithValueVertexWriter.LINE_TOKENIZE_VALUE;
+import static org.apache.giraph.io.IdWithValueTextOutputFormat.IdWithValueVertexWriter.REVERSE_ID_AND_VALUE;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.ArrayList;
+
+public class TestIdWithValueTextOutputFormat {
+  @Test
+  public void testHappyPath() throws IOException, InterruptedException {
+    Configuration conf = new Configuration();
+    Text expected = new Text("Four Tops\t4.0");
+
+    IdWithValueTestWorker(conf, expected);
+  }
+
+  @Test
+  public void testReverseIdAndValue() throws IOException, InterruptedException {
+    Configuration conf = new Configuration();
+    conf.setBoolean(REVERSE_ID_AND_VALUE, true);
+    Text expected = new Text("4.0\tFour Tops");
+
+    IdWithValueTestWorker(conf, expected);
+  }
+
+  @Test
+  public void testWithDifferentDelimiter()  throws IOException,
+      InterruptedException {
+    Configuration conf = new Configuration();
+    conf.set(LINE_TOKENIZE_VALUE, "blah");
+    Text expected = new Text("Four Topsblah4.0");
+
+    IdWithValueTestWorker(conf, expected);
+  }
+
+  private void IdWithValueTestWorker(Configuration conf, Text expected)
+      throws IOException, InterruptedException {
+    TaskAttemptContext tac = mock(TaskAttemptContext.class);
+    when(tac.getConfiguration()).thenReturn(conf);
+
+    Vertex vertex = mock(Vertex.class);
+    when(vertex.getId()).thenReturn(new Text("Four Tops"));
+    when(vertex.getValue()).thenReturn(new DoubleWritable(4d));
+
+    // Create empty iterator == no edges
+    when(vertex.getEdges()).thenReturn(new ArrayList<Text>());
+
+    RecordWriter<Text, Text> tw = mock(RecordWriter.class);
+    IdWithValueVertexWriter writer = new IdWithValueVertexWriter(tw);
+    writer.initialize(tac);
+    writer.writeVertex(vertex);
+
+    verify(tw).write(expected, null);
+    verify(vertex, times(0)).getEdges();
+    verify(vertex, times(0)).getEdgeValue(Matchers.<WritableComparable>any());
+  }
+}

Added: giraph/trunk/src/test/java/org/apache/giraph/io/TestLongDoubleDoubleAdjacencyListVertexInputFormat.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/test/java/org/apache/giraph/io/TestLongDoubleDoubleAdjacencyListVertexInputFormat.java?rev=1371498&view=auto
==============================================================================
--- giraph/trunk/src/test/java/org/apache/giraph/io/TestLongDoubleDoubleAdjacencyListVertexInputFormat.java (added)
+++ giraph/trunk/src/test/java/org/apache/giraph/io/TestLongDoubleDoubleAdjacencyListVertexInputFormat.java Thu Aug  9 21:43:36 2012
@@ -0,0 +1,154 @@
+/*
+ * 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.io;
+
+
+import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.EdgeListVertex;
+import org.apache.giraph.graph.GiraphJob;
+import org.apache.giraph.graph.GraphState;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.giraph.io.TestTextDoubleDoubleAdjacencyListVertexInputFormat.assertValidVertex;
+import static org.apache.giraph.io.TestTextDoubleDoubleAdjacencyListVertexInputFormat.setGraphState;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+
+public class TestLongDoubleDoubleAdjacencyListVertexInputFormat {
+
+  private RecordReader<LongWritable, Text> rr;
+  private Configuration conf;
+  private TaskAttemptContext tac;
+  private GraphState<LongWritable, DoubleWritable, DoubleWritable, BooleanWritable> graphState;
+
+  @Before
+  public void setUp() throws IOException, InterruptedException {
+    rr = mock(RecordReader.class);
+    when(rr.nextKeyValue()).thenReturn(true);
+    conf = new Configuration();
+    conf.setClass(GiraphJob.VERTEX_CLASS, DummyVertex.class, Vertex.class);
+    conf.setClass(GiraphJob.VERTEX_ID_CLASS, LongWritable.class, Writable.class);
+    conf.setClass(GiraphJob.VERTEX_VALUE_CLASS, DoubleWritable.class, Writable.class);
+    graphState = mock(GraphState.class);
+    tac = mock(TaskAttemptContext.class);
+    when(tac.getConfiguration()).thenReturn(conf);
+  }
+
+  @Test
+  public void testIndexMustHaveValue() throws IOException, InterruptedException {
+    String input = "123";
+
+    when(rr.getCurrentValue()).thenReturn(new Text(input));
+    LongDoubleDoubleAdjacencyListVertexInputFormat.VertexReader<BooleanWritable> vr =
+        new LongDoubleDoubleAdjacencyListVertexInputFormat.VertexReader<BooleanWritable>(rr);
+
+    vr.initialize(null, tac);
+
+    try {
+      vr.nextVertex();
+      vr.getCurrentVertex();
+      fail("Should have thrown an IllegalArgumentException");
+    } catch (IllegalArgumentException iae) {
+      assertTrue(iae.getMessage().startsWith("Line did not split correctly: "));
+    }
+  }
+
+  @Test
+  public void testEdgesMustHaveValues() throws IOException, InterruptedException {
+    String input = "99\t55.2\t100";
+
+    when(rr.getCurrentValue()).thenReturn(new Text(input));
+    LongDoubleDoubleAdjacencyListVertexInputFormat.VertexReader vr =
+        new LongDoubleDoubleAdjacencyListVertexInputFormat.VertexReader(rr);
+
+    vr.initialize(null, tac);
+
+    try {
+      vr.nextVertex();
+      vr.getCurrentVertex();
+      fail("Should have thrown an IllegalArgumentException");
+    } catch (IllegalArgumentException iae) {
+      assertTrue(iae.getMessage().startsWith("Line did not split correctly: "));
+    }
+  }
+
+  @Test
+  public void testHappyPath() throws Exception {
+    String input = "42\t0.1\t99\t0.2\t2000\t0.3\t4000\t0.4";
+
+    when(rr.getCurrentValue()).thenReturn(new Text(input));
+    LongDoubleDoubleAdjacencyListVertexInputFormat.VertexReader<BooleanWritable> vr =
+        new LongDoubleDoubleAdjacencyListVertexInputFormat.VertexReader<BooleanWritable>(rr);
+
+    vr.initialize(null, tac);
+
+    assertTrue("Should have been able to read vertex", vr.nextVertex());
+    Vertex<LongWritable, DoubleWritable, DoubleWritable, BooleanWritable>
+        vertex = vr.getCurrentVertex();
+    setGraphState(vertex, graphState);
+    assertValidVertex(conf, graphState, vertex,
+        new LongWritable(42), new DoubleWritable(0.1),
+        new Edge<LongWritable, DoubleWritable>(new LongWritable(99), new DoubleWritable(0.2)),
+        new Edge<LongWritable, DoubleWritable>(new LongWritable(2000), new DoubleWritable(0.3)),
+        new Edge<LongWritable, DoubleWritable>(new LongWritable(4000), new DoubleWritable(0.4)));
+    assertEquals(vertex.getNumEdges(), 3);
+  }
+
+  @Test
+  public void testDifferentSeparators() throws Exception {
+    String input = "12345:42.42:9999999:99.9";
+
+    when(rr.getCurrentValue()).thenReturn(new Text(input));
+    conf.set(AdjacencyListVertexReader.LINE_TOKENIZE_VALUE, ":");
+    LongDoubleDoubleAdjacencyListVertexInputFormat.VertexReader<BooleanWritable> vr =
+        new LongDoubleDoubleAdjacencyListVertexInputFormat.VertexReader<BooleanWritable>(rr);
+
+    vr.initialize(null, tac);
+    assertTrue("Should have been able to read vertex", vr.nextVertex());
+    Vertex<LongWritable, DoubleWritable, DoubleWritable, BooleanWritable>
+        vertex = vr.getCurrentVertex();
+    setGraphState(vertex, graphState);
+    assertValidVertex(conf, graphState, vertex, new LongWritable(12345), new DoubleWritable(42.42),
+       new Edge<LongWritable, DoubleWritable>(new LongWritable(9999999), new DoubleWritable(99.9)));
+    assertEquals(vertex.getNumEdges(), 1);
+  }
+
+  public static class DummyVertex
+      extends EdgeListVertex<LongWritable, DoubleWritable,
+      DoubleWritable, BooleanWritable> {
+    @Override
+    public void compute(Iterable<BooleanWritable> messages) throws IOException {
+      // ignore
+    }
+  }
+}

Added: giraph/trunk/src/test/java/org/apache/giraph/io/TestTextDoubleDoubleAdjacencyListVertexInputFormat.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/test/java/org/apache/giraph/io/TestTextDoubleDoubleAdjacencyListVertexInputFormat.java?rev=1371498&view=auto
==============================================================================
--- giraph/trunk/src/test/java/org/apache/giraph/io/TestTextDoubleDoubleAdjacencyListVertexInputFormat.java (added)
+++ giraph/trunk/src/test/java/org/apache/giraph/io/TestTextDoubleDoubleAdjacencyListVertexInputFormat.java Thu Aug  9 21:43:36 2012
@@ -0,0 +1,228 @@
+/*
+ * 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.io;
+
+import org.apache.giraph.graph.BspUtils;
+import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.EdgeListVertex;
+import org.apache.giraph.graph.GiraphJob;
+import org.apache.giraph.graph.GraphState;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class TestTextDoubleDoubleAdjacencyListVertexInputFormat {
+
+  private RecordReader<LongWritable, Text> rr;
+  private Configuration conf;
+  private TaskAttemptContext tac;
+  private GraphState<Text, DoubleWritable, DoubleWritable, BooleanWritable> graphState;
+
+  @Before
+  public void setUp() throws IOException, InterruptedException {
+    rr = mock(RecordReader.class);
+    when(rr.nextKeyValue()).thenReturn(true).thenReturn(false);
+    conf = new Configuration();
+    conf.setClass(GiraphJob.VERTEX_CLASS, DummyVertex.class, Vertex.class);
+    conf.setClass(GiraphJob.VERTEX_ID_CLASS, Text.class, Writable.class);
+    conf.setClass(GiraphJob.VERTEX_VALUE_CLASS, DoubleWritable.class, Writable.class);
+    graphState = mock(GraphState.class);
+    tac = mock(TaskAttemptContext.class);
+    when(tac.getConfiguration()).thenReturn(conf);
+  }
+
+  @Test
+  public void testIndexMustHaveValue() throws IOException, InterruptedException {
+    String input = "hi";
+
+    when(rr.getCurrentValue()).thenReturn(new Text(input));
+    TextDoubleDoubleAdjacencyListVertexInputFormat.VertexReader<BooleanWritable> vr =
+        new TextDoubleDoubleAdjacencyListVertexInputFormat.VertexReader<BooleanWritable>(rr);
+
+    vr.initialize(null, tac);
+
+    try {
+      vr.nextVertex();
+      vr.getCurrentVertex();
+      fail("Should have thrown an IllegalArgumentException");
+    } catch (IllegalArgumentException iae) {
+      assertTrue(iae.getMessage().startsWith("Line did not split correctly: "));
+    }
+  }
+
+  @Test
+  public void testEdgesMustHaveValues() throws IOException, InterruptedException {
+    String input = "index\t55.66\tindex2";
+
+    when(rr.getCurrentValue()).thenReturn(new Text(input));
+    TextDoubleDoubleAdjacencyListVertexInputFormat.VertexReader<BooleanWritable> vr =
+        new TextDoubleDoubleAdjacencyListVertexInputFormat.VertexReader<BooleanWritable>(rr);
+    vr.initialize(null, tac);
+    try {
+      vr.nextVertex();
+      vr.getCurrentVertex();
+      fail("Should have thrown an IllegalArgumentException");
+    } catch (IllegalArgumentException iae) {
+      assertTrue(iae.getMessage().startsWith("Line did not split correctly: "));
+    }
+  }
+
+  public static void setGraphState(Vertex vertex, GraphState graphState) throws Exception {
+    Class<? extends Vertex> c = Vertex.class;
+    Method m = c.getDeclaredMethod("setGraphState", GraphState.class);
+    m.setAccessible(true);
+    m.invoke(vertex, graphState);
+  }
+
+  public static <I extends WritableComparable, V extends Writable,
+      E extends Writable, M extends Writable> void assertValidVertex(Configuration conf,
+      GraphState<I, V, E, M> graphState, Vertex<I, V, E, M> actual,
+      I expectedId, V expectedValue, Edge<I, E>... edges)
+      throws Exception {
+    Vertex<I, V, E, M> expected = BspUtils.createVertex(conf);
+    setGraphState(expected, graphState);
+
+    // FIXME! maybe can't work if not instantiated properly
+    Map<I, E> edgeMap = Maps.newHashMap();
+    for(Edge<I, E> edge : edges) {
+      edgeMap.put(edge.getTargetVertexId(), edge.getValue());
+    }
+    expected.initialize(expectedId, expectedValue, edgeMap, null);
+    assertValid(expected, actual);
+  }
+
+  public static
+  <I extends WritableComparable, V extends Writable, E extends Writable, M extends Writable> void
+  assertValid(Vertex<I, V, E, M> expected, Vertex<I, V, E, M> actual) {
+    assertEquals(expected.getId(), actual.getId());
+    assertEquals(expected.getValue(), actual.getValue());
+    assertEquals(expected.getTotalNumEdges(), actual.getTotalNumEdges());
+    List<Edge<I, E>> expectedEdges = Lists.newArrayList();
+    List<Edge<I, E>> actualEdges = Lists.newArrayList();
+    Iterables.addAll(actualEdges, actual.getEdges());
+    Iterables.addAll(expectedEdges, expected.getEdges());
+    Collections.sort(expectedEdges);
+    Collections.sort(actualEdges);
+    for(int i = 0; i < expectedEdges.size(); i++) {
+      assertEquals(expectedEdges.get(i), actualEdges.get(i));
+    }
+  }
+
+  @Test
+  public void testHappyPath() throws Exception {
+    String input = "Hi\t0\tCiao\t1.123\tBomdia\t2.234\tOla\t3.345";
+
+    when(rr.getCurrentValue()).thenReturn(new Text(input));
+    TextDoubleDoubleAdjacencyListVertexInputFormat.VertexReader<BooleanWritable> vr =
+        new TextDoubleDoubleAdjacencyListVertexInputFormat.VertexReader<BooleanWritable>(rr);
+
+    vr.initialize(null, tac);
+    assertTrue("Should have been able to add a vertex", vr.nextVertex());
+    Vertex<Text, DoubleWritable, DoubleWritable, BooleanWritable> vertex =
+        vr.getCurrentVertex();
+    setGraphState(vertex, graphState);
+    assertValidVertex(conf, graphState, vertex, new Text("Hi"), new DoubleWritable(0),
+        new Edge<Text, DoubleWritable>(new Text("Ciao"), new DoubleWritable(1.123d)),
+        new Edge<Text, DoubleWritable>(new Text("Bomdia"), new DoubleWritable(2.234d)),
+        new Edge<Text, DoubleWritable>(new Text("Ola"), new DoubleWritable(3.345d)));
+    assertEquals(vertex.getNumEdges(), 3);
+  }
+
+  @Test
+  public void testLineSanitizer() throws Exception {
+    String input = "Bye\t0.01\tCiao\t1.001\tTchau\t2.0001\tAdios\t3.00001";
+
+    AdjacencyListVertexReader.LineSanitizer toUpper =
+        new AdjacencyListVertexReader.LineSanitizer() {
+      @Override
+      public String sanitize(String s) {
+        return s.toUpperCase();
+      }
+    };
+
+    when(rr.getCurrentValue()).thenReturn(new Text(input));
+    TextDoubleDoubleAdjacencyListVertexInputFormat.VertexReader<BooleanWritable> vr =
+        new TextDoubleDoubleAdjacencyListVertexInputFormat.VertexReader<BooleanWritable>(rr, toUpper);
+
+    vr.initialize(null, tac);
+    assertTrue("Should have been able to read vertex", vr.nextVertex());
+    Vertex<Text, DoubleWritable, DoubleWritable, BooleanWritable> vertex =
+        vr.getCurrentVertex();
+    setGraphState(vertex, graphState);
+    assertValidVertex(conf, graphState, vertex,
+        new Text("BYE"), new DoubleWritable(0.01d),
+        new Edge<Text, DoubleWritable>(new Text("CIAO"), new DoubleWritable(1.001d)),
+        new Edge<Text, DoubleWritable>(new Text("TCHAU"), new DoubleWritable(2.0001d)),
+        new Edge<Text, DoubleWritable>(new Text("ADIOS"), new DoubleWritable(3.00001d)));
+
+    assertEquals(vertex.getNumEdges(), 3);
+  }
+
+  @Test
+  public void testDifferentSeparators() throws Exception {
+    String input = "alpha:42:beta:99";
+
+    when(rr.getCurrentValue()).thenReturn(new Text(input));
+    conf.set(AdjacencyListVertexReader.LINE_TOKENIZE_VALUE, ":");
+    TextDoubleDoubleAdjacencyListVertexInputFormat.VertexReader<BooleanWritable> vr =
+        new TextDoubleDoubleAdjacencyListVertexInputFormat.VertexReader<BooleanWritable>(rr);
+
+    vr.initialize(null, tac);
+    assertTrue("Should have been able to read vertex", vr.nextVertex());
+    Vertex<Text, DoubleWritable, DoubleWritable, BooleanWritable> vertex =
+        vr.getCurrentVertex();
+    setGraphState(vertex, graphState);
+    assertValidVertex(conf, graphState, vertex, new Text("alpha"), new DoubleWritable(42d),
+        new Edge<Text, DoubleWritable>(new Text("beta"), new DoubleWritable(99d)));
+    assertEquals(vertex.getNumEdges(), 1);
+  }
+
+  public static class DummyVertex
+      extends EdgeListVertex<Text, DoubleWritable,
+      DoubleWritable, BooleanWritable> {
+    @Override
+    public void compute(Iterable<BooleanWritable> messages) throws IOException {
+      // ignore
+    }
+  }
+}