aboutsummaryrefslogtreecommitdiff
path: root/core/src/test/scala/spark/JavaAPISuite.java
blob: 0b5354774b06e7e728446a58046451d69185ae18 (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
package spark;

import java.io.File;
import java.io.IOException;
import java.io.Serializable;
import java.util.*;

import scala.Tuple2;

import com.google.common.base.Charsets;
import com.google.common.io.Files;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.SequenceFileInputFormat;
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
import org.apache.hadoop.mapreduce.Job;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;

import spark.api.java.JavaDoubleRDD;
import spark.api.java.JavaPairRDD;
import spark.api.java.JavaRDD;
import spark.api.java.JavaSparkContext;
import spark.api.java.function.*;
import spark.partial.BoundedDouble;
import spark.partial.PartialResult;
import spark.storage.StorageLevel;
import spark.util.StatCounter;


// The test suite itself is Serializable so that anonymous Function implementations can be
// serialized, as an alternative to converting these anonymous classes to static inner classes;
// see http://stackoverflow.com/questions/758570/.
public class JavaAPISuite implements Serializable {
  private transient JavaSparkContext sc;

  @Before
  public void setUp() {
    sc = new JavaSparkContext("local", "JavaAPISuite");
  }

  @After
  public void tearDown() {
    sc.stop();
    sc = null;
    // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
    System.clearProperty("spark.master.port");
  }

  static class ReverseIntComparator implements Comparator<Integer>, Serializable {

    @Override
    public int compare(Integer a, Integer b) {
      if (a > b) return -1;
      else if (a < b) return 1;
      else return 0;
    }
  };

  @Test
  public void sparkContextUnion() {
    // Union of non-specialized JavaRDDs
    List<String> strings = Arrays.asList("Hello", "World");
    JavaRDD<String> s1 = sc.parallelize(strings);
    JavaRDD<String> s2 = sc.parallelize(strings);
    // Varargs
    JavaRDD<String> sUnion = sc.union(s1, s2);
    Assert.assertEquals(4, sUnion.count());
    // List
    List<JavaRDD<String>> list = new ArrayList<JavaRDD<String>>();
    list.add(s2);
    sUnion = sc.union(s1, list);
    Assert.assertEquals(4, sUnion.count());

    // Union of JavaDoubleRDDs
    List<Double> doubles = Arrays.asList(1.0, 2.0);
    JavaDoubleRDD d1 = sc.parallelizeDoubles(doubles);
    JavaDoubleRDD d2 = sc.parallelizeDoubles(doubles);
    JavaDoubleRDD dUnion = sc.union(d1, d2);
    Assert.assertEquals(4, dUnion.count());

    // Union of JavaPairRDDs
    List<Tuple2<Integer, Integer>> pairs = new ArrayList<Tuple2<Integer, Integer>>();
    pairs.add(new Tuple2<Integer, Integer>(1, 2));
    pairs.add(new Tuple2<Integer, Integer>(3, 4));
    JavaPairRDD<Integer, Integer> p1 = sc.parallelizePairs(pairs);
    JavaPairRDD<Integer, Integer> p2 = sc.parallelizePairs(pairs);
    JavaPairRDD<Integer, Integer> pUnion = sc.union(p1, p2);
    Assert.assertEquals(4, pUnion.count());
  }

  @Test
  public void sortByKey() {
    List<Tuple2<Integer, Integer>> pairs = new ArrayList<Tuple2<Integer, Integer>>();
    pairs.add(new Tuple2<Integer, Integer>(0, 4));
    pairs.add(new Tuple2<Integer, Integer>(3, 2));
    pairs.add(new Tuple2<Integer, Integer>(-1, 1));

    JavaPairRDD<Integer, Integer> rdd = sc.parallelizePairs(pairs);

    // Default comparator
    JavaPairRDD<Integer, Integer> sortedRDD = rdd.sortByKey();
    Assert.assertEquals(new Tuple2<Integer, Integer>(-1, 1), sortedRDD.first());
    List<Tuple2<Integer, Integer>> sortedPairs = sortedRDD.collect();
    Assert.assertEquals(new Tuple2<Integer, Integer>(0, 4), sortedPairs.get(1));
    Assert.assertEquals(new Tuple2<Integer, Integer>(3, 2), sortedPairs.get(2));

    // Custom comparator
    sortedRDD = rdd.sortByKey(new ReverseIntComparator(), false);
    Assert.assertEquals(new Tuple2<Integer, Integer>(-1, 1), sortedRDD.first());
    sortedPairs = sortedRDD.collect();
    Assert.assertEquals(new Tuple2<Integer, Integer>(0, 4), sortedPairs.get(1));
    Assert.assertEquals(new Tuple2<Integer, Integer>(3, 2), sortedPairs.get(2));
  }

  static int foreachCalls = 0;

  @Test
  public void foreach() {
    foreachCalls = 0;
    JavaRDD<String> rdd = sc.parallelize(Arrays.asList("Hello", "World"));
    rdd.foreach(new VoidFunction<String>() {
      @Override
      public void call(String s) {
        foreachCalls++;
      }
    });
    Assert.assertEquals(2, foreachCalls);
  }

  @Test
  public void lookup() {
    JavaPairRDD<String, String> categories = sc.parallelizePairs(Arrays.asList(
      new Tuple2<String, String>("Apples", "Fruit"),
      new Tuple2<String, String>("Oranges", "Fruit"),
      new Tuple2<String, String>("Oranges", "Citrus")
      ));
    Assert.assertEquals(2, categories.lookup("Oranges").size());
    Assert.assertEquals(2, categories.groupByKey().lookup("Oranges").get(0).size());
  }

  @Test
  public void groupBy() {
    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13));
    Function<Integer, Boolean> isOdd = new Function<Integer, Boolean>() {
      @Override
      public Boolean call(Integer x) {
        return x % 2 == 0;
      }
    };
    JavaPairRDD<Boolean, List<Integer>> oddsAndEvens = rdd.groupBy(isOdd);
    Assert.assertEquals(2, oddsAndEvens.count());
    Assert.assertEquals(2, oddsAndEvens.lookup(true).get(0).size());  // Evens
    Assert.assertEquals(5, oddsAndEvens.lookup(false).get(0).size()); // Odds

    oddsAndEvens = rdd.groupBy(isOdd, 1);
    Assert.assertEquals(2, oddsAndEvens.count());
    Assert.assertEquals(2, oddsAndEvens.lookup(true).get(0).size());  // Evens
    Assert.assertEquals(5, oddsAndEvens.lookup(false).get(0).size()); // Odds
  }

  @Test
  public void cogroup() {
    JavaPairRDD<String, String> categories = sc.parallelizePairs(Arrays.asList(
      new Tuple2<String, String>("Apples", "Fruit"),
      new Tuple2<String, String>("Oranges", "Fruit"),
      new Tuple2<String, String>("Oranges", "Citrus")
      ));
    JavaPairRDD<String, Integer> prices = sc.parallelizePairs(Arrays.asList(
      new Tuple2<String, Integer>("Oranges", 2),
      new Tuple2<String, Integer>("Apples", 3)
    ));
    JavaPairRDD<String, Tuple2<List<String>, List<Integer>>> cogrouped = categories.cogroup(prices);
    Assert.assertEquals("[Fruit, Citrus]", cogrouped.lookup("Oranges").get(0)._1().toString());
    Assert.assertEquals("[2]", cogrouped.lookup("Oranges").get(0)._2().toString());

    cogrouped.collect();
  }

  @Test
  public void foldReduce() {
    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13));
    Function2<Integer, Integer, Integer> add = new Function2<Integer, Integer, Integer>() {
      @Override
      public Integer call(Integer a, Integer b) {
        return a + b;
      }
    };

    int sum = rdd.fold(0, add);
    Assert.assertEquals(33, sum);

    sum = rdd.reduce(add);
    Assert.assertEquals(33, sum);
  }

  @Test
  public void reduceByKey() {
    List<Tuple2<Integer, Integer>> pairs = Arrays.asList(
      new Tuple2<Integer, Integer>(2, 1),
      new Tuple2<Integer, Integer>(2, 1),
      new Tuple2<Integer, Integer>(1, 1),
      new Tuple2<Integer, Integer>(3, 2),
      new Tuple2<Integer, Integer>(3, 1)
    );
    JavaPairRDD<Integer, Integer> rdd = sc.parallelizePairs(pairs);
    JavaPairRDD<Integer, Integer> counts = rdd.reduceByKey(
      new Function2<Integer, Integer, Integer>() {
        @Override
        public Integer call(Integer a, Integer b) {
         return a + b;
        }
    });
    Assert.assertEquals(1, counts.lookup(1).get(0).intValue());
    Assert.assertEquals(2, counts.lookup(2).get(0).intValue());
    Assert.assertEquals(3, counts.lookup(3).get(0).intValue());

    Map<Integer, Integer> localCounts = counts.collectAsMap();
    Assert.assertEquals(1, localCounts.get(1).intValue());
    Assert.assertEquals(2, localCounts.get(2).intValue());
    Assert.assertEquals(3, localCounts.get(3).intValue());

   localCounts = rdd.reduceByKeyLocally(new Function2<Integer, Integer,
      Integer>() {
      @Override
      public Integer call(Integer a, Integer b) {
        return a + b;
      }
    });
    Assert.assertEquals(1, localCounts.get(1).intValue());
    Assert.assertEquals(2, localCounts.get(2).intValue());
    Assert.assertEquals(3, localCounts.get(3).intValue());
  }

  @Test
  public void approximateResults() {
    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13));
    Map<Integer, Long> countsByValue = rdd.countByValue();
    Assert.assertEquals(2, countsByValue.get(1).longValue());
    Assert.assertEquals(1, countsByValue.get(13).longValue());

    PartialResult<Map<Integer, BoundedDouble>> approx = rdd.countByValueApprox(1);
    Map<Integer, BoundedDouble> finalValue = approx.getFinalValue();
    Assert.assertEquals(2.0, finalValue.get(1).mean(), 0.01);
    Assert.assertEquals(1.0, finalValue.get(13).mean(), 0.01);
  }

  @Test
  public void take() {
    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13));
    Assert.assertEquals(1, rdd.first().intValue());
    List<Integer> firstTwo = rdd.take(2);
    List<Integer> sample = rdd.takeSample(false, 2, 42);
  }

  @Test
  public void cartesian() {
    JavaDoubleRDD doubleRDD = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0));
    JavaRDD<String> stringRDD = sc.parallelize(Arrays.asList("Hello", "World"));
    JavaPairRDD<String, Double> cartesian = stringRDD.cartesian(doubleRDD);
    Assert.assertEquals(new Tuple2<String, Double>("Hello", 1.0), cartesian.first());
  }

  @Test
  public void javaDoubleRDD() {
    JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0));
    JavaDoubleRDD distinct = rdd.distinct();
    Assert.assertEquals(5, distinct.count());
    JavaDoubleRDD filter = rdd.filter(new Function<Double, Boolean>() {
      @Override
      public Boolean call(Double x) {
        return x > 2.0;
      }
    });
    Assert.assertEquals(3, filter.count());
    JavaDoubleRDD union = rdd.union(rdd);
    Assert.assertEquals(12, union.count());
    union = union.cache();
    Assert.assertEquals(12, union.count());

    Assert.assertEquals(20, rdd.sum(), 0.01);
    StatCounter stats = rdd.stats();
    Assert.assertEquals(20, stats.sum(), 0.01);
    Assert.assertEquals(20/6.0, rdd.mean(), 0.01);
    Assert.assertEquals(20/6.0, rdd.mean(), 0.01);
    Assert.assertEquals(6.22222, rdd.variance(), 0.01);
    Assert.assertEquals(2.49444, rdd.stdev(), 0.01);

    Double first = rdd.first();
    List<Double> take = rdd.take(5);
  }

  @Test
  public void map() {
    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
    JavaDoubleRDD doubles = rdd.map(new DoubleFunction<Integer>() {
      @Override
      public Double call(Integer x) {
        return 1.0 * x;
      }
    }).cache();
    JavaPairRDD<Integer, Integer> pairs = rdd.map(new PairFunction<Integer, Integer, Integer>() {
      @Override
      public Tuple2<Integer, Integer> call(Integer x) {
        return new Tuple2<Integer, Integer>(x, x);
      }
    }).cache();
    JavaRDD<String> strings = rdd.map(new Function<Integer, String>() {
      @Override
      public String call(Integer x) {
        return x.toString();
      }
    }).cache();
  }

  @Test
  public void flatMap() {
    JavaRDD<String> rdd = sc.parallelize(Arrays.asList("Hello World!",
      "The quick brown fox jumps over the lazy dog."));
    JavaRDD<String> words = rdd.flatMap(new FlatMapFunction<String, String>() {
      @Override
      public Iterable<String> call(String x) {
        return Arrays.asList(x.split(" "));
      }
    });
    Assert.assertEquals("Hello", words.first());
    Assert.assertEquals(11, words.count());

    JavaPairRDD<String, String> pairs = rdd.flatMap(
      new PairFlatMapFunction<String, String, String>() {

        @Override
        public Iterable<Tuple2<String, String>> call(String s) {
          List<Tuple2<String, String>> pairs = new LinkedList<Tuple2<String, String>>();
          for (String word : s.split(" ")) pairs.add(new Tuple2<String, String>(word, word));
          return pairs;
        }
      }
    );
    Assert.assertEquals(new Tuple2<String, String>("Hello", "Hello"), pairs.first());
    Assert.assertEquals(11, pairs.count());

    JavaDoubleRDD doubles = rdd.flatMap(new DoubleFlatMapFunction<String>() {
      @Override
      public Iterable<Double> call(String s) {
        List<Double> lengths = new LinkedList<Double>();
        for (String word : s.split(" ")) lengths.add(word.length() * 1.0);
        return lengths;
      }
    });
    Double x = doubles.first();
    Assert.assertEquals(5.0, doubles.first().doubleValue(), 0.01);
    Assert.assertEquals(11, pairs.count());
  }

  @Test
  public void mapPartitions() {
    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2);
    JavaRDD<Integer> partitionSums = rdd.mapPartitions(
      new FlatMapFunction<Iterator<Integer>, Integer>() {
        @Override
        public Iterable<Integer> call(Iterator<Integer> iter) {
          int sum = 0;
          while (iter.hasNext()) {
            sum += iter.next();
          }
          return Collections.singletonList(sum);
        }
    });
    Assert.assertEquals("[3, 7]", partitionSums.collect().toString());
  }

  @Test
  public void persist() {
    JavaDoubleRDD doubleRDD = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0));
    doubleRDD = doubleRDD.persist(StorageLevel.DISK_ONLY());
    Assert.assertEquals(20, doubleRDD.sum(), 0.1);

    List<Tuple2<Integer, String>> pairs = Arrays.asList(
      new Tuple2<Integer, String>(1, "a"),
      new Tuple2<Integer, String>(2, "aa"),
      new Tuple2<Integer, String>(3, "aaa")
    );
    JavaPairRDD<Integer, String> pairRDD = sc.parallelizePairs(pairs);
    pairRDD = pairRDD.persist(StorageLevel.DISK_ONLY());
    Assert.assertEquals("a", pairRDD.first()._2());

    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
    rdd = rdd.persist(StorageLevel.DISK_ONLY());
    Assert.assertEquals(1, rdd.first().intValue());
  }

  @Test
  public void iterator() {
    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2);
    TaskContext context = new TaskContext(0, 0, 0);
    Assert.assertEquals(1, rdd.iterator(rdd.splits().get(0), context).next().intValue());
  }

  @Test
  public void glom() {
    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2);
    Assert.assertEquals("[1, 2]", rdd.glom().first().toString());
  }

  // File input / output tests are largely adapted from FileSuite:

  @Test
  public void textFiles() throws IOException {
    File tempDir = Files.createTempDir();
    String outputDir = new File(tempDir, "output").getAbsolutePath();
    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4));
    rdd.saveAsTextFile(outputDir);
    // Read the plain text file and check it's OK
    File outputFile = new File(outputDir, "part-00000");
    String content = Files.toString(outputFile, Charsets.UTF_8);
    Assert.assertEquals("1\n2\n3\n4\n", content);
    // Also try reading it in as a text file RDD
    List<String> expected = Arrays.asList("1", "2", "3", "4");
    JavaRDD<String> readRDD = sc.textFile(outputDir);
    Assert.assertEquals(expected, readRDD.collect());
  }

  @Test
  public void sequenceFile() {
    File tempDir = Files.createTempDir();
    String outputDir = new File(tempDir, "output").getAbsolutePath();
    List<Tuple2<Integer, String>> pairs = Arrays.asList(
      new Tuple2<Integer, String>(1, "a"),
      new Tuple2<Integer, String>(2, "aa"),
      new Tuple2<Integer, String>(3, "aaa")
    );
    JavaPairRDD<Integer, String> rdd = sc.parallelizePairs(pairs);

    rdd.map(new PairFunction<Tuple2<Integer, String>, IntWritable, Text>() {
      @Override
      public Tuple2<IntWritable, Text> call(Tuple2<Integer, String> pair) {
        return new Tuple2<IntWritable, Text>(new IntWritable(pair._1()), new Text(pair._2()));
      }
    }).saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class);

    // Try reading the output back as an object file
    JavaPairRDD<Integer, String> readRDD = sc.sequenceFile(outputDir, IntWritable.class,
      Text.class).map(new PairFunction<Tuple2<IntWritable, Text>, Integer, String>() {
      @Override
      public Tuple2<Integer, String> call(Tuple2<IntWritable, Text> pair) {
        return new Tuple2<Integer, String>(pair._1().get(), pair._2().toString());
      }
    });
    Assert.assertEquals(pairs, readRDD.collect());
  }

  @Test
  public void writeWithNewAPIHadoopFile() {
    File tempDir = Files.createTempDir();
    String outputDir = new File(tempDir, "output").getAbsolutePath();
    List<Tuple2<Integer, String>> pairs = Arrays.asList(
      new Tuple2<Integer, String>(1, "a"),
      new Tuple2<Integer, String>(2, "aa"),
      new Tuple2<Integer, String>(3, "aaa")
    );
    JavaPairRDD<Integer, String> rdd = sc.parallelizePairs(pairs);

    rdd.map(new PairFunction<Tuple2<Integer, String>, IntWritable, Text>() {
      @Override
      public Tuple2<IntWritable, Text> call(Tuple2<Integer, String> pair) {
        return new Tuple2<IntWritable, Text>(new IntWritable(pair._1()), new Text(pair._2()));
      }
    }).saveAsNewAPIHadoopFile(outputDir, IntWritable.class, Text.class,
      org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat.class);

    JavaPairRDD<IntWritable, Text> output = sc.sequenceFile(outputDir, IntWritable.class,
      Text.class);
    Assert.assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>,
      String>() {
      @Override
      public String call(Tuple2<IntWritable, Text> x) {
        return x.toString();
      }
    }).collect().toString());
  }

  @Test
  public void readWithNewAPIHadoopFile() throws IOException {
    File tempDir = Files.createTempDir();
    String outputDir = new File(tempDir, "output").getAbsolutePath();
    List<Tuple2<Integer, String>> pairs = Arrays.asList(
      new Tuple2<Integer, String>(1, "a"),
      new Tuple2<Integer, String>(2, "aa"),
      new Tuple2<Integer, String>(3, "aaa")
    );
    JavaPairRDD<Integer, String> rdd = sc.parallelizePairs(pairs);

    rdd.map(new PairFunction<Tuple2<Integer, String>, IntWritable, Text>() {
      @Override
      public Tuple2<IntWritable, Text> call(Tuple2<Integer, String> pair) {
        return new Tuple2<IntWritable, Text>(new IntWritable(pair._1()), new Text(pair._2()));
      }
    }).saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class);

    JavaPairRDD<IntWritable, Text> output = sc.newAPIHadoopFile(outputDir,
      org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat.class, IntWritable.class,
      Text.class, new Job().getConfiguration());
    Assert.assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>,
      String>() {
      @Override
      public String call(Tuple2<IntWritable, Text> x) {
        return x.toString();
      }
    }).collect().toString());
  }

  @Test
  public void objectFilesOfInts() {
    File tempDir = Files.createTempDir();
    String outputDir = new File(tempDir, "output").getAbsolutePath();
    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4));
    rdd.saveAsObjectFile(outputDir);
    // Try reading the output back as an object file
    List<Integer> expected = Arrays.asList(1, 2, 3, 4);
    JavaRDD<Integer> readRDD = sc.objectFile(outputDir);
    Assert.assertEquals(expected, readRDD.collect());
  }

  @Test
  public void objectFilesOfComplexTypes() {
    File tempDir = Files.createTempDir();
    String outputDir = new File(tempDir, "output").getAbsolutePath();
    List<Tuple2<Integer, String>> pairs = Arrays.asList(
      new Tuple2<Integer, String>(1, "a"),
      new Tuple2<Integer, String>(2, "aa"),
      new Tuple2<Integer, String>(3, "aaa")
    );
    JavaPairRDD<Integer, String> rdd = sc.parallelizePairs(pairs);
    rdd.saveAsObjectFile(outputDir);
    // Try reading the output back as an object file
    JavaRDD<Tuple2<Integer, String>> readRDD = sc.objectFile(outputDir);
    Assert.assertEquals(pairs, readRDD.collect());
  }

  @Test
  public void hadoopFile() {
    File tempDir = Files.createTempDir();
    String outputDir = new File(tempDir, "output").getAbsolutePath();
    List<Tuple2<Integer, String>> pairs = Arrays.asList(
      new Tuple2<Integer, String>(1, "a"),
      new Tuple2<Integer, String>(2, "aa"),
      new Tuple2<Integer, String>(3, "aaa")
    );
    JavaPairRDD<Integer, String> rdd = sc.parallelizePairs(pairs);

    rdd.map(new PairFunction<Tuple2<Integer, String>, IntWritable, Text>() {
      @Override
      public Tuple2<IntWritable, Text> call(Tuple2<Integer, String> pair) {
        return new Tuple2<IntWritable, Text>(new IntWritable(pair._1()), new Text(pair._2()));
      }
    }).saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class);

    JavaPairRDD<IntWritable, Text> output = sc.hadoopFile(outputDir,
      SequenceFileInputFormat.class, IntWritable.class, Text.class);
    Assert.assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>,
      String>() {
      @Override
      public String call(Tuple2<IntWritable, Text> x) {
        return x.toString();
      }
    }).collect().toString());
  }

  @Test
  public void zip() {
    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
    JavaDoubleRDD doubles = rdd.map(new DoubleFunction<Integer>() {
      @Override
      public Double call(Integer x) {
        return 1.0 * x;
      }
    });
    JavaPairRDD<Integer, Double> zipped = rdd.zip(doubles);
    zipped.count();
  }

  @Test
  public void accumulators() {
    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));

    final Accumulator<Integer> intAccum = sc.intAccumulator(10);
    rdd.foreach(new VoidFunction<Integer>() {
      public void call(Integer x) {
        intAccum.add(x);
      }
    });
    Assert.assertEquals((Integer) 25, intAccum.value());

    final Accumulator<Double> doubleAccum = sc.doubleAccumulator(10.0);
    rdd.foreach(new VoidFunction<Integer>() {
      public void call(Integer x) {
        doubleAccum.add((double) x);
      }
    });
    Assert.assertEquals((Double) 25.0, doubleAccum.value());

    // Try a custom accumulator type
    AccumulatorParam<Float> floatAccumulatorParam = new AccumulatorParam<Float>() {
      public Float addInPlace(Float r, Float t) {
        return r + t;
      }

      public Float addAccumulator(Float r, Float t) {
        return r + t;
      }

      public Float zero(Float initialValue) {
        return 0.0f;
      }
    };

    final Accumulator<Float> floatAccum = sc.accumulator((Float) 10.0f, floatAccumulatorParam);
    rdd.foreach(new VoidFunction<Integer>() {
      public void call(Integer x) {
        floatAccum.add((float) x);
      }
    });
    Assert.assertEquals((Float) 25.0f, floatAccum.value());
  }

  @Test
  public void checkpointAndComputation() {
    File tempDir = Files.createTempDir();
    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
    sc.setCheckpointDir(tempDir.getAbsolutePath(), true);
    Assert.assertEquals(false, rdd.isCheckpointed());
    rdd.checkpoint();
    rdd.count(); // Forces the DAG to cause a checkpoint
    Assert.assertEquals(true, rdd.isCheckpointed());
    Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5), rdd.collect());
  }

  @Test
  public void checkpointAndRestore() {
    File tempDir = Files.createTempDir();
    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
    sc.setCheckpointDir(tempDir.getAbsolutePath(), true);
    Assert.assertEquals(false, rdd.isCheckpointed());
    rdd.checkpoint();
    rdd.count(); // Forces the DAG to cause a checkpoint
    Assert.assertEquals(true, rdd.isCheckpointed());

    Assert.assertTrue(rdd.getCheckpointFile().isPresent());
    JavaRDD<Integer> recovered = sc.checkpointFile(rdd.getCheckpointFile().get());
    Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5), recovered.collect());
  }
}