/
PlanBinder.java
582 lines (515 loc) · 20.2 KB
/
PlanBinder.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
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
/**
* 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.languagebinding.api.java.common;
import java.io.IOException;
import java.util.HashMap;
import org.apache.flink.api.common.typeutils.CompositeType;
import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.io.CsvInputFormat;
import org.apache.flink.api.java.io.PrintingOutputFormat;
import org.apache.flink.api.java.operators.AggregateOperator;
import org.apache.flink.api.java.operators.CrossOperator.DefaultCross;
import org.apache.flink.api.java.operators.CrossOperator.ProjectCross;
import org.apache.flink.api.java.operators.Grouping;
import org.apache.flink.api.java.operators.JoinOperator.DefaultJoin;
import org.apache.flink.api.java.operators.JoinOperator.ProjectJoin;
import org.apache.flink.api.java.operators.SortedGrouping;
import org.apache.flink.api.java.operators.UdfOperator;
import org.apache.flink.api.java.operators.UnsortedGrouping;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.fs.Path;
import org.apache.flink.languagebinding.api.java.common.OperationInfo.DatasizeHint;
import static org.apache.flink.languagebinding.api.java.common.OperationInfo.DatasizeHint.HUGE;
import static org.apache.flink.languagebinding.api.java.common.OperationInfo.DatasizeHint.NONE;
import static org.apache.flink.languagebinding.api.java.common.OperationInfo.DatasizeHint.TINY;
import org.apache.flink.languagebinding.api.java.common.OperationInfo.ProjectionEntry;
import org.apache.flink.languagebinding.api.java.common.streaming.Receiver;
/**
* Generic class to construct a Flink plan based on external data.
*
* @param <INFO>
*/
public abstract class PlanBinder<INFO extends OperationInfo> {
public static final String PLANBINDER_CONFIG_BCVAR_COUNT = "PLANBINDER_BCVAR_COUNT";
public static final String PLANBINDER_CONFIG_BCVAR_NAME_PREFIX = "PLANBINDER_BCVAR_";
protected static String FLINK_HDFS_PATH = "hdfs:/tmp";
public static final String FLINK_TMP_DATA_DIR = System.getProperty("java.io.tmpdir") + "/flink_data";
public static boolean DEBUG = false;
protected HashMap<Integer, Object> sets = new HashMap();
public static ExecutionEnvironment env;
protected Receiver receiver;
public static final int MAPPED_FILE_SIZE = 1024 * 1024 * 64;
//====Plan==========================================================================================================
protected void receivePlan() throws IOException {
receiveParameters();
receiveOperations();
}
//====Environment===================================================================================================
/**
* This enum contains the identifiers for all supported environment parameters.
*/
private enum Parameters {
DOP,
MODE,
RETRY,
DEBUG
}
private void receiveParameters() throws IOException {
Integer parameterCount = (Integer) receiver.getRecord(true);
for (int x = 0; x < parameterCount; x++) {
Tuple value = (Tuple) receiver.getRecord(true);
switch (Parameters.valueOf(((String) value.getField(0)).toUpperCase())) {
case DOP:
Integer dop = (Integer) value.getField(1);
env.setParallelism(dop);
break;
case MODE:
FLINK_HDFS_PATH = (Boolean) value.getField(1) ? "file:/tmp/flink" : "hdfs:/tmp/flink";
break;
case RETRY:
int retry = (Integer) value.getField(1);
env.setNumberOfExecutionRetries(retry);
break;
case DEBUG:
DEBUG = (Boolean) value.getField(1);
break;
}
}
if (env.getParallelism() < 0) {
env.setParallelism(1);
}
}
//====Operations====================================================================================================
/**
* This enum contains the identifiers for all supported non-UDF DataSet operations.
*/
protected enum Operation {
SOURCE_CSV, SOURCE_TEXT, SOURCE_VALUE, SOURCE_SEQ, SINK_CSV, SINK_TEXT, SINK_PRINT,
PROJECTION, SORT, UNION, FIRST, DISTINCT, GROUPBY, AGGREGATE,
REBALANCE, PARTITION_HASH,
BROADCAST
}
/**
* This enum contains the identifiers for all supported UDF DataSet operations.
*/
protected enum AbstractOperation {
COGROUP, CROSS, CROSS_H, CROSS_T, FILTER, FLATMAP, GROUPREDUCE, JOIN, JOIN_H, JOIN_T, MAP, REDUCE, MAPPARTITION,
}
protected void receiveOperations() throws IOException {
Integer operationCount = (Integer) receiver.getRecord(true);
for (int x = 0; x < operationCount; x++) {
String identifier = (String) receiver.getRecord();
Operation op = null;
AbstractOperation aop = null;
try {
op = Operation.valueOf(identifier.toUpperCase());
} catch (IllegalArgumentException iae) {
try {
aop = AbstractOperation.valueOf(identifier.toUpperCase());
} catch (IllegalArgumentException iae2) {
throw new IllegalArgumentException("Invalid operation specified: " + identifier);
}
}
if (op != null) {
switch (op) {
case SOURCE_CSV:
createCsvSource(createOperationInfo(op));
break;
case SOURCE_TEXT:
createTextSource(createOperationInfo(op));
break;
case SOURCE_VALUE:
createValueSource(createOperationInfo(op));
break;
case SOURCE_SEQ:
createSequenceSource(createOperationInfo(op));
break;
case SINK_CSV:
createCsvSink(createOperationInfo(op));
break;
case SINK_TEXT:
createTextSink(createOperationInfo(op));
break;
case SINK_PRINT:
createPrintSink(createOperationInfo(op));
break;
case BROADCAST:
createBroadcastVariable(createOperationInfo(op));
break;
case AGGREGATE:
createAggregationOperation(createOperationInfo(op));
break;
case DISTINCT:
createDistinctOperation(createOperationInfo(op));
break;
case FIRST:
createFirstOperation(createOperationInfo(op));
break;
case PARTITION_HASH:
createHashPartitionOperation(createOperationInfo(op));
break;
case PROJECTION:
createProjectOperation(createOperationInfo(op));
break;
case REBALANCE:
createRebalanceOperation(createOperationInfo(op));
break;
case GROUPBY:
createGroupOperation(createOperationInfo(op));
break;
case SORT:
createSortOperation(createOperationInfo(op));
break;
case UNION:
createUnionOperation(createOperationInfo(op));
break;
}
}
if (aop != null) {
switch (aop) {
case COGROUP:
createCoGroupOperation(createOperationInfo(aop));
break;
case CROSS:
createCrossOperation(NONE, createOperationInfo(aop));
break;
case CROSS_H:
createCrossOperation(HUGE, createOperationInfo(aop));
break;
case CROSS_T:
createCrossOperation(TINY, createOperationInfo(aop));
break;
case FILTER:
createFilterOperation(createOperationInfo(aop));
break;
case FLATMAP:
createFlatMapOperation(createOperationInfo(aop));
break;
case GROUPREDUCE:
createGroupReduceOperation(createOperationInfo(aop));
break;
case JOIN:
createJoinOperation(NONE, createOperationInfo(aop));
break;
case JOIN_H:
createJoinOperation(HUGE, createOperationInfo(aop));
break;
case JOIN_T:
createJoinOperation(TINY, createOperationInfo(aop));
break;
case MAP:
createMapOperation(createOperationInfo(aop));
break;
case MAPPARTITION:
createMapPartitionOperation(createOperationInfo(aop));
break;
case REDUCE:
createReduceOperation(createOperationInfo(aop));
break;
}
}
}
}
/**
* This method creates an OperationInfo object based on the operation-identifier passed.
*
* @param operationIdentifier
* @return
* @throws IOException
*/
protected OperationInfo createOperationInfo(Operation operationIdentifier) throws IOException {
return new OperationInfo(receiver, operationIdentifier);
}
/**
* This method creates an OperationInfo object based on the operation-identifier passed.
*
* @param operationIdentifier
* @return
* @throws IOException
*/
protected abstract INFO createOperationInfo(AbstractOperation operationIdentifier) throws IOException;
private void createCsvSource(OperationInfo info) throws IOException {
if (!(info.types instanceof CompositeType)) {
throw new RuntimeException("The output type of a csv source has to be a tuple or a " +
"pojo type. The derived type is " + info);
}
sets.put(info.setID, env.createInput(new CsvInputFormat(new Path(info.path),
info.lineDelimiter, info.fieldDelimiter, (CompositeType)info.types), info.types)
.name("CsvSource"));
}
private void createTextSource(OperationInfo info) throws IOException {
sets.put(info.setID, env.readTextFile(info.path).name("TextSource"));
}
private void createValueSource(OperationInfo info) throws IOException {
sets.put(info.setID, env.fromElements(info.values).name("ValueSource"));
}
private void createSequenceSource(OperationInfo info) throws IOException {
sets.put(info.setID, env.generateSequence(info.from, info.to).name("SequenceSource"));
}
private void createCsvSink(OperationInfo info) throws IOException {
DataSet parent = (DataSet) sets.get(info.parentID);
parent.writeAsCsv(info.path, info.lineDelimiter, info.fieldDelimiter, info.writeMode).name("CsvSink");
}
private void createTextSink(OperationInfo info) throws IOException {
DataSet parent = (DataSet) sets.get(info.parentID);
parent.writeAsText(info.path, info.writeMode).name("TextSink");
}
private void createPrintSink(OperationInfo info) throws IOException {
DataSet parent = (DataSet) sets.get(info.parentID);
parent.output(new PrintingOutputFormat(info.toError));
}
private void createBroadcastVariable(OperationInfo info) throws IOException {
UdfOperator op1 = (UdfOperator) sets.get(info.parentID);
DataSet op2 = (DataSet) sets.get(info.otherID);
op1.withBroadcastSet(op2, info.name);
Configuration c = ((UdfOperator) op1).getParameters();
if (c == null) {
c = new Configuration();
}
int count = c.getInteger(PLANBINDER_CONFIG_BCVAR_COUNT, 0);
c.setInteger(PLANBINDER_CONFIG_BCVAR_COUNT, count + 1);
c.setString(PLANBINDER_CONFIG_BCVAR_NAME_PREFIX + count, info.name);
op1.withParameters(c);
}
private void createAggregationOperation(OperationInfo info) throws IOException {
DataSet op = (DataSet) sets.get(info.parentID);
AggregateOperator ao = op.aggregate(info.aggregates[0].agg, info.aggregates[0].field);
for (int x = 1; x < info.count; x++) {
ao = ao.and(info.aggregates[x].agg, info.aggregates[x].field);
}
sets.put(info.setID, ao.name("Aggregation"));
}
private void createDistinctOperation(OperationInfo info) throws IOException {
DataSet op = (DataSet) sets.get(info.parentID);
sets.put(info.setID, info.keys.length == 0 ? op.distinct() : op.distinct(info.keys).name("Distinct"));
}
private void createFirstOperation(OperationInfo info) throws IOException {
DataSet op = (DataSet) sets.get(info.parentID);
sets.put(info.setID, op.first(info.count).name("First"));
}
private void createGroupOperation(OperationInfo info) throws IOException {
DataSet op1 = (DataSet) sets.get(info.parentID);
sets.put(info.setID, op1.groupBy(info.keys));
}
private void createHashPartitionOperation(OperationInfo info) throws IOException {
DataSet op1 = (DataSet) sets.get(info.parentID);
sets.put(info.setID, op1.partitionByHash(info.keys));
}
private void createProjectOperation(OperationInfo info) throws IOException {
DataSet op1 = (DataSet) sets.get(info.parentID);
sets.put(info.setID, op1.project(info.fields).name("Projection"));
}
private void createRebalanceOperation(OperationInfo info) throws IOException {
DataSet op = (DataSet) sets.get(info.parentID);
sets.put(info.setID, op.rebalance().name("Rebalance"));
}
private void createSortOperation(OperationInfo info) throws IOException {
Grouping op1 = (Grouping) sets.get(info.parentID);
if (op1 instanceof UnsortedGrouping) {
sets.put(info.setID, ((UnsortedGrouping) op1).sortGroup(info.field, info.order));
return;
}
if (op1 instanceof SortedGrouping) {
sets.put(info.setID, ((SortedGrouping) op1).sortGroup(info.field, info.order));
}
}
private void createUnionOperation(OperationInfo info) throws IOException {
DataSet op1 = (DataSet) sets.get(info.parentID);
DataSet op2 = (DataSet) sets.get(info.otherID);
sets.put(info.setID, op1.union(op2).name("Union"));
}
private void createCoGroupOperation(INFO info) {
DataSet op1 = (DataSet) sets.get(info.parentID);
DataSet op2 = (DataSet) sets.get(info.otherID);
sets.put(info.setID, applyCoGroupOperation(op1, op2, info.keys1, info.keys2, info));
}
private void createCrossOperation(DatasizeHint mode, INFO info) {
DataSet op1 = (DataSet) sets.get(info.parentID);
DataSet op2 = (DataSet) sets.get(info.otherID);
if (info.types != null && (info.projections == null || info.projections.length == 0)) {
sets.put(info.setID, applyCrossOperation(op1, op2, mode, info));
} else {
DefaultCross defaultResult;
switch (mode) {
case NONE:
defaultResult = op1.cross(op2);
break;
case HUGE:
defaultResult = op1.crossWithHuge(op2);
break;
case TINY:
defaultResult = op1.crossWithTiny(op2);
break;
default:
throw new IllegalArgumentException("Invalid Cross mode specified: " + mode);
}
if (info.projections.length == 0) {
sets.put(info.setID, defaultResult.name("DefaultCross"));
} else {
ProjectCross project = null;
for (ProjectionEntry pe : info.projections) {
switch (pe.side) {
case FIRST:
project = project == null ? defaultResult.projectFirst(pe.keys) : project.projectFirst(pe.keys);
break;
case SECOND:
project = project == null ? defaultResult.projectSecond(pe.keys) : project.projectSecond(pe.keys);
break;
}
}
sets.put(info.setID, project.name("ProjectCross"));
}
}
}
private void createFilterOperation(INFO info) {
DataSet op1 = (DataSet) sets.get(info.parentID);
sets.put(info.setID, applyFilterOperation(op1, info));
}
private void createFlatMapOperation(INFO info) {
DataSet op1 = (DataSet) sets.get(info.parentID);
sets.put(info.setID, applyFlatMapOperation(op1, info));
}
private void createGroupReduceOperation(INFO info) {
Object op1 = sets.get(info.parentID);
if (op1 instanceof DataSet) {
sets.put(info.setID, applyGroupReduceOperation((DataSet) op1, info));
return;
}
if (op1 instanceof UnsortedGrouping) {
sets.put(info.setID, applyGroupReduceOperation((UnsortedGrouping) op1, info));
return;
}
if (op1 instanceof SortedGrouping) {
sets.put(info.setID, applyGroupReduceOperation((SortedGrouping) op1, info));
}
}
private void createJoinOperation(DatasizeHint mode, INFO info) {
DataSet op1 = (DataSet) sets.get(info.parentID);
DataSet op2 = (DataSet) sets.get(info.otherID);
if (info.types != null && (info.projections == null || info.projections.length == 0)) {
sets.put(info.setID, applyJoinOperation(op1, op2, info.keys1, info.keys2, mode, info));
} else {
DefaultJoin defaultResult = createDefaultJoin(op1, op2, info.keys1, info.keys2, mode);
if (info.projections.length == 0) {
sets.put(info.setID, defaultResult.name("DefaultJoin"));
} else {
ProjectJoin project = null;
for (ProjectionEntry pe : info.projections) {
switch (pe.side) {
case FIRST:
project = project == null ? defaultResult.projectFirst(pe.keys) : project.projectFirst(pe.keys);
break;
case SECOND:
project = project == null ? defaultResult.projectSecond(pe.keys) : project.projectSecond(pe.keys);
break;
}
}
sets.put(info.setID, project.name("ProjectJoin"));
}
}
}
protected DefaultJoin createDefaultJoin(DataSet op1, DataSet op2, String[] firstKeys, String[] secondKeys, DatasizeHint mode) {
switch (mode) {
case NONE:
return op1.join(op2).where(firstKeys).equalTo(secondKeys);
case HUGE:
return op1.joinWithHuge(op2).where(firstKeys).equalTo(secondKeys);
case TINY:
return op1.joinWithTiny(op2).where(firstKeys).equalTo(secondKeys);
default:
throw new IllegalArgumentException("Invalid join mode specified.");
}
}
private void createMapOperation(INFO info) {
DataSet op1 = (DataSet) sets.get(info.parentID);
sets.put(info.setID, applyMapOperation(op1, info));
}
private void createMapPartitionOperation(INFO info) {
DataSet op1 = (DataSet) sets.get(info.parentID);
sets.put(info.setID, applyMapPartitionOperation(op1, info));
}
private void createReduceOperation(INFO info) {
Object op1 = sets.get(info.parentID);
if (op1 instanceof DataSet) {
sets.put(info.setID, applyReduceOperation((DataSet) op1, info));
return;
}
if (op1 instanceof UnsortedGrouping) {
sets.put(info.setID, applyReduceOperation((UnsortedGrouping) op1, info));
}
}
protected abstract DataSet applyCoGroupOperation(DataSet op1, DataSet op2, String[] firstKeys, String[] secondKeys, INFO info);
protected abstract DataSet applyCrossOperation(DataSet op1, DataSet op2, DatasizeHint mode, INFO info);
protected abstract DataSet applyFilterOperation(DataSet op1, INFO info);
protected abstract DataSet applyFlatMapOperation(DataSet op1, INFO info);
protected abstract DataSet applyGroupReduceOperation(DataSet op1, INFO info);
protected abstract DataSet applyGroupReduceOperation(UnsortedGrouping op1, INFO info);
protected abstract DataSet applyGroupReduceOperation(SortedGrouping op1, INFO info);
protected abstract DataSet applyJoinOperation(DataSet op1, DataSet op2, String[] firstKeys, String[] secondKeys, DatasizeHint mode, INFO info);
protected abstract DataSet applyMapOperation(DataSet op1, INFO info);
protected abstract DataSet applyMapPartitionOperation(DataSet op1, INFO info);
protected abstract DataSet applyReduceOperation(DataSet op1, INFO info);
protected abstract DataSet applyReduceOperation(UnsortedGrouping op1, INFO info);
//====Utility=======================================================================================================
protected static String[] normalizeKeys(Object keys) {
if (keys instanceof Tuple) {
Tuple tupleKeys = (Tuple) keys;
if (tupleKeys.getArity() == 0) {
return new String[0];
}
if (tupleKeys.getField(0) instanceof Integer) {
String[] stringKeys = new String[tupleKeys.getArity()];
for (int x = 0; x < stringKeys.length; x++) {
stringKeys[x] = "f" + (Integer) tupleKeys.getField(x);
}
return stringKeys;
}
if (tupleKeys.getField(0) instanceof String) {
return tupleToStringArray(tupleKeys);
}
throw new RuntimeException("Key argument contains field that is neither an int nor a String.");
}
if (keys instanceof int[]) {
int[] intKeys = (int[]) keys;
String[] stringKeys = new String[intKeys.length];
for (int x = 0; x < stringKeys.length; x++) {
stringKeys[x] = "f" + intKeys[x];
}
return stringKeys;
}
throw new RuntimeException("Key argument is neither an int[] nor a Tuple.");
}
protected static int[] toIntArray(Object key) {
if (key instanceof Tuple) {
Tuple tuple = (Tuple) key;
int[] keys = new int[tuple.getArity()];
for (int y = 0; y < tuple.getArity(); y++) {
keys[y] = (Integer) tuple.getField(y);
}
return keys;
}
if (key instanceof int[]) {
return (int[]) key;
}
throw new RuntimeException("Key argument is neither an int[] nor a Tuple.");
}
protected static String[] tupleToStringArray(Tuple tuple) {
String[] keys = new String[tuple.getArity()];
for (int y = 0; y < tuple.getArity(); y++) {
keys[y] = (String) tuple.getField(y);
}
return keys;
}
}