diff options
author | Matei Zaharia <matei@eecs.berkeley.edu> | 2012-08-12 23:10:19 +0200 |
---|---|---|
committer | Matei Zaharia <matei@eecs.berkeley.edu> | 2012-08-12 23:10:19 +0200 |
commit | 6ae3c375a9e8a97ff96649b3637e4a011d849990 (patch) | |
tree | d455d4435336f64d1cf0a2f5e540f4424e2d7628 /examples/src/main/java | |
parent | 0141879c400732242ca90c41ae44e85dfe546db8 (diff) | |
download | spark-6ae3c375a9e8a97ff96649b3637e4a011d849990.tar.gz spark-6ae3c375a9e8a97ff96649b3637e4a011d849990.tar.bz2 spark-6ae3c375a9e8a97ff96649b3637e4a011d849990.zip |
Renamed apply() to call() in Java API and allowed it to throw Exceptions
Diffstat (limited to 'examples/src/main/java')
-rw-r--r-- | examples/src/main/java/spark/examples/JavaHdfsLR.java | 9 | ||||
-rw-r--r-- | examples/src/main/java/spark/examples/JavaTC.java | 16 | ||||
-rw-r--r-- | examples/src/main/java/spark/examples/JavaWordCount.java | 18 |
3 files changed, 22 insertions, 21 deletions
diff --git a/examples/src/main/java/spark/examples/JavaHdfsLR.java b/examples/src/main/java/spark/examples/JavaHdfsLR.java index c7a6b4405a..71fc13fbce 100644 --- a/examples/src/main/java/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/spark/examples/JavaHdfsLR.java @@ -26,8 +26,7 @@ public class JavaHdfsLR { } static class ParsePoint extends Function<String, DataPoint> { - - public DataPoint apply(String line) { + public DataPoint call(String line) { StringTokenizer tok = new StringTokenizer(line, " "); double y = Double.parseDouble(tok.nextToken()); double[] x = new double[D]; @@ -41,8 +40,7 @@ public class JavaHdfsLR { } static class VectorSum extends Function2<double[], double[], double[]> { - - public double[] apply(double[] a, double[] b) { + public double[] call(double[] a, double[] b) { double[] result = new double[D]; for (int j = 0; j < D; j++) { result[j] = a[j] + b[j]; @@ -52,14 +50,13 @@ public class JavaHdfsLR { } static class ComputeGradient extends Function<DataPoint, double[]> { - double[] weights; public ComputeGradient(double[] weights) { this.weights = weights; } - public double[] apply(DataPoint p) { + public double[] call(DataPoint p) { double[] gradient = new double[D]; for (int i = 0; i < D; i++) { double dot = dot(weights, p.x); diff --git a/examples/src/main/java/spark/examples/JavaTC.java b/examples/src/main/java/spark/examples/JavaTC.java index e6ca69ff97..25a465ec8e 100644 --- a/examples/src/main/java/spark/examples/JavaTC.java +++ b/examples/src/main/java/spark/examples/JavaTC.java @@ -11,6 +11,9 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +/** + * Transitive closure on a graph, implemented in Java. + */ public class JavaTC { static int numEdges = 200; @@ -32,7 +35,7 @@ public class JavaTC { Integer, Integer> { static ProjectFn INSTANCE = new ProjectFn(); - public Tuple2<Integer, Integer> apply(Tuple2<Integer, Tuple2<Integer, Integer>> triple) { + public Tuple2<Integer, Integer> call(Tuple2<Integer, Tuple2<Integer, Integer>> triple) { return new Tuple2<Integer, Integer>(triple._2()._2(), triple._2()._1()); } } @@ -53,12 +56,11 @@ public class JavaTC { // the graph to obtain the path (x, z). // Because join() joins on keys, the edges are stored in reversed order. - JavaPairRDD<Integer, Integer> edges = tc.map(new PairFunction<Tuple2<Integer, Integer>, - Integer, Integer>() { - @Override - public Tuple2<Integer, Integer> apply(Tuple2<Integer, Integer> e) { - return new Tuple2<Integer, Integer>(e._2(), e._1()); - } + JavaPairRDD<Integer, Integer> edges = tc.map( + new PairFunction<Tuple2<Integer, Integer>, Integer, Integer>() { + public Tuple2<Integer, Integer> call(Tuple2<Integer, Integer> e) { + return new Tuple2<Integer, Integer>(e._2(), e._1()); + } }); long oldCount = 0; diff --git a/examples/src/main/java/spark/examples/JavaWordCount.java b/examples/src/main/java/spark/examples/JavaWordCount.java index fb2feec09d..a44cf8a120 100644 --- a/examples/src/main/java/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/spark/examples/JavaWordCount.java @@ -12,9 +12,7 @@ import java.util.Arrays; import java.util.List; public class JavaWordCount { - public static void main(String[] args) throws Exception { - if (args.length < 2) { System.err.println("Usage: JavaWordCount <master> <file>"); System.exit(1); @@ -23,16 +21,20 @@ public class JavaWordCount { JavaSparkContext ctx = new JavaSparkContext(args[0], "JavaWordCount"); JavaRDD<String> lines = ctx.textFile(args[1], 1); - JavaPairRDD<String, Integer> counts = lines.flatMap(new FlatMapFunction<String, String>() { - public Iterable<String> apply(String s) { + JavaRDD<String> words = lines.flatMap(new FlatMapFunction<String, String>() { + public Iterable<String> call(String s) { return Arrays.asList(s.split(" ")); } - }).map(new PairFunction<String, String, Integer>() { - public Tuple2<String, Integer> apply(String s) { + }); + + JavaPairRDD<String, Integer> ones = words.map(new PairFunction<String, String, Integer>() { + public Tuple2<String, Integer> call(String s) { return new Tuple2(s, 1); } - }).reduceByKey(new Function2<Integer, Integer, Integer>() { - public Integer apply(Integer i1, Integer i2) { + }); + + JavaPairRDD<String, Integer> counts = ones.reduceByKey(new Function2<Integer, Integer, Integer>() { + public Integer call(Integer i1, Integer i2) { return i1 + i2; } }); |