This repository has been archived by the owner on Nov 1, 2021. It is now read-only.
-
Notifications
You must be signed in to change notification settings - Fork 37
/
TroubledStreamingJobSolution2.java
239 lines (196 loc) · 9.71 KB
/
TroubledStreamingJobSolution2.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
package com.ververica.flinktraining.solutions.troubleshoot;
import org.apache.flink.api.common.functions.RichFlatMapFunction;
import org.apache.flink.api.java.utils.ParameterTool;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.metrics.Counter;
import org.apache.flink.runtime.metrics.DescriptiveStatisticsHistogram;
import org.apache.flink.streaming.api.TimeCharacteristic;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.api.windowing.time.Time;
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
import org.apache.flink.util.Collector;
import org.apache.flink.util.OutputTag;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.ververica.flinktraining.provided.troubleshoot.FakeKafkaRecord;
import com.ververica.flinktraining.provided.troubleshoot.WindowedMeasurements;
import com.ververica.flinktraining.provided.troubleshoot.SourceUtils;
import java.io.IOException;
import java.util.concurrent.TimeUnit;
import static com.ververica.flinktraining.exercises.troubleshoot.TroubledStreamingJobUtils.createConfiguredEnvironment;
public class TroubledStreamingJobSolution2 {
public static void main(String[] args) throws Exception {
ParameterTool parameters = ParameterTool.fromArgs(args);
final boolean local = parameters.getBoolean("local", false);
StreamExecutionEnvironment env = createConfiguredEnvironment(parameters, local);
//Time Characteristics
env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
env.getConfig().setAutoWatermarkInterval(2000);
//Checkpointing Configuration
env.enableCheckpointing(5000);
env.getCheckpointConfig().setMinPauseBetweenCheckpoints(4000);
DataStream<JsonNode> sourceStream = env
.addSource(SourceUtils.createFakeKafkaSource())
.name("FakeKafkaSource")
.uid("FakeKafkaSource")
.assignTimestampsAndWatermarks(
new MeasurementTSExtractor(Time.of(250, TimeUnit.MILLISECONDS),
Time.of(1, TimeUnit.SECONDS)))
.name("Watermarks")
.uid("Watermarks")
.flatMap(new MeasurementDeserializer())
.name("Deserialization")
.uid("Deserialization");
OutputTag<JsonNode> lateDataTag = new OutputTag<JsonNode>("late-data") {
private static final long serialVersionUID = 33513631677208956L;
};
SingleOutputStreamOperator<WindowedMeasurements> aggregatedPerLocation = sourceStream
.keyBy(jsonNode -> jsonNode.get("location").asText())
.timeWindow(Time.of(1, TimeUnit.SECONDS))
.sideOutputLateData(lateDataTag)
.process(new MeasurementWindowAggregatingFunction())
.name("WindowedAggregationPerLocation")
.uid("WindowedAggregationPerLocation");
if (local) {
aggregatedPerLocation.print()
.name("NormalOutput")
.uid("NormalOutput")
.disableChaining();
aggregatedPerLocation.getSideOutput(lateDataTag).printToErr()
.name("LateDataSink")
.uid("LateDataSink")
.disableChaining();
} else {
aggregatedPerLocation.addSink(new DiscardingSink<>())
.name("NormalOutput")
.uid("NormalOutput")
.disableChaining();
aggregatedPerLocation.getSideOutput(lateDataTag).addSink(new DiscardingSink<>())
.name("LateDataSink")
.uid("LateDataSink")
.disableChaining();
}
env.execute(TroubledStreamingJobSolution2.class.getSimpleName());
}
/**
* Deserializes the JSON Kafka message.
*/
public static class MeasurementDeserializer extends RichFlatMapFunction<FakeKafkaRecord, JsonNode> {
private static final long serialVersionUID = 2L;
private Counter numInvalidRecords;
@Override
public void open(final Configuration parameters) throws Exception {
super.open(parameters);
numInvalidRecords = getRuntimeContext().getMetricGroup().counter("numInvalidRecords");
}
@Override
public void flatMap(final FakeKafkaRecord kafkaRecord, final Collector<JsonNode> out) {
final JsonNode node;
try {
node = deserialize(kafkaRecord.getValue());
} catch (IOException e) {
numInvalidRecords.inc();
return;
}
out.collect(node);
}
private JsonNode deserialize(final byte[] bytes) throws IOException {
return ObjectMapperSingleton.getInstance().readValue(bytes, JsonNode.class);
}
}
public static class MeasurementTSExtractor implements AssignerWithPeriodicWatermarks<FakeKafkaRecord> {
private static final long serialVersionUID = 2L;
private long currentMaxTimestamp;
private long lastEmittedWatermark = Long.MIN_VALUE;
private long lastRecordProcessingTime;
private final long maxOutOfOrderness;
private final long idleTimeout;
MeasurementTSExtractor(Time maxOutOfOrderness, Time idleTimeout) {
if (maxOutOfOrderness.toMilliseconds() < 0) {
throw new RuntimeException("Tried to set the maximum allowed " +
"lateness to " + maxOutOfOrderness +
". This parameter cannot be negative.");
}
if (idleTimeout.toMilliseconds() < 0) {
throw new RuntimeException("Tried to set the idle Timeout" + idleTimeout +
". This parameter cannot be negative.");
}
this.maxOutOfOrderness = maxOutOfOrderness.toMilliseconds();
this.idleTimeout = idleTimeout.toMilliseconds();
this.currentMaxTimestamp = Long.MIN_VALUE;
}
public long getMaxOutOfOrdernessInMillis() {
return maxOutOfOrderness;
}
@Override
public final Watermark getCurrentWatermark() {
// if last record was processed more than the idleTimeout in the past, consider this
// source idle and set timestamp to current processing time
long currentProcessingTime = System.currentTimeMillis();
if (lastRecordProcessingTime < currentProcessingTime - idleTimeout) {
this.currentMaxTimestamp = currentProcessingTime;
}
long potentialWM = this.currentMaxTimestamp - maxOutOfOrderness;
if (potentialWM >= lastEmittedWatermark) {
lastEmittedWatermark = potentialWM;
}
return new Watermark(lastEmittedWatermark);
}
@Override
public final long extractTimestamp(FakeKafkaRecord element, long previousElementTimestamp) {
lastRecordProcessingTime = System.currentTimeMillis();
long timestamp = element.getTimestamp();
if (timestamp > currentMaxTimestamp) {
currentMaxTimestamp = timestamp;
}
return timestamp;
}
}
public static class MeasurementWindowAggregatingFunction
extends ProcessWindowFunction<JsonNode, WindowedMeasurements, String, TimeWindow> {
private static final long serialVersionUID = 1L;
private static final int EVENT_TIME_LAG_WINDOW_SIZE = 10_000;
private transient DescriptiveStatisticsHistogram eventTimeLag;
MeasurementWindowAggregatingFunction() {
}
@Override
public void process(
final String location,
final Context context,
final Iterable<JsonNode> input,
final Collector<WindowedMeasurements> out) {
WindowedMeasurements aggregate = new WindowedMeasurements();
for (JsonNode record : input) {
double result = Double.parseDouble(record.get("value").asText());
aggregate.setSumPerWindow(aggregate.getSumPerWindow() + result);
aggregate.setEventsPerWindow(aggregate.getEventsPerWindow() + 1);
}
final TimeWindow window = context.window();
aggregate.setWindowStart(window.getStart());
aggregate.setWindowEnd(window.getEnd());
aggregate.setLocation(location);
eventTimeLag.update(System.currentTimeMillis() - window.getEnd());
out.collect(aggregate);
}
@Override
public void open(Configuration parameters) throws Exception {
super.open(parameters);
eventTimeLag = getRuntimeContext().getMetricGroup().histogram("eventTimeLag",
new DescriptiveStatisticsHistogram(EVENT_TIME_LAG_WINDOW_SIZE));
}
}
private static class ObjectMapperSingleton {
static ObjectMapper getInstance() {
ObjectMapper objectMapper = new ObjectMapper();
objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
return objectMapper;
}
}
}