forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 7
/
SinkWriterOperatorTestBase.java
255 lines (208 loc) · 9.7 KB
/
SinkWriterOperatorTestBase.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
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
/*
* 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.flink.streaming.runtime.operators.sink;
import org.apache.flink.api.common.typeutils.base.IntSerializer;
import org.apache.flink.api.connector.sink.Sink;
import org.apache.flink.api.connector.sink.SinkWriter;
import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
import org.apache.flink.util.TestLogger;
import org.junit.Test;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import static org.hamcrest.Matchers.contains;
import static org.hamcrest.Matchers.equalTo;
import static org.junit.Assert.assertThat;
/** Base class for Tests for subclasses of {@link AbstractSinkWriterOperator}. */
public abstract class SinkWriterOperatorTestBase extends TestLogger {
protected abstract AbstractSinkWriterOperatorFactory<Integer, String> createWriterOperator(
TestSink sink);
@Test
public void nonBufferingWriterEmitsWithoutFlush() throws Exception {
final long initialTime = 0;
final OneInputStreamOperatorTestHarness<Integer, String> testHarness =
createTestHarness(
TestSink.newBuilder()
.setWriter(new TestSink.DefaultSinkWriter())
.withWriterState()
.build());
testHarness.open();
testHarness.processWatermark(initialTime);
testHarness.processElement(1, initialTime + 1);
testHarness.processElement(2, initialTime + 2);
testHarness.prepareSnapshotPreBarrier(1L);
testHarness.snapshot(1L, 1L);
assertThat(
testHarness.getOutput(),
contains(
new Watermark(initialTime),
new StreamRecord<>(Tuple3.of(1, initialTime + 1, initialTime).toString()),
new StreamRecord<>(Tuple3.of(2, initialTime + 2, initialTime).toString())));
}
@Test
public void nonBufferingWriterEmitsOnFlush() throws Exception {
final long initialTime = 0;
final OneInputStreamOperatorTestHarness<Integer, String> testHarness =
createTestHarness(
TestSink.newBuilder()
.setWriter(new TestSink.DefaultSinkWriter())
.withWriterState()
.build());
testHarness.open();
testHarness.processWatermark(initialTime);
testHarness.processElement(1, initialTime + 1);
testHarness.processElement(2, initialTime + 2);
testHarness.endInput();
assertThat(
testHarness.getOutput(),
contains(
new Watermark(initialTime),
new StreamRecord<>(Tuple3.of(1, initialTime + 1, initialTime).toString()),
new StreamRecord<>(Tuple3.of(2, initialTime + 2, initialTime).toString())));
}
@Test
public void bufferingWriterDoesNotEmitWithoutFlush() throws Exception {
final long initialTime = 0;
final OneInputStreamOperatorTestHarness<Integer, String> testHarness =
createTestHarness(
TestSink.newBuilder()
.setWriter(new BufferingSinkWriter())
.withWriterState()
.build());
testHarness.open();
testHarness.processWatermark(initialTime);
testHarness.processElement(1, initialTime + 1);
testHarness.processElement(2, initialTime + 2);
testHarness.prepareSnapshotPreBarrier(1L);
testHarness.snapshot(1L, 1L);
assertThat(testHarness.getOutput(), contains(new Watermark(initialTime)));
}
@Test
public void bufferingWriterEmitsOnFlush() throws Exception {
final long initialTime = 0;
final OneInputStreamOperatorTestHarness<Integer, String> testHarness =
createTestHarness(
TestSink.newBuilder()
.setWriter(new BufferingSinkWriter())
.withWriterState()
.build());
testHarness.open();
testHarness.processWatermark(initialTime);
testHarness.processElement(1, initialTime + 1);
testHarness.processElement(2, initialTime + 2);
testHarness.endInput();
assertThat(
testHarness.getOutput(),
contains(
new Watermark(initialTime),
new StreamRecord<>(Tuple3.of(1, initialTime + 1, initialTime).toString()),
new StreamRecord<>(Tuple3.of(2, initialTime + 2, initialTime).toString())));
}
@Test
public void timeBasedBufferingSinkWriter() throws Exception {
final long initialTime = 0;
final OneInputStreamOperatorTestHarness<Integer, String> testHarness =
createTestHarness(
TestSink.newBuilder()
.setWriter(new TimeBasedBufferingSinkWriter())
.withWriterState()
.build());
testHarness.open();
testHarness.setProcessingTime(0L);
testHarness.processElement(1, initialTime + 1);
testHarness.processElement(2, initialTime + 2);
testHarness.prepareSnapshotPreBarrier(1L);
assertThat(testHarness.getOutput().size(), equalTo(0));
testHarness.getProcessingTimeService().setCurrentTime(2001);
testHarness.prepareSnapshotPreBarrier(2L);
testHarness.endInput();
assertThat(
testHarness.getOutput(),
contains(
new StreamRecord<>(
Tuple3.of(1, initialTime + 1, Long.MIN_VALUE).toString()),
new StreamRecord<>(
Tuple3.of(2, initialTime + 2, Long.MIN_VALUE).toString())));
}
@Test
public void watermarkPropagatedToSinkWriter() throws Exception {
final long initialTime = 0;
final TestSink.DefaultSinkWriter writer = new TestSink.DefaultSinkWriter();
final OneInputStreamOperatorTestHarness<Integer, String> testHarness =
createTestHarness(
TestSink.newBuilder().setWriter(writer).withWriterState().build());
testHarness.open();
testHarness.processWatermark(initialTime);
testHarness.processWatermark(initialTime + 1);
assertThat(
testHarness.getOutput(),
contains(new Watermark(initialTime), new Watermark(initialTime + 1)));
assertThat(
writer.watermarks,
contains(
new org.apache.flink.api.common.eventtime.Watermark(initialTime),
new org.apache.flink.api.common.eventtime.Watermark(initialTime + 1)));
}
/**
* A {@link SinkWriter} that only returns committables from {@link #prepareCommit(boolean)} when
* {@code flush} is {@code true}.
*/
static class BufferingSinkWriter extends TestSink.DefaultSinkWriter {
@Override
public List<String> prepareCommit(boolean flush) {
if (!flush) {
return Collections.emptyList();
}
List<String> result = elements;
elements = new ArrayList<>();
return result;
}
}
/**
* A {@link SinkWriter} that buffers the committables and send the cached committables per
* second.
*/
static class TimeBasedBufferingSinkWriter extends TestSink.DefaultSinkWriter
implements Sink.ProcessingTimeService.ProcessingTimeCallback {
private final List<String> cachedCommittables = new ArrayList<>();
@Override
public void write(Integer element, Context context) {
cachedCommittables.add(
Tuple3.of(element, context.timestamp(), context.currentWatermark()).toString());
}
void setProcessingTimerService(Sink.ProcessingTimeService processingTimerService) {
super.setProcessingTimerService(processingTimerService);
this.processingTimerService.registerProcessingTimer(1000, this);
}
@Override
public void onProcessingTime(long time) throws IOException {
elements.addAll(cachedCommittables);
cachedCommittables.clear();
this.processingTimerService.registerProcessingTimer(time + 1000, this);
}
}
protected OneInputStreamOperatorTestHarness<Integer, String> createTestHarness(TestSink sink)
throws Exception {
return new OneInputStreamOperatorTestHarness<>(
createWriterOperator(sink), IntSerializer.INSTANCE);
}
}