-
Notifications
You must be signed in to change notification settings - Fork 1
/
BenchmarkApplication.java
432 lines (352 loc) · 23.1 KB
/
BenchmarkApplication.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
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
package fr.unice.namb.flink;
import fr.unice.namb.flink.connectors.SyntheticConnector;
import fr.unice.namb.flink.operators.BusyWaitFlatMap;
import fr.unice.namb.flink.operators.WindowedBusyWaitFunction;
import fr.unice.namb.flink.utils.KafkaDeserializationSchema;
import fr.unice.namb.utils.common.AppBuilder;
import fr.unice.namb.utils.common.Task;
import fr.unice.namb.utils.configuration.Config;
import fr.unice.namb.utils.configuration.schema.FlinkConfigSchema;
import fr.unice.namb.utils.configuration.schema.NambConfigSchema;
import jdk.nashorn.internal.runtime.regexp.joni.exception.ValueException;
import org.apache.commons.lang3.tuple.MutablePair;
import org.apache.flink.api.common.serialization.DeserializationSchema;
import org.apache.flink.api.common.serialization.SimpleStringSchema;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.tuple.Tuple4;
import org.apache.flink.streaming.api.datastream.AllWindowedStream;
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.timestamps.AscendingTimestampExtractor;
import org.apache.flink.streaming.api.windowing.time.Time;
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Properties;
public class BenchmarkApplication {
private static DataStream<Tuple4<String, String, Long, Long>> setRouting(SingleOutputStreamOperator<Tuple4<String, String, Long, Long>> operator, Config.TrafficRouting routing, Object field, boolean apply) throws IllegalArgumentException {
if (apply){
switch (routing) {
case hash:
if (field instanceof Integer)
return operator.keyBy((int) field);
else if (field instanceof String)
return operator.keyBy((String) field);
else
throw new IllegalArgumentException("Field must be <int> or <String> instead it is <" + field.getClass().getName() + ">");
case balanced:
return operator.rebalance();
case broadcast:
return operator.broadcast();
case none:
return operator;
default:
throw new ValueException(routing + " is not a valid routing type");
}
}
return operator;
}
private static DataStream<Tuple4<String, String, Long, Long>> setRouting(DataStream<Tuple4<String, String, Long, Long>> operator, Config.TrafficRouting routing, Object field) throws IllegalArgumentException {
switch (routing) {
case hash:
if (field instanceof Integer)
return operator.keyBy((int) field);
else if (field instanceof String)
return operator.keyBy((String) field);
else
throw new IllegalArgumentException("Field must be <int> or <String> instead it is <" + field.getClass().getName() + ">");
case balanced:
return operator.rebalance();
case broadcast:
return operator.broadcast();
case none:
return operator;
default:
throw new ValueException(routing + " is not a valid routing type");
}
}
private static DataStream<Tuple4<String, String, Long, Long>> setRouting(SingleOutputStreamOperator<Tuple4<String, String, Long, Long>> operator, Config.TrafficRouting routing, boolean apply) throws IllegalArgumentException {
return setRouting(operator, routing, 0, apply);
}
private static DataStream<Tuple4<String, String, Long, Long>> setRouting(SingleOutputStreamOperator<Tuple4<String, String, Long, Long>> operator, Config.TrafficRouting routing) throws IllegalArgumentException {
return setRouting(operator, routing, 0, true);
}
private static DataStream<Tuple4<String, String, Long, Long>> setRouting(DataStream<Tuple4<String, String, Long, Long>> operator, Config.TrafficRouting routing) throws IllegalArgumentException {
return setRouting(operator, routing, 0);
}
private static AllWindowedStream<Tuple4<String, String, Long, Long>, TimeWindow> setWindow(SingleOutputStreamOperator<Tuple4<String, String, Long, Long>> parent, Config.TrafficRouting trafficRouting, Config.WindowingType type, int duration, int interval, boolean applyRouting) {
switch (type) {
case tumbling:
return setRouting(parent, trafficRouting, applyRouting).timeWindowAll(Time.seconds(duration));
case sliding:
return setRouting(parent, trafficRouting, applyRouting).timeWindowAll(Time.seconds(duration), Time.seconds(interval));
}
return null;
}
private static AllWindowedStream<Tuple4<String, String, Long, Long>, TimeWindow> setWindow(DataStream<Tuple4<String, String, Long, Long>> parent, Config.TrafficRouting trafficRouting, Config.WindowingType type, int duration, int interval) {
switch (type) {
case tumbling:
return setRouting(parent, trafficRouting).timeWindowAll(Time.seconds(duration));
case sliding:
return setRouting(parent, trafficRouting).timeWindowAll(Time.seconds(duration), Time.seconds(interval));
}
return null;
}
private static AllWindowedStream<Tuple4<String, String, Long, Long>, TimeWindow> setWindow(SingleOutputStreamOperator<Tuple4<String, String, Long, Long>> parent, Config.TrafficRouting trafficRouting, Config.WindowingType type, int duration, boolean apply){
return setWindow(parent, trafficRouting, type, duration, 0);
}
private static AllWindowedStream<Tuple4<String, String, Long, Long>, TimeWindow> setWindow(DataStream<Tuple4<String, String, Long, Long>> parent, Config.TrafficRouting trafficRouting, Config.WindowingType type, int duration){
return setWindow(parent, trafficRouting, type, duration, 0);
}
private static StreamExecutionEnvironment buildBenchmarkEnvironment(NambConfigSchema conf, double debugFrequency) throws Exception{
AppBuilder app = new AppBuilder(conf);
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
if(! app.isPipelineDefined()) {
/*
Workflow Schema Translation
*/
// DataFlow configurations
int depth = conf.getWorkflow().getDepth();
int totalParallelism = conf.getWorkflow().getScalability().getParallelism();
Config.ParaBalancing paraBalancing = conf.getWorkflow().getScalability().getBalancing();
double variability = conf.getWorkflow().getScalability().getVariability();
Config.ConnectionShape topologyShape = conf.getWorkflow().getConnection().getShape();
Config.TrafficRouting trafficRouting = conf.getWorkflow().getConnection().getRouting();
double processingLoad = conf.getWorkflow().getWorkload().getProcessing();
Config.LoadBalancing loadBalancing = conf.getWorkflow().getWorkload().getBalancing();
// DataStream configurations
int dataSize = conf.getDatastream().getSynthetic().getData().getSize();
int dataValues = conf.getDatastream().getSynthetic().getData().getValues();
Config.DataDistribution dataValuesBalancing = conf.getDatastream().getSynthetic().getData().getDistribution();
Config.ArrivalDistribution distribution = conf.getDatastream().getSynthetic().getFlow().getDistribution();
int rate = conf.getDatastream().getSynthetic().getFlow().getRate();
// Generating app builder
ArrayList<Integer> dagLevelsWidth = app.getDagLevelsWidth();
ArrayList<Integer> componentsParallelism = app.getComponentsParallelism();
// Bolt-specific configurations
int numberOfSources = dagLevelsWidth.get(0);
int numberOfOperators = app.getTotalComponents() - numberOfSources;
// Windowing
boolean windowingEnabled = conf.getWorkflow().getWindowing().isEnabled();
Config.WindowingType windowingType = conf.getWorkflow().getWindowing().getType();
int windowDuration = conf.getWorkflow().getWindowing().getDuration();
int windowInterval = conf.getWorkflow().getWindowing().getInterval();
int windowedTasks = (depth > 3) ? 2 : 1;
Iterator<Integer> cpIterator = componentsParallelism.iterator();
ArrayList<MutablePair<String, DataStream<Tuple4<String, String, Long, Long>>>> sourcesList = new ArrayList<>();
ArrayList<MutablePair<String, SingleOutputStreamOperator<Tuple4<String, String, Long, Long>>>> operatorsList = new ArrayList<>();
String sourceName;
/*
Sources Definition
*/
if (app.isExternalSource()){
int s = 1;
sourceName = "kafka_source_" + s;
Properties properties = new Properties();
properties.setProperty("bootstrap.servers", app.getKafkaServer());
properties.setProperty("zookeeper.connect", app.getZookeeperServer());
properties.setProperty("group.id", app.getKafkaGroup());
FlinkKafkaConsumer<Tuple4<String, String, Long, Long>> kafkaConsumer = new FlinkKafkaConsumer<>(app.getKafkaTopic(), new KafkaDeserializationSchema(debugFrequency, sourceName), properties);
DataStream<Tuple4<String, String, Long, Long>> source = env
.addSource(kafkaConsumer)
.setParallelism(cpIterator.next())
.name(sourceName);
sourcesList.add(new MutablePair<>(sourceName, source));
}
else {
for (int s = 1; s <= numberOfSources; s++) {
sourceName = "source_" + s;
DataStream<Tuple4<String, String, Long, Long>> source = env.addSource(new SyntheticConnector(dataSize, dataValues, dataValuesBalancing, distribution, rate, debugFrequency, sourceName))
.setParallelism(cpIterator.next())
.name(sourceName);
sourcesList.add(new MutablePair<>(sourceName, source));
}
if (numberOfSources > 1) {
sourceName = "unified_source";
DataStream<Tuple4<String, String, Long, Long>> source = sourcesList.get(0).getRight().union(sourcesList.get(1).getRight());
for (int s = 2; s < numberOfSources; s++) {
source.union(sourcesList.get(s).getRight());
}
sourcesList.add(new MutablePair<>(sourceName, source));
}
}
/*
Tasks definition
*/
int operatorID = 1;
int cycles;
String operatorName;
for (int i = 1; i < depth; i++) {
int levelWidth = dagLevelsWidth.get(i);
boolean isWindowed = depth - i <= windowedTasks && windowingEnabled; // this level contains windowed tasks
if (i == 1) {
for (int opCount = 0; opCount < levelWidth; opCount++) {
operatorName = "op_" + operatorID;
cycles = app.getNextProcessing();
DataStream<Tuple4<String, String, Long, Long>> parent = sourcesList.get(sourcesList.size() - 1).getRight();
SingleOutputStreamOperator<Tuple4<String, String, Long, Long>> op = null;
if (isWindowed) {
operatorName = "windowed-" + operatorName;
op = setWindow(parent, trafficRouting, windowingType, windowDuration, windowInterval)
.apply(new WindowedBusyWaitFunction(cycles, debugFrequency));
} else {
double filtering = (app.getFilteringDagLevel() == i) ? app.getFiltering() : 0;
op = setRouting(parent, trafficRouting)
.flatMap(new BusyWaitFlatMap(cycles, filtering, debugFrequency, operatorName));
}
op.setParallelism(cpIterator.next())
.name(operatorName);
operatorsList.add(new MutablePair<>(operatorName, op));
operatorID++;
}
} else {
if (topologyShape == Config.ConnectionShape.diamond && dagLevelsWidth.get(i - 1) > 1) {// diamond shape union
DataStream<Tuple4<String, String, Long, Long>> diamondUnion = operatorsList.get(operatorID - 2).getRight().union(operatorsList.get(operatorID - 3).getRight());
//TODO: maybe this can be optimized?
for (int o = 2; o < dagLevelsWidth.get(i - 1); o++) {
diamondUnion.union(operatorsList.get(o).getRight());
}
operatorName = "op_" + operatorID;
cycles = app.getNextProcessing();
SingleOutputStreamOperator<Tuple4<String, String, Long, Long>> op = null;
if (isWindowed) {
operatorName = "windowed-" + operatorName;
op = setWindow(diamondUnion, trafficRouting, windowingType, windowDuration, windowInterval)
.apply(new WindowedBusyWaitFunction(cycles, debugFrequency));
} else {
double filtering = (app.getFilteringDagLevel() == i) ? app.getFiltering() : 0;
op = setRouting(diamondUnion, trafficRouting)
.flatMap(new BusyWaitFlatMap(cycles, filtering, debugFrequency, operatorName));
}
op.setParallelism(cpIterator.next())
.name(operatorName);
operatorsList.add(new MutablePair<>(operatorName, op));
operatorID++;
} else {
int parentOperatorIdx = (topologyShape == Config.ConnectionShape.diamond ||
(topologyShape == Config.ConnectionShape.star && i > 3)) ? i - 1 : i - 2;
SingleOutputStreamOperator<Tuple4<String, String, Long, Long>> parent = operatorsList.get(parentOperatorIdx).getRight();
for (int opCount = 0; opCount < levelWidth; opCount++) {
operatorName = "op_" + operatorID;
cycles = app.getNextProcessing();
SingleOutputStreamOperator<Tuple4<String, String, Long, Long>> op = null;
if (isWindowed) {
operatorName = "windowed-" + operatorName;
op = setWindow(parent, trafficRouting, windowingType, windowDuration, windowInterval, false)
.apply(new WindowedBusyWaitFunction(cycles, debugFrequency));
} else {
double filtering = (app.getFilteringDagLevel() == i) ? app.getFiltering() : 0;
op = setRouting(parent, Config.TrafficRouting.none)
.flatMap(new BusyWaitFlatMap(cycles, filtering, debugFrequency, operatorName));
// op = parent.map(new BusyWaitMap(cycles, debugFrequency));
}
op.setParallelism(cpIterator.next())
.name(operatorName);
operatorsList.add(new MutablePair<>(operatorName, op));
operatorID++;
}
}
}
}
}
else{
/*
Pipeline Schema Translation
*/
HashMap<String, Task> pipeline = app.getPipelineTree();
ArrayList<String> dagLevel = app.getPipelineTreeSources();
HashMap<String, Object> createdTasks = new HashMap<>();
while (dagLevel.size() > 0){
ArrayList<String> nextDagLevel = new ArrayList<>();
for (String task : dagLevel) {
if (!createdTasks.containsKey(task)) {
Task newTask = pipeline.get(task);
if (newTask.getType() == Config.ComponentType.source) {
DataStream<Tuple4<String, String, Long, Long>> source = null;
if(newTask.isExternal()){
Properties properties = new Properties();
properties.setProperty("bootstrap.servers", newTask.getKafkaServer());
properties.setProperty("zookeeper.connect", newTask.getZookeeperServer());
properties.setProperty("group.id", newTask.getKafkaGroup());
FlinkKafkaConsumer<Tuple4<String, String, Long, Long>> kafkaConsumer = new FlinkKafkaConsumer<>(newTask.getKafkaTopic(), new KafkaDeserializationSchema(debugFrequency, newTask.getName()), properties);
source = env
.addSource(kafkaConsumer)
.setParallelism((int) newTask.getParallelism())
.name(newTask.getName());
}
else {
source = env.addSource(new SyntheticConnector(newTask.getDataSize(), newTask.getDataValues(), newTask.getDataDistribution(), newTask.getFlowDistribution(), newTask.getFlowRate(), debugFrequency, newTask.getName()))
.setParallelism((int) newTask.getParallelism())
.name(newTask.getName());
}
createdTasks.put(newTask.getName(), source);
}
else{
ArrayList<String> parentsList = newTask.getParents();
SingleOutputStreamOperator<Tuple4<String, String, Long, Long>> op = null;
DataStream<Tuple4<String, String, Long, Long>> streamUnion = null;
if(parentsList.size() > 1) {
if( pipeline.get(parentsList.get(0)).getType() == Config.ComponentType.source){
streamUnion = ((DataStream<Tuple4<String, String, Long, Long>>) createdTasks.get(parentsList.get(0))).union((DataStream<Tuple4<String, String, Long, Long>>) createdTasks.get(parentsList.get(1)));
for (int i=2; i<parentsList.size(); i++) {
streamUnion.union((DataStream<Tuple4<String, String, Long, Long>>) createdTasks.get(parentsList.get(i)));
}
}
else{
streamUnion = ((SingleOutputStreamOperator<Tuple4<String, String, Long, Long>>) createdTasks.get(parentsList.get(0))).union((SingleOutputStreamOperator<Tuple4<String, String, Long, Long>>) createdTasks.get(parentsList.get(1)));
for (int i=2; i<parentsList.size(); i++) {
streamUnion.union((SingleOutputStreamOperator<Tuple4<String, String, Long, Long>>) createdTasks.get(parentsList.get(i)));
}
}
//TODO: impolement windowing
op = setRouting(streamUnion, newTask.getRouting())
.flatMap(new BusyWaitFlatMap(newTask.getProcessing(), newTask.getFiltering(), newTask.getDataSize(), debugFrequency, newTask.getName()));
}
else{
if( pipeline.get(parentsList.get(0)).getType() == Config.ComponentType.source){
DataStream<Tuple4<String, String, Long, Long>> parent = (DataStream<Tuple4<String, String, Long, Long>>) createdTasks.get(parentsList.get(0));
op = setRouting(parent, newTask.getRouting())
.flatMap(new BusyWaitFlatMap(newTask.getProcessing(), newTask.getFiltering(), newTask.getDataSize(), debugFrequency, newTask.getName()));
}
else{
SingleOutputStreamOperator<Tuple4<String, String, Long, Long>> parent = (SingleOutputStreamOperator<Tuple4<String, String, Long, Long>>) createdTasks.get(parentsList.get(0));
op = setRouting(parent, newTask.getRouting())
.flatMap(new BusyWaitFlatMap(newTask.getProcessing(), newTask.getFiltering(), newTask.getDataSize(), debugFrequency, newTask.getName()));
}
}
op.setParallelism((int)newTask.getParallelism())
.name(newTask.getName());
createdTasks.put(newTask.getName(), op);
}
}
nextDagLevel.addAll(pipeline.get(task).getChilds());
}
dagLevel = new ArrayList<>(nextDagLevel);
}
}
return env;
}
public static void main(String[] args) throws Exception{
String nambConfFilePath = args[0];
String flinkConfFilePath = args[1];
//Obtaining Configurations
Config confParser = new Config(NambConfigSchema.class, nambConfFilePath);
NambConfigSchema nambConf = (NambConfigSchema) confParser.getConfigSchema();
Config flinkConfigParser = new Config(FlinkConfigSchema.class, flinkConfFilePath);
FlinkConfigSchema flinkConf = (FlinkConfigSchema) flinkConfigParser.getConfigSchema();
if(nambConf != null && flinkConf != null) {
confParser.validateConf(nambConf);
StreamExecutionEnvironment env = buildBenchmarkEnvironment(nambConf, flinkConf.getDebugFrequency());
if (env != null){
String executionName = "namb_bench_" + System.currentTimeMillis();
env.execute(executionName);
}
} else {
throw new Exception("Something went wrong during configuration loading");
}
}
}