/
StreamTwoInputProcessor.java
268 lines (232 loc) · 9.84 KB
/
StreamTwoInputProcessor.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
256
257
258
259
260
261
262
263
264
265
266
267
268
/*
* 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.io;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
import org.apache.flink.runtime.event.task.AbstractEvent;
import org.apache.flink.runtime.io.network.api.reader.AbstractReader;
import org.apache.flink.runtime.io.network.api.reader.ReaderBase;
import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer;
import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer.DeserializationResult;
import org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer;
import org.apache.flink.runtime.io.network.buffer.Buffer;
import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
import org.apache.flink.runtime.plugable.DeserializationDelegate;
import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate;
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer;
import org.apache.flink.streaming.runtime.tasks.CheckpointBarrier;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.Collection;
/**
* Input reader for {@link org.apache.flink.streaming.runtime.tasks.TwoInputStreamTask}.
*
* <p>
* This also keeps track of {@link org.apache.flink.streaming.api.watermark.Watermark} events and forwards them to event subscribers
* once the {@link org.apache.flink.streaming.api.watermark.Watermark} from all inputs advances.
*
* @param <IN1> The type of the records that arrive on the first input
* @param <IN2> The type of the records that arrive on the second input
*/
public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements ReaderBase, StreamingReader {
@SuppressWarnings("unused")
private static final Logger LOG = LoggerFactory.getLogger(StreamTwoInputProcessor.class);
private final RecordDeserializer<DeserializationDelegate<Object>>[] recordDeserializers;
private RecordDeserializer<DeserializationDelegate<Object>> currentRecordDeserializer;
// We need to keep track of the channel from which a buffer came, so that we can
// appropriately map the watermarks to input channels
int currentChannel = -1;
private boolean isFinished;
private final BarrierBuffer barrierBuffer;
private long[] watermarks1;
private long lastEmittedWatermark1;
private long[] watermarks2;
private long lastEmittedWatermark2;
private int numInputChannels1;
private int numInputChannels2;
private DeserializationDelegate<Object> deserializationDelegate1;
private DeserializationDelegate<Object> deserializationDelegate2;
@SuppressWarnings("unchecked")
public StreamTwoInputProcessor(
Collection<InputGate> inputGates1,
Collection<InputGate> inputGates2,
TypeSerializer<IN1> inputSerializer1,
TypeSerializer<IN2> inputSerializer2,
boolean enableWatermarkMultiplexing) {
super(InputGateUtil.createInputGate(inputGates1, inputGates2));
barrierBuffer = new BarrierBuffer(inputGate, this);
StreamRecordSerializer<IN1> inputRecordSerializer1;
if (enableWatermarkMultiplexing) {
inputRecordSerializer1 = new MultiplexingStreamRecordSerializer<IN1>(inputSerializer1);
} else {
inputRecordSerializer1 = new StreamRecordSerializer<IN1>(inputSerializer1);
}
this.deserializationDelegate1 = new NonReusingDeserializationDelegate(inputRecordSerializer1);
StreamRecordSerializer<IN2> inputRecordSerializer2;
if (enableWatermarkMultiplexing) {
inputRecordSerializer2 = new MultiplexingStreamRecordSerializer<IN2>(inputSerializer2);
} else {
inputRecordSerializer2 = new StreamRecordSerializer<IN2>(inputSerializer2);
}
this.deserializationDelegate2 = new NonReusingDeserializationDelegate(inputRecordSerializer2);
// Initialize one deserializer per input channel
this.recordDeserializers = new SpillingAdaptiveSpanningRecordDeserializer[inputGate
.getNumberOfInputChannels()];
for (int i = 0; i < recordDeserializers.length; i++) {
recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer();
}
// determine which unioned channels belong to input 1 and which belong to input 2
numInputChannels1 = 0;
for (InputGate gate: inputGates1) {
numInputChannels1 += gate.getNumberOfInputChannels();
}
numInputChannels2 = inputGate.getNumberOfInputChannels() - numInputChannels1;
watermarks1 = new long[numInputChannels1];
for (int i = 0; i < numInputChannels1; i++) {
watermarks1[i] = Long.MIN_VALUE;
}
lastEmittedWatermark1 = Long.MIN_VALUE;
watermarks2 = new long[numInputChannels2];
for (int i = 0; i < numInputChannels2; i++) {
watermarks2[i] = Long.MIN_VALUE;
}
lastEmittedWatermark2 = Long.MIN_VALUE;
}
@SuppressWarnings("unchecked")
public boolean processInput(TwoInputStreamOperator<IN1, IN2, ?> streamOperator) throws Exception {
if (isFinished) {
return false;
}
while (true) {
if (currentRecordDeserializer != null) {
DeserializationResult result;
if (currentChannel < numInputChannels1) {
result = currentRecordDeserializer.getNextRecord(deserializationDelegate1);
} else {
result = currentRecordDeserializer.getNextRecord(deserializationDelegate2);
}
if (result.isBufferConsumed()) {
currentRecordDeserializer.getCurrentBuffer().recycle();
currentRecordDeserializer = null;
}
if (result.isFullRecord()) {
if (currentChannel < numInputChannels1) {
Object recordOrWatermark = deserializationDelegate1.getInstance();
if (recordOrWatermark instanceof Watermark) {
handleWatermark(streamOperator, (Watermark) recordOrWatermark, currentChannel);
continue;
} else {
streamOperator.processElement1((StreamRecord<IN1>) deserializationDelegate1.getInstance());
return true;
}
} else {
Object recordOrWatermark = deserializationDelegate2.getInstance();
if (recordOrWatermark instanceof Watermark) {
handleWatermark(streamOperator, (Watermark) recordOrWatermark, currentChannel);
continue;
} else {
streamOperator.processElement2((StreamRecord<IN2>) deserializationDelegate2.getInstance());
return true;
}
}
}
}
final BufferOrEvent bufferOrEvent = barrierBuffer.getNextNonBlocked();
if (bufferOrEvent.isBuffer()) {
currentChannel = bufferOrEvent.getChannelIndex();
currentRecordDeserializer = recordDeserializers[currentChannel];
currentRecordDeserializer.setNextBuffer(bufferOrEvent.getBuffer());
} else {
// Event received
final AbstractEvent event = bufferOrEvent.getEvent();
if (event instanceof CheckpointBarrier) {
barrierBuffer.processBarrier(bufferOrEvent);
} else {
if (handleEvent(event)) {
if (inputGate.isFinished()) {
if (!barrierBuffer.isEmpty()) {
throw new RuntimeException("BarrierBuffer should be empty at this point");
}
isFinished = true;
return false;
} else if (hasReachedEndOfSuperstep()) {
return false;
} // else: More data is coming...
}
}
}
}
}
private void handleWatermark(TwoInputStreamOperator<IN1, IN2, ?> operator, Watermark mark, int channelIndex) throws Exception {
if (channelIndex < numInputChannels1) {
long watermarkMillis = mark.getTimestamp();
if (watermarkMillis > watermarks1[channelIndex]) {
watermarks1[channelIndex] = watermarkMillis;
long newMinWatermark = Long.MAX_VALUE;
for (long aWatermarks1 : watermarks1) {
if (aWatermarks1 < newMinWatermark) {
newMinWatermark = aWatermarks1;
}
}
if (newMinWatermark > lastEmittedWatermark1) {
lastEmittedWatermark1 = newMinWatermark;
operator.processWatermark1(new Watermark(lastEmittedWatermark1));
}
}
} else {
channelIndex = channelIndex - numInputChannels1;
long watermarkMillis = mark.getTimestamp();
if (watermarkMillis > watermarks2[channelIndex]) {
watermarks2[channelIndex] = watermarkMillis;
long newMinWatermark = Long.MAX_VALUE;
for (long aWatermarks2 : watermarks2) {
if (aWatermarks2 < newMinWatermark) {
newMinWatermark = aWatermarks2;
}
}
if (newMinWatermark > lastEmittedWatermark2) {
lastEmittedWatermark2 = newMinWatermark;
operator.processWatermark2(new Watermark(lastEmittedWatermark2));
}
}
}
}
@Override
public void setReporter(AccumulatorRegistry.Reporter reporter) {
for (RecordDeserializer<?> deserializer : recordDeserializers) {
deserializer.setReporter(reporter);
}
}
public void clearBuffers() {
for (RecordDeserializer<?> deserializer : recordDeserializers) {
Buffer buffer = deserializer.getCurrentBuffer();
if (buffer != null && !buffer.isRecycled()) {
buffer.recycle();
}
}
}
public void cleanup() throws IOException {
barrierBuffer.cleanup();
}
}