You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mahout.apache.org by ss...@apache.org on 2011/09/14 11:18:47 UTC

svn commit: r1170491 [2/2] - in /mahout/trunk/core/src: main/java/org/apache/mahout/common/ main/java/org/apache/mahout/graph/ main/java/org/apache/mahout/graph/common/ main/java/org/apache/mahout/graph/linkanalysis/ main/java/org/apache/mahout/graph/m...

Copied: mahout/trunk/core/src/test/java/org/apache/mahout/graph/common/EnumerateTrianglesJobTest.java (from r1170124, mahout/trunk/core/src/test/java/org/apache/mahout/graph/triangles/EnumerateTrianglesJobTest.java)
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/test/java/org/apache/mahout/graph/common/EnumerateTrianglesJobTest.java?p2=mahout/trunk/core/src/test/java/org/apache/mahout/graph/common/EnumerateTrianglesJobTest.java&p1=mahout/trunk/core/src/test/java/org/apache/mahout/graph/triangles/EnumerateTrianglesJobTest.java&r1=1170124&r2=1170491&rev=1170491&view=diff
==============================================================================
--- mahout/trunk/core/src/test/java/org/apache/mahout/graph/triangles/EnumerateTrianglesJobTest.java (original)
+++ mahout/trunk/core/src/test/java/org/apache/mahout/graph/common/EnumerateTrianglesJobTest.java Wed Sep 14 09:18:46 2011
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.mahout.graph.triangles;
+package org.apache.mahout.graph.common;
 
 import java.io.File;
 import java.util.Arrays;
@@ -34,7 +34,7 @@ import org.apache.mahout.common.MahoutTe
 import org.apache.mahout.common.Pair;
 import org.apache.mahout.common.iterator.sequencefile.SequenceFileIterable;
 import org.apache.mahout.graph.model.Triangle;
-import org.apache.mahout.graph.model.UndirectedEdgeWithDegrees;
+import org.apache.mahout.graph.model.UndirectedEdge;
 import org.apache.mahout.graph.model.Vertex;
 import org.easymock.EasyMock;
 import org.junit.Test;
@@ -42,6 +42,56 @@ import org.junit.Test;
 public class EnumerateTrianglesJobTest extends MahoutTestCase {
 
   @Test
+  public void testScatterEdges() throws Exception {
+    Mapper.Context ctx = EasyMock.createMock(Mapper.Context.class);
+
+    ctx.write(new Vertex(123), new Vertex(456));
+    ctx.write(new Vertex(456), new Vertex(123));
+
+    EasyMock.replay(ctx);
+
+    new EnumerateTrianglesJob.ScatterEdgesMapper()
+        .map(new UndirectedEdge(new Vertex(123), new Vertex(456)), null, ctx);
+
+    EasyMock.verify(ctx);
+  }
+
+  @Test
+  public void testSumDegrees() throws Exception {
+    Reducer.Context ctx = EasyMock.createMock(Reducer.Context.class);
+
+    Vertex vertex = new Vertex(1);
+
+    ctx.write(new UndirectedEdge(vertex, new Vertex(3)), new VertexWithDegree(vertex, 3));
+    ctx.write(new UndirectedEdge(vertex, new Vertex(5)), new VertexWithDegree(vertex, 3));
+    ctx.write(new UndirectedEdge(vertex, new Vertex(7)), new VertexWithDegree(vertex, 3));
+
+    EasyMock.replay(ctx);
+
+    new EnumerateTrianglesJob.SumDegreesReducer()
+        .reduce(vertex, Arrays.asList(new Vertex(3), new Vertex(5), new Vertex(7)), ctx);
+
+    EasyMock.verify(ctx);
+  }
+
+  @Test
+  public void testJoinDegrees() throws Exception {
+    Reducer.Context ctx = EasyMock.createMock(Reducer.Context.class);
+    Vertex first = new Vertex(1);
+    Vertex second = new Vertex(3);
+
+    ctx.write(new UndirectedEdgeWithDegrees(new VertexWithDegree(first, 1), new VertexWithDegree(second, 3)),
+        NullWritable.get());
+
+    EasyMock.replay(ctx);
+
+    new EnumerateTrianglesJob.JoinDegreesReducer().reduce(new UndirectedEdge(first, second),
+        Arrays.asList(new VertexWithDegree(first, 1), new VertexWithDegree(second, 3)), ctx);
+
+    EasyMock.verify(ctx);
+  }
+
+  @Test
   public void testScatterEdgesToLowerVertexDegree() throws Exception {
     Mapper.Context ctx = EasyMock.createMock(Mapper.Context.class);
 
@@ -100,7 +150,7 @@ public class EnumerateTrianglesJobTest e
 
   @Test
   public void toyIntegrationTest() throws Exception {
-    File inputFile = getTestTempFile("edgesWithDegrees.seq");
+    File inputFile = getTestTempFile("edges.seq");
     File outputDir = getTestTempDir("output");
     outputDir.delete();
     File tempDir = getTestTempDir("tmp");
@@ -109,21 +159,21 @@ public class EnumerateTrianglesJobTest e
     FileSystem fs = FileSystem.get(conf);
 
     SequenceFile.Writer writer = new SequenceFile.Writer(fs, conf, new Path(inputFile.getAbsolutePath().toString()),
-        UndirectedEdgeWithDegrees.class, NullWritable.class);
+        UndirectedEdge.class, NullWritable.class);
 
     try {
-      writer.append(new UndirectedEdgeWithDegrees(0, 7, 1, 3), NullWritable.get());
-      writer.append(new UndirectedEdgeWithDegrees(0, 7, 2, 3), NullWritable.get());
-      writer.append(new UndirectedEdgeWithDegrees(0, 7, 3, 3), NullWritable.get());
-      writer.append(new UndirectedEdgeWithDegrees(0, 7, 4, 3), NullWritable.get());
-      writer.append(new UndirectedEdgeWithDegrees(0, 7, 5, 2), NullWritable.get());
-      writer.append(new UndirectedEdgeWithDegrees(0, 7, 6, 1), NullWritable.get());
-      writer.append(new UndirectedEdgeWithDegrees(0, 7, 7, 2), NullWritable.get());
-      writer.append(new UndirectedEdgeWithDegrees(1, 3, 2, 3), NullWritable.get());
-      writer.append(new UndirectedEdgeWithDegrees(1, 3, 3, 3), NullWritable.get());
-      writer.append(new UndirectedEdgeWithDegrees(2, 3, 3, 3), NullWritable.get());
-      writer.append(new UndirectedEdgeWithDegrees(4, 3, 5, 2), NullWritable.get());
-      writer.append(new UndirectedEdgeWithDegrees(4, 3, 7, 2), NullWritable.get());
+      writer.append(new UndirectedEdge(0, 1), NullWritable.get());
+      writer.append(new UndirectedEdge(0, 2), NullWritable.get());
+      writer.append(new UndirectedEdge(0, 3), NullWritable.get());
+      writer.append(new UndirectedEdge(0, 4), NullWritable.get());
+      writer.append(new UndirectedEdge(0, 5), NullWritable.get());
+      writer.append(new UndirectedEdge(0, 6), NullWritable.get());
+      writer.append(new UndirectedEdge(0, 7), NullWritable.get());
+      writer.append(new UndirectedEdge(1, 2), NullWritable.get());
+      writer.append(new UndirectedEdge(1, 3), NullWritable.get());
+      writer.append(new UndirectedEdge(2, 3), NullWritable.get());
+      writer.append(new UndirectedEdge(4, 5), NullWritable.get());
+      writer.append(new UndirectedEdge(4, 7), NullWritable.get());
     } finally {
       Closeables.closeQuietly(writer);
     }

Modified: mahout/trunk/core/src/test/java/org/apache/mahout/graph/linkanalysis/PageRankJobTest.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/test/java/org/apache/mahout/graph/linkanalysis/PageRankJobTest.java?rev=1170491&r1=1170490&r2=1170491&view=diff
==============================================================================
--- mahout/trunk/core/src/test/java/org/apache/mahout/graph/linkanalysis/PageRankJobTest.java (original)
+++ mahout/trunk/core/src/test/java/org/apache/mahout/graph/linkanalysis/PageRankJobTest.java Wed Sep 14 09:18:46 2011
@@ -27,8 +27,8 @@ import org.apache.hadoop.io.NullWritable
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.mahout.common.iterator.FileLineIterable;
 import org.apache.mahout.graph.GraphTestCase;
-import org.apache.mahout.graph.common.GraphUtils;
 import org.apache.mahout.graph.model.Edge;
+import org.apache.mahout.graph.preprocessing.GraphUtils;
 import org.apache.mahout.math.DenseMatrix;
 import org.apache.mahout.math.Matrix;
 import org.apache.mahout.math.hadoop.MathHelper;
@@ -43,24 +43,6 @@ import java.util.Map;
 public class PageRankJobTest extends GraphTestCase {
 
   @Test
-  public void indexAndCountDegree() throws Exception {
-
-    Mapper.Context ctx = EasyMock.createMock(Mapper.Context.class);
-
-    ctx.write(new IntWritable(7), new IntWritable(1));
-
-    EasyMock.replay(ctx);
-
-    OpenLongIntHashMap index = new OpenLongIntHashMap();
-    index.put(123L, 7);
-    PageRankJob.IndexAndCountDegreeMapper indexAndCountDegreeMapper = new PageRankJob.IndexAndCountDegreeMapper();
-    setField(indexAndCountDegreeMapper, "vertexIDsToIndex", index);
-    indexAndCountDegreeMapper.map(new Edge(123L, 456L), NullWritable.get(), ctx);
-
-    EasyMock.verify(ctx);
-  }
-
-  @Test
   public void toyIntegrationTest() throws Exception {
 
     File verticesFile = getTestTempFile("vertices.txt");
@@ -92,42 +74,31 @@ public class PageRankJobTest extends Gra
     pageRank.run(new String[] { "--vertexIndex", indexedVerticesFile.getAbsolutePath(),
         "--edges", edgesFile.getAbsolutePath(), "--output", outputDir.getAbsolutePath(),
         "--numVertices", String.valueOf(numVertices), "--numIterations", String.valueOf(3),
-        "--teleportationProbability", String.valueOf(0.8), "--tempDir", tempDir.getAbsolutePath() });
+        "--stayingProbability", String.valueOf(0.8), "--tempDir", tempDir.getAbsolutePath() });
 
-    DenseMatrix expectedTransitionMatrix = new DenseMatrix(new double[][]{
-        { 0,           0.4, 0,   0 },
+    Matrix expectedAdjacenyMatrix = new DenseMatrix(new double[][] {
+        { 0,           0.4, 0,   0   },
         { 0.266666667, 0,   0,   0.4 },
         { 0.266666667, 0,   0.8, 0.4 },
-        { 0.266666667, 0.4, 0,   0 } });
+        { 0.266666667, 0.4, 0,   0   } });
 
-    Matrix actualTransitionMatrix = MathHelper.readMatrix(conf, new Path(tempDir.getAbsolutePath(),
-        "transitionMatrix/part-r-00000"), numVertices, numVertices);
+    Matrix actualAdjacencyMatrix = MathHelper.readMatrix(conf, new Path(tempDir.getAbsolutePath(),
+        "adjacencyMatrix/part-r-00000"), numVertices, numVertices);
 
-    assertEquals(expectedTransitionMatrix, actualTransitionMatrix);
+    assertMatrixEquals(expectedAdjacenyMatrix, actualAdjacencyMatrix);
 
     Map<Long,Double> rankPerVertex = Maps.newHashMap();
-    for (String line : new FileLineIterable(new File(outputDir, "part-r-00000"))) {
+    for (String line : new FileLineIterable(new File(outputDir, "part-m-00000"))) {
       String[] tokens = Iterables.toArray(Splitter.on("\t").split(line), String.class);
       rankPerVertex.put(Long.parseLong(tokens[0]), Double.parseDouble(tokens[1]));
     }
 
     assertEquals(4, rankPerVertex.size());
-    assertEquals(rankPerVertex.get(12l), 0.1206666, EPSILON);
-    assertEquals(rankPerVertex.get(34L), 0.1571111, EPSILON);
-    assertEquals(rankPerVertex.get(56L), 0.5651111, EPSILON);
-    assertEquals(rankPerVertex.get(78L), 0.1571111, EPSILON);
-
+    assertEquals(0.1206666, rankPerVertex.get(12L), EPSILON);
+    assertEquals(0.1571111, rankPerVertex.get(34L), EPSILON);
+    assertEquals(0.5651111, rankPerVertex.get(56L), EPSILON);
+    assertEquals(0.1571111, rankPerVertex.get(78L), EPSILON);
   }
 
-  void assertEquals(Matrix expected, Matrix actual) {
-    assertEquals(expected.numRows(), actual.numRows());
-    assertEquals(actual.numCols(), actual.numCols());
-    for (int row = 0; row < expected.numRows(); row++) {
-      for (int col = 0; col < expected.numCols(); col ++) {
-        assertEquals("Non-matching values in [" + row + "," + col + "]",
-            expected.get(row, col), actual.get(row, col), EPSILON);
-      }
-    }
-  }
 
 }

Added: mahout/trunk/core/src/test/java/org/apache/mahout/graph/linkanalysis/RandomWalkWithRestartJobTest.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/test/java/org/apache/mahout/graph/linkanalysis/RandomWalkWithRestartJobTest.java?rev=1170491&view=auto
==============================================================================
--- mahout/trunk/core/src/test/java/org/apache/mahout/graph/linkanalysis/RandomWalkWithRestartJobTest.java (added)
+++ mahout/trunk/core/src/test/java/org/apache/mahout/graph/linkanalysis/RandomWalkWithRestartJobTest.java Wed Sep 14 09:18:46 2011
@@ -0,0 +1,100 @@
+/**
+ * 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.mahout.graph.linkanalysis;
+
+import com.google.common.base.Splitter;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Maps;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.mahout.common.iterator.FileLineIterable;
+import org.apache.mahout.graph.GraphTestCase;
+import org.apache.mahout.graph.preprocessing.GraphUtils;
+import org.apache.mahout.graph.model.Edge;
+import org.apache.mahout.math.DenseMatrix;
+import org.apache.mahout.math.Matrix;
+import org.apache.mahout.math.hadoop.MathHelper;
+import org.junit.Test;
+
+import java.io.File;
+import java.util.Map;
+
+public class RandomWalkWithRestartJobTest extends GraphTestCase {
+
+  @Test
+  public void toyIntegrationTest() throws Exception {
+
+    File verticesFile = getTestTempFile("vertices.txt");
+    File edgesFile = getTestTempFile("edges.seq");
+    File indexedVerticesFile = getTestTempFile("indexedVertices.seq");
+    File outputDir = getTestTempDir("output");
+    outputDir.delete();
+    File tempDir = getTestTempDir();
+
+    Configuration conf = new Configuration();
+
+    writeLines(verticesFile, "12", "34", "56", "78");
+
+    writeComponents(edgesFile, conf, Edge.class,
+        new Edge(12, 34),
+        new Edge(12, 56),
+        new Edge(34, 34),
+        new Edge(34, 78),
+        new Edge(56, 12),
+        new Edge(56, 34),
+        new Edge(56, 56),
+        new Edge(56, 78),
+        new Edge(78, 34));
+
+    int numVertices = GraphUtils.indexVertices(conf, new Path(verticesFile.getAbsolutePath()),
+        new Path(indexedVerticesFile.getAbsolutePath()));
+
+    RandomWalkWithRestartJob randomWalkWithRestart = new RandomWalkWithRestartJob();
+    randomWalkWithRestart.setConf(conf);
+    randomWalkWithRestart.run(new String[]{"--vertexIndex", indexedVerticesFile.getAbsolutePath(),
+        "--edges", edgesFile.getAbsolutePath(), "--sourceVertexIndex", String.valueOf(2),
+        "--output", outputDir.getAbsolutePath(), "--numVertices", String.valueOf(numVertices),
+        "--numIterations", String.valueOf(2), "--stayingProbability", String.valueOf(0.75),
+        "--tempDir", tempDir.getAbsolutePath()});
+
+    Matrix expectedAdjacenyMatrix = new DenseMatrix(new double[][] {
+        { 0,     0,     0.1875, 0    },
+        { 0.375, 0.375, 0.1875, 0.75 },
+        { 0.375, 0,     0.1875, 0    },
+        { 0,     0.375, 0.1875, 0    } });
+
+    Matrix actualAdjacencyMatrix = MathHelper.readMatrix(conf, new Path(tempDir.getAbsolutePath(),
+        "adjacencyMatrix/part-r-00000"), numVertices, numVertices);
+
+    assertMatrixEquals(expectedAdjacenyMatrix, actualAdjacencyMatrix);
+
+    Map<Long,Double> steadyStateProbabilities = Maps.newHashMap();
+    for (String line : new FileLineIterable(new File(outputDir, "part-m-00000"))) {
+      String[] tokens = Iterables.toArray(Splitter.on("\t").split(line), String.class);
+      steadyStateProbabilities.put(Long.parseLong(tokens[0]), Double.parseDouble(tokens[1]));
+    }
+
+    assertEquals(4, steadyStateProbabilities.size());
+
+    assertEquals(steadyStateProbabilities.get(12L), 75.0 / 1024.0, EPSILON);
+    assertEquals(steadyStateProbabilities.get(34L), 363.0 / 1024.0, EPSILON);
+    assertEquals(steadyStateProbabilities.get(56L), 349.0 / 1024.0, EPSILON);
+    assertEquals(steadyStateProbabilities.get(78L), 237.0 / 1024.0, EPSILON);
+  }
+
+}

Added: mahout/trunk/core/src/test/java/org/apache/mahout/graph/preprocessing/AdjacencyMatrixJobTest.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/test/java/org/apache/mahout/graph/preprocessing/AdjacencyMatrixJobTest.java?rev=1170491&view=auto
==============================================================================
--- mahout/trunk/core/src/test/java/org/apache/mahout/graph/preprocessing/AdjacencyMatrixJobTest.java (added)
+++ mahout/trunk/core/src/test/java/org/apache/mahout/graph/preprocessing/AdjacencyMatrixJobTest.java Wed Sep 14 09:18:46 2011
@@ -0,0 +1,147 @@
+/**
+ * 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.mahout.graph.preprocessing;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.mahout.graph.GraphTestCase;
+import org.apache.mahout.graph.model.Edge;
+import org.apache.mahout.math.DenseMatrix;
+import org.apache.mahout.math.Matrix;
+import org.apache.mahout.math.function.DoubleFunction;
+import org.apache.mahout.math.hadoop.MathHelper;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+
+public class AdjacencyMatrixJobTest extends GraphTestCase {
+
+  File verticesFile;
+  File edgesFile;
+  File indexedVerticesFile;
+  File outputDir;
+  File tempDir;
+
+  int numVertices;
+  double stayingProbability;
+  Matrix expectedAdjacencyMatrix;
+
+  Configuration conf;
+
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    verticesFile = getTestTempFile("vertices.txt");
+    edgesFile = getTestTempFile("edges.seq");
+    indexedVerticesFile = getTestTempFile("indexedVertices.seq");
+    outputDir = getTestTempDir("output");
+    outputDir.delete();
+    tempDir = getTestTempDir();
+
+    conf = new Configuration();
+
+    writeLines(verticesFile, "12", "34", "56", "78");
+
+    writeComponents(edgesFile, conf, Edge.class,
+        new Edge(12, 34),
+        new Edge(12, 56),
+        new Edge(34, 34),
+        new Edge(34, 78),
+        new Edge(56, 12),
+        new Edge(56, 34),
+        new Edge(56, 56),
+        new Edge(56, 78),
+        new Edge(78, 34));
+
+    numVertices = GraphUtils.indexVertices(conf, new Path(verticesFile.getAbsolutePath()),
+        new Path(indexedVerticesFile.getAbsolutePath()));
+
+    expectedAdjacencyMatrix = new DenseMatrix(new double[][] {
+        { 0, 0, 1, 0 },
+        { 1, 1, 1, 1 },
+        { 1, 0, 1, 0 },
+        { 0, 1, 1, 0 } });
+
+    stayingProbability = 0.5;
+  }
+
+  @Test
+  public void adjacencyMatrix() throws Exception {
+    AdjacencyMatrixJob createAdjacencyMatrix = new AdjacencyMatrixJob();
+    createAdjacencyMatrix.setConf(conf);
+    createAdjacencyMatrix.run(new String[] { "--vertexIndex", indexedVerticesFile.getAbsolutePath(),
+        "--edges", edgesFile.getAbsolutePath(), "--output", outputDir.getAbsolutePath(),
+        "--numVertices", String.valueOf(numVertices), "--tempDir", tempDir.getAbsolutePath() });
+
+    Matrix actualAdjacencyMatrix = MathHelper.readMatrix(conf, new Path(outputDir.getAbsolutePath(), "part-r-00000"),
+        numVertices, numVertices);
+    assertMatrixEquals(expectedAdjacencyMatrix, actualAdjacencyMatrix);
+  }
+
+  @Test
+  public void substochastifiedAdjacencyMatrix() throws Exception {
+    AdjacencyMatrixJob createAdjacencyMatrix = new AdjacencyMatrixJob();
+    createAdjacencyMatrix.setConf(conf);
+    createAdjacencyMatrix.run(new String[] { "--vertexIndex", indexedVerticesFile.getAbsolutePath(),
+        "--edges", edgesFile.getAbsolutePath(), "--output", outputDir.getAbsolutePath(),
+        "--numVertices", String.valueOf(numVertices), "--substochastify", String.valueOf(true),
+        "--tempDir", tempDir.getAbsolutePath() });
+
+    Matrix actualAdjacencyMatrix = MathHelper.readMatrix(conf, new Path(outputDir.getAbsolutePath(), "part-r-00000"),
+        numVertices, numVertices);
+
+    substochastifyExpectedAdjacencyMatrix();
+
+    assertMatrixEquals(expectedAdjacencyMatrix, actualAdjacencyMatrix);
+  }
+
+  @Test
+  public void substochasitifiedAdjacencyMatrixWithTeleports() throws Exception {
+    AdjacencyMatrixJob createAdjacencyMatrix = new AdjacencyMatrixJob();
+    createAdjacencyMatrix.setConf(conf);
+    createAdjacencyMatrix.run(new String[] { "--vertexIndex", indexedVerticesFile.getAbsolutePath(),
+        "--edges", edgesFile.getAbsolutePath(), "--output", outputDir.getAbsolutePath(),
+        "--numVertices", String.valueOf(numVertices), "--substochastify", String.valueOf(true),
+        "--stayingProbability", String.valueOf(stayingProbability), "--tempDir", tempDir.getAbsolutePath() });
+
+    Matrix actualAdjacencyMatrix = MathHelper.readMatrix(conf, new Path(outputDir.getAbsolutePath(), "part-r-00000"),
+        numVertices, numVertices);
+
+    substochastifyExpectedAdjacencyMatrix();
+    adjustExpectedAdjacencyMatrixForTeleports();
+
+    assertMatrixEquals(expectedAdjacencyMatrix, actualAdjacencyMatrix);
+  }
+
+  private void substochastifyExpectedAdjacencyMatrix() {
+    for (int column = 0; column < expectedAdjacencyMatrix.numCols(); column++) {
+      expectedAdjacencyMatrix.assignColumn(column, expectedAdjacencyMatrix.viewColumn(column).normalize(1));
+    }
+  }
+
+  private void adjustExpectedAdjacencyMatrixForTeleports() {
+    expectedAdjacencyMatrix.assign(new DoubleFunction() {
+      @Override
+      public double apply(double val) {
+        return val * stayingProbability;
+      }
+    });
+  }
+
+}
\ No newline at end of file

Added: mahout/trunk/core/src/test/java/org/apache/mahout/graph/preprocessing/SimplifyGraphJobTest.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/test/java/org/apache/mahout/graph/preprocessing/SimplifyGraphJobTest.java?rev=1170491&view=auto
==============================================================================
--- mahout/trunk/core/src/test/java/org/apache/mahout/graph/preprocessing/SimplifyGraphJobTest.java (added)
+++ mahout/trunk/core/src/test/java/org/apache/mahout/graph/preprocessing/SimplifyGraphJobTest.java Wed Sep 14 09:18:46 2011
@@ -0,0 +1,126 @@
+/**
+ * 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.mahout.graph.preprocessing;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.Set;
+
+import com.google.common.collect.Sets;
+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.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.mahout.common.MahoutTestCase;
+import org.apache.mahout.common.Pair;
+import org.apache.mahout.common.iterator.sequencefile.SequenceFileIterable;
+import org.apache.mahout.graph.model.UndirectedEdge;
+import org.apache.mahout.graph.model.Vertex;
+import org.easymock.EasyMock;
+import org.junit.Test;
+
+public class SimplifyGraphJobTest extends MahoutTestCase {
+
+  @Test
+  public void testEdgeMapping() throws Exception {
+    Mapper.Context ctx = EasyMock.createMock(Mapper.Context.class);
+
+    ctx.write(new UndirectedEdge(new Vertex(123L), new Vertex(456L)), NullWritable.get());
+
+    EasyMock.replay(ctx);
+
+    new SimplifyGraphJob.SimplifyGraphMapper().map(null, new Text("123,456"), ctx);
+
+    EasyMock.verify(ctx);
+  }
+
+  @Test
+  public void testLoopRemoval() throws Exception {
+    Mapper.Context ctx = EasyMock.createMock(Mapper.Context.class);
+
+    EasyMock.replay(ctx);
+
+    new SimplifyGraphJob.SimplifyGraphMapper().map(null, new Text("123,123"), ctx);
+
+    EasyMock.verify(ctx);
+  }
+
+  @Test
+  public void testIgnoreUnparseableLines() throws Exception {
+    Mapper.Context ctx = EasyMock.createMock(Mapper.Context.class);
+
+    EasyMock.replay(ctx);
+
+    new SimplifyGraphJob.SimplifyGraphMapper().map(null, new Text("abc,123"), ctx);
+
+    EasyMock.verify(ctx);
+  }
+
+  @Test
+  public void testAggregation() throws Exception {
+    Reducer.Context ctx = EasyMock.createMock(Reducer.Context.class);
+    UndirectedEdge edge = new UndirectedEdge(new Vertex(123L), new Vertex(456L));
+
+    ctx.write(edge, NullWritable.get());
+
+    EasyMock.replay(ctx);
+
+    new SimplifyGraphJob.SimplifyGraphReducer().reduce(edge, Arrays.asList(NullWritable.get(), NullWritable.get()), ctx);
+
+    EasyMock.verify(ctx);
+  }
+
+  @Test
+  public void toyIntegrationTest() throws Exception {
+    File inputFile = getTestTempFile("graph.txt");
+    File outputDir = getTestTempDir("output");
+    outputDir.delete();
+    File tempDir = getTestTempDir("tmp");
+
+    writeLines(inputFile,
+        "0,0",
+        "0,1",
+        "1,0",
+        "1,0",
+        "2,3",
+        "4,3",
+        "4,2");
+
+    Configuration conf = new Configuration();
+    SimplifyGraphJob simplifyGraphJob = new SimplifyGraphJob();
+    simplifyGraphJob.setConf(conf);
+    simplifyGraphJob.run(new String[] { "--input", inputFile.getAbsolutePath(), "--output", outputDir.getAbsolutePath(),
+        "--tempDir", tempDir.getAbsolutePath() });
+
+    Set<UndirectedEdge> edges = Sets.newHashSet();
+    for (Pair<UndirectedEdge,NullWritable> result :
+        new SequenceFileIterable<UndirectedEdge, NullWritable>(new Path(outputDir.getAbsolutePath() + "/part-r-00000"),
+        false, conf)) {
+      edges.add(result.getFirst());
+    }
+
+    assertEquals(4, edges.size());
+    assertTrue(edges.contains(new UndirectedEdge(1, 0)));
+    assertTrue(edges.contains(new UndirectedEdge(2, 3)));
+    assertTrue(edges.contains(new UndirectedEdge(2, 4)));
+    assertTrue(edges.contains(new UndirectedEdge(3, 4)));
+  }
+
+}
\ No newline at end of file