aboutsummaryrefslogtreecommitdiff
path: root/streaming/src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java
blob: 9b7701003d8d0811ed7284b59bd0862b215f7714 (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
/*
 * 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;

import java.io.Serializable;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Set;

import scala.Tuple2;

import com.google.common.collect.Sets;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.function.Function;
import org.apache.spark.api.java.function.VoidFunction;
import org.apache.spark.streaming.api.java.JavaDStream;
import org.apache.spark.util.ManualClock;
import org.junit.Assert;
import org.junit.Test;

import org.apache.spark.HashPartitioner;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.Optional;
import org.apache.spark.api.java.function.Function3;
import org.apache.spark.api.java.function.Function4;
import org.apache.spark.streaming.api.java.JavaPairDStream;
import org.apache.spark.streaming.api.java.JavaMapWithStateDStream;

public class JavaMapWithStateSuite extends LocalJavaStreamingContext implements Serializable {

  /**
   * This test is only for testing the APIs. It's not necessary to run it.
   */
  public void testAPI() {
    JavaPairRDD<String, Boolean> initialRDD = null;
    JavaPairDStream<String, Integer> wordsDstream = null;

    Function4<Time, String, Optional<Integer>, State<Boolean>, Optional<Double>> mappingFunc =
        new Function4<Time, String, Optional<Integer>, State<Boolean>, Optional<Double>>() {
          @Override
          public Optional<Double> call(
              Time time, String word, Optional<Integer> one, State<Boolean> state) {
            // Use all State's methods here
            state.exists();
            state.get();
            state.isTimingOut();
            state.remove();
            state.update(true);
            return Optional.of(2.0);
          }
        };

    JavaMapWithStateDStream<String, Integer, Boolean, Double> stateDstream =
        wordsDstream.mapWithState(
            StateSpec.function(mappingFunc)
                .initialState(initialRDD)
                .numPartitions(10)
                .partitioner(new HashPartitioner(10))
                .timeout(Durations.seconds(10)));

    stateDstream.stateSnapshots();

    Function3<String, Optional<Integer>, State<Boolean>, Double> mappingFunc2 =
        new Function3<String, Optional<Integer>, State<Boolean>, Double>() {
          @Override
          public Double call(String key, Optional<Integer> one, State<Boolean> state) {
            // Use all State's methods here
            state.exists();
            state.get();
            state.isTimingOut();
            state.remove();
            state.update(true);
            return 2.0;
          }
        };

    JavaMapWithStateDStream<String, Integer, Boolean, Double> stateDstream2 =
        wordsDstream.mapWithState(
            StateSpec.function(mappingFunc2)
                .initialState(initialRDD)
                .numPartitions(10)
                .partitioner(new HashPartitioner(10))
                .timeout(Durations.seconds(10)));

    stateDstream2.stateSnapshots();
  }

  @Test
  public void testBasicFunction() {
    List<List<String>> inputData = Arrays.asList(
        Collections.<String>emptyList(),
        Arrays.asList("a"),
        Arrays.asList("a", "b"),
        Arrays.asList("a", "b", "c"),
        Arrays.asList("a", "b"),
        Arrays.asList("a"),
        Collections.<String>emptyList()
    );

    List<Set<Integer>> outputData = Arrays.asList(
        Collections.<Integer>emptySet(),
        Sets.newHashSet(1),
        Sets.newHashSet(2, 1),
        Sets.newHashSet(3, 2, 1),
        Sets.newHashSet(4, 3),
        Sets.newHashSet(5),
        Collections.<Integer>emptySet()
    );

    @SuppressWarnings("unchecked")
    List<Set<Tuple2<String, Integer>>> stateData = Arrays.asList(
        Collections.<Tuple2<String, Integer>>emptySet(),
        Sets.newHashSet(new Tuple2<>("a", 1)),
        Sets.newHashSet(new Tuple2<>("a", 2), new Tuple2<>("b", 1)),
        Sets.newHashSet(new Tuple2<>("a", 3), new Tuple2<>("b", 2), new Tuple2<>("c", 1)),
        Sets.newHashSet(new Tuple2<>("a", 4), new Tuple2<>("b", 3), new Tuple2<>("c", 1)),
        Sets.newHashSet(new Tuple2<>("a", 5), new Tuple2<>("b", 3), new Tuple2<>("c", 1)),
        Sets.newHashSet(new Tuple2<>("a", 5), new Tuple2<>("b", 3), new Tuple2<>("c", 1))
    );

    Function3<String, Optional<Integer>, State<Integer>, Integer> mappingFunc =
        new Function3<String, Optional<Integer>, State<Integer>, Integer>() {
          @Override
          public Integer call(String key, Optional<Integer> value, State<Integer> state) {
            int sum = value.orElse(0) + (state.exists() ? state.get() : 0);
            state.update(sum);
            return sum;
          }
        };
    testOperation(
        inputData,
        StateSpec.function(mappingFunc),
        outputData,
        stateData);
  }

  private <K, S, T> void testOperation(
      List<List<K>> input,
      StateSpec<K, Integer, S, T> mapWithStateSpec,
      List<Set<T>> expectedOutputs,
      List<Set<Tuple2<K, S>>> expectedStateSnapshots) {
    int numBatches = expectedOutputs.size();
    JavaDStream<K> inputStream = JavaTestUtils.attachTestInputStream(ssc, input, 2);
    JavaMapWithStateDStream<K, Integer, S, T> mapWithStateDStream =
        JavaPairDStream.fromJavaDStream(inputStream.map(new Function<K, Tuple2<K, Integer>>() {
          @Override
          public Tuple2<K, Integer> call(K x) {
            return new Tuple2<>(x, 1);
          }
        })).mapWithState(mapWithStateSpec);

    final List<Set<T>> collectedOutputs =
        Collections.synchronizedList(new ArrayList<Set<T>>());
    mapWithStateDStream.foreachRDD(new VoidFunction<JavaRDD<T>>() {
      @Override
      public void call(JavaRDD<T> rdd) {
        collectedOutputs.add(Sets.newHashSet(rdd.collect()));
      }
    });
    final List<Set<Tuple2<K, S>>> collectedStateSnapshots =
        Collections.synchronizedList(new ArrayList<Set<Tuple2<K, S>>>());
    mapWithStateDStream.stateSnapshots().foreachRDD(new VoidFunction<JavaPairRDD<K, S>>() {
      @Override
      public void call(JavaPairRDD<K, S> rdd) {
        collectedStateSnapshots.add(Sets.newHashSet(rdd.collect()));
      }
    });
    BatchCounter batchCounter = new BatchCounter(ssc.ssc());
    ssc.start();
    ((ManualClock) ssc.ssc().scheduler().clock())
        .advance(ssc.ssc().progressListener().batchDuration() * numBatches + 1);
    batchCounter.waitUntilBatchesCompleted(numBatches, 10000);

    Assert.assertEquals(expectedOutputs, collectedOutputs);
    Assert.assertEquals(expectedStateSnapshots, collectedStateSnapshots);
  }
}