aboutsummaryrefslogtreecommitdiff
path: root/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java
blob: 868df64e8c94449ad5cd39d1c85f9ba68b636fd7 (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
/*
 * 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.spark.streaming.kafka;

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

import scala.Tuple2;

import kafka.serializer.StringDecoder;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;

import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.function.Function;
import org.apache.spark.api.java.function.VoidFunction;
import org.apache.spark.storage.StorageLevel;
import org.apache.spark.streaming.Duration;
import org.apache.spark.streaming.api.java.JavaDStream;
import org.apache.spark.streaming.api.java.JavaPairDStream;
import org.apache.spark.streaming.api.java.JavaStreamingContext;

public class JavaKafkaStreamSuite implements Serializable {
  private transient JavaStreamingContext ssc = null;
  private transient Random random = new Random();
  private transient KafkaTestUtils kafkaTestUtils = null;

  @Before
  public void setUp() {
    kafkaTestUtils = new KafkaTestUtils();
    kafkaTestUtils.setup();
    SparkConf sparkConf = new SparkConf()
      .setMaster("local[4]").setAppName(this.getClass().getSimpleName());
    ssc = new JavaStreamingContext(sparkConf, new Duration(500));
  }

  @After
  public void tearDown() {
    if (ssc != null) {
      ssc.stop();
      ssc = null;
    }

    if (kafkaTestUtils != null) {
      kafkaTestUtils.teardown();
      kafkaTestUtils = null;
    }
  }

  @Test
  public void testKafkaStream() throws InterruptedException {
    String topic = "topic1";
    Map<String, Integer> topics = new HashMap<>();
    topics.put(topic, 1);

    Map<String, Integer> sent = new HashMap<>();
    sent.put("a", 5);
    sent.put("b", 3);
    sent.put("c", 10);

    kafkaTestUtils.createTopic(topic, 1);
    kafkaTestUtils.sendMessages(topic, sent);

    Map<String, String> kafkaParams = new HashMap<>();
    kafkaParams.put("zookeeper.connect", kafkaTestUtils.zkAddress());
    kafkaParams.put("group.id", "test-consumer-" + random.nextInt(10000));
    kafkaParams.put("auto.offset.reset", "smallest");

    JavaPairDStream<String, String> stream = KafkaUtils.createStream(ssc,
      String.class,
      String.class,
      StringDecoder.class,
      StringDecoder.class,
      kafkaParams,
      topics,
      StorageLevel.MEMORY_ONLY_SER());

    final Map<String, Long> result = Collections.synchronizedMap(new HashMap<String, Long>());

    JavaDStream<String> words = stream.map(
      new Function<Tuple2<String, String>, String>() {
        @Override
        public String call(Tuple2<String, String> tuple2) {
          return tuple2._2();
        }
      }
    );

    words.countByValue().foreachRDD(new VoidFunction<JavaPairRDD<String, Long>>() {
        @Override
        public void call(JavaPairRDD<String, Long> rdd) {
          List<Tuple2<String, Long>> ret = rdd.collect();
          for (Tuple2<String, Long> r : ret) {
            if (result.containsKey(r._1())) {
              result.put(r._1(), result.get(r._1()) + r._2());
            } else {
              result.put(r._1(), r._2());
            }
          }
        }
      }
    );

    ssc.start();

    long startTime = System.currentTimeMillis();
    boolean sizeMatches = false;
    while (!sizeMatches && System.currentTimeMillis() - startTime < 20000) {
      sizeMatches = sent.size() == result.size();
      Thread.sleep(200);
    }
    Assert.assertEquals(sent.size(), result.size());
    for (Map.Entry<String, Integer> e : sent.entrySet()) {
      Assert.assertEquals(e.getValue().intValue(), result.get(e.getKey()).intValue());
    }
  }
}