forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
/
StreamGroupedReduceOperatorTest.java
163 lines (127 loc) · 5.7 KB
/
StreamGroupedReduceOperatorTest.java
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
/*
* 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:https://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.flink.streaming.api.operators;
import org.apache.flink.api.common.functions.ReduceFunction;
import org.apache.flink.api.common.functions.RichReduceFunction;
import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.base.IntSerializer;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
import org.apache.flink.streaming.util.TestHarnessUtil;
import org.junit.Assert;
import org.junit.Test;
import java.util.concurrent.ConcurrentLinkedQueue;
/**
* Tests for {@link StreamGroupedReduceOperator}. These test that:
*
* <ul>
* <li>RichFunction methods are called correctly</li>
* <li>Timestamps of processed elements match the input timestamp</li>
* <li>Watermarks are correctly forwarded</li>
* </ul>
*/
public class StreamGroupedReduceOperatorTest {
@Test
public void testGroupedReduce() throws Exception {
KeySelector<Integer, Integer> keySelector = new IntegerKeySelector();
StreamGroupedReduceOperator<Integer> operator = new StreamGroupedReduceOperator<>(new MyReducer(), IntSerializer.INSTANCE);
OneInputStreamOperatorTestHarness<Integer, Integer> testHarness =
new KeyedOneInputStreamOperatorTestHarness<>(operator, keySelector, BasicTypeInfo.INT_TYPE_INFO);
long initialTime = 0L;
ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
testHarness.open();
testHarness.processElement(new StreamRecord<>(1, initialTime + 1));
testHarness.processElement(new StreamRecord<>(1, initialTime + 2));
testHarness.processWatermark(new Watermark(initialTime + 2));
testHarness.processElement(new StreamRecord<>(2, initialTime + 3));
testHarness.processElement(new StreamRecord<>(2, initialTime + 4));
testHarness.processElement(new StreamRecord<>(3, initialTime + 5));
expectedOutput.add(new StreamRecord<>(1, initialTime + 1));
expectedOutput.add(new StreamRecord<>(2, initialTime + 2));
expectedOutput.add(new Watermark(initialTime + 2));
expectedOutput.add(new StreamRecord<>(2, initialTime + 3));
expectedOutput.add(new StreamRecord<>(4, initialTime + 4));
expectedOutput.add(new StreamRecord<>(3, initialTime + 5));
TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
}
@Test
public void testOpenClose() throws Exception {
KeySelector<Integer, Integer> keySelector = new IntegerKeySelector();
StreamGroupedReduceOperator<Integer> operator =
new StreamGroupedReduceOperator<>(new TestOpenCloseReduceFunction(), IntSerializer.INSTANCE);
OneInputStreamOperatorTestHarness<Integer, Integer> testHarness =
new KeyedOneInputStreamOperatorTestHarness<>(operator, keySelector, BasicTypeInfo.INT_TYPE_INFO);
long initialTime = 0L;
testHarness.open();
testHarness.processElement(new StreamRecord<>(1, initialTime));
testHarness.processElement(new StreamRecord<>(2, initialTime));
testHarness.close();
Assert.assertTrue("RichFunction methods where not called.", TestOpenCloseReduceFunction.closeCalled);
Assert.assertTrue("Output contains no elements.", testHarness.getOutput().size() > 0);
}
// This must only be used in one test, otherwise the static fields will be changed
// by several tests concurrently
private static class TestOpenCloseReduceFunction extends RichReduceFunction<Integer> {
private static final long serialVersionUID = 1L;
public static boolean openCalled = false;
public static boolean closeCalled = false;
@Override
public void open(Configuration parameters) throws Exception {
super.open(parameters);
if (closeCalled) {
Assert.fail("Close called before open.");
}
openCalled = true;
}
@Override
public void close() throws Exception {
super.close();
if (!openCalled) {
Assert.fail("Open was not called before close.");
}
closeCalled = true;
}
@Override
public Integer reduce(Integer in1, Integer in2) throws Exception {
if (!openCalled) {
Assert.fail("Open was not called before run.");
}
return in1 + in2;
}
}
// Utilities
private static class MyReducer implements ReduceFunction<Integer> {
private static final long serialVersionUID = 1L;
@Override
public Integer reduce(Integer value1, Integer value2) throws Exception {
return value1 + value2;
}
}
private static class IntegerKeySelector implements KeySelector<Integer, Integer> {
private static final long serialVersionUID = 1L;
@Override
public Integer getKey(Integer value) throws Exception {
return value;
}
}
private static TypeInformation<Integer> typeInfo = BasicTypeInfo.INT_TYPE_INFO;
}