diff --git a/examples/src/main/java/spark/examples/JavaPageRank.java b/examples/src/main/java/spark/examples/JavaPageRank.java
index 8e58bf24337c47f6b7aa44356fd9b0b6bb5fee94..9d90ef91748bc9bed3109d5d49e92f467f571858 100644
--- a/examples/src/main/java/spark/examples/JavaPageRank.java
+++ b/examples/src/main/java/spark/examples/JavaPageRank.java
@@ -23,6 +23,7 @@ import spark.api.java.JavaRDD;
 import spark.api.java.JavaSparkContext;
 import spark.api.java.function.FlatMapFunction;
 import spark.api.java.function.Function;
+import spark.api.java.function.PairFlatMapFunction;
 import spark.api.java.function.PairFunction;
 
 import java.util.List;
@@ -83,7 +84,7 @@ public class JavaPageRank {
     for (int current = 0; current < Integer.parseInt(args[2]); current++) {
       // Calculates URL contributions to the rank of other URLs.
       JavaPairRDD<String, Double> contribs = links.join(ranks).values()
-        .flatMap(new FlatMapFunction<Tuple2<List<String>, Double>, Tuple2<String, Double>>() {
+        .flatMap(new PairFlatMapFunction<Tuple2<List<String>, Double>, String, Double>() {
           @Override
           public Iterable<Tuple2<String, Double>> call(Tuple2<List<String>, Double> s) {
             List<Tuple2<String, Double>> results = new ArrayList<Tuple2<String, Double>>();
@@ -93,11 +94,6 @@ public class JavaPageRank {
 
             return results;
           }
-      }).map(new PairFunction<Tuple2<String, Double>, String, Double>() {
-        @Override
-        public Tuple2<String, Double> call(Tuple2<String, Double> s) {
-          return s;
-        }
       });
 
       // Re-calculates URL ranks based on neighbor contributions.