/
DoFnOperator.java
executable file
·1629 lines (1427 loc) · 63.8 KB
/
DoFnOperator.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
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* 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.beam.runners.flink.translation.wrappers.streaming;
import static org.apache.flink.util.Preconditions.checkArgument;
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.io.Serializable;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.locks.Lock;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.beam.runners.core.DoFnRunner;
import org.apache.beam.runners.core.DoFnRunners;
import org.apache.beam.runners.core.InMemoryBundleFinalizer;
import org.apache.beam.runners.core.NullSideInputReader;
import org.apache.beam.runners.core.ProcessFnRunner;
import org.apache.beam.runners.core.PushbackSideInputDoFnRunner;
import org.apache.beam.runners.core.SideInputHandler;
import org.apache.beam.runners.core.SideInputReader;
import org.apache.beam.runners.core.SimplePushbackSideInputDoFnRunner;
import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems;
import org.apache.beam.runners.core.StateInternals;
import org.apache.beam.runners.core.StateNamespace;
import org.apache.beam.runners.core.StateNamespaces.WindowNamespace;
import org.apache.beam.runners.core.StatefulDoFnRunner;
import org.apache.beam.runners.core.StepContext;
import org.apache.beam.runners.core.TimerInternals;
import org.apache.beam.runners.core.TimerInternals.TimerData;
import org.apache.beam.runners.core.construction.SerializablePipelineOptions;
import org.apache.beam.runners.flink.FlinkPipelineOptions;
import org.apache.beam.runners.flink.metrics.DoFnRunnerWithMetricsUpdate;
import org.apache.beam.runners.flink.metrics.FlinkMetricContainer;
import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer;
import org.apache.beam.runners.flink.translation.utils.CheckpointStats;
import org.apache.beam.runners.flink.translation.utils.Workarounds;
import org.apache.beam.runners.flink.translation.wrappers.streaming.stableinput.BufferingDoFnRunner;
import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkBroadcastStateInternals;
import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkStateInternals;
import org.apache.beam.runners.fnexecution.control.BundleCheckpointHandlers.StateAndTimerBundleCheckpointHandler;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.StructuredCoder;
import org.apache.beam.sdk.coders.VarIntCoder;
import org.apache.beam.sdk.io.FileSystems;
import org.apache.beam.sdk.metrics.MetricName;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.state.StateSpec;
import org.apache.beam.sdk.state.TimeDomain;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.DoFn.BundleFinalizer;
import org.apache.beam.sdk.transforms.DoFnSchemaInformation;
import org.apache.beam.sdk.transforms.join.RawUnionValue;
import org.apache.beam.sdk.transforms.reflect.DoFnInvoker;
import org.apache.beam.sdk.transforms.reflect.DoFnInvokers;
import org.apache.beam.sdk.transforms.reflect.DoFnSignature;
import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
import org.apache.beam.sdk.util.NoopLock;
import org.apache.beam.sdk.util.WindowedValue;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollectionView;
import org.apache.beam.sdk.values.TupleTag;
import org.apache.beam.sdk.values.WindowingStrategy;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Joiner;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.state.ListState;
import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.api.common.state.MapState;
import org.apache.flink.api.common.state.MapStateDescriptor;
import org.apache.flink.api.common.typeutils.base.StringSerializer;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.runtime.state.InternalPriorityQueue;
import org.apache.flink.runtime.state.KeyedStateBackend;
import org.apache.flink.runtime.state.OperatorStateBackend;
import org.apache.flink.runtime.state.StateInitializationContext;
import org.apache.flink.runtime.state.StateSnapshotContext;
import org.apache.flink.runtime.state.VoidNamespace;
import org.apache.flink.streaming.api.CheckpointingMode;
import org.apache.flink.streaming.api.graph.StreamConfig;
import org.apache.flink.streaming.api.operators.ChainingStrategy;
import org.apache.flink.streaming.api.operators.InternalTimeServiceManager;
import org.apache.flink.streaming.api.operators.InternalTimer;
import org.apache.flink.streaming.api.operators.InternalTimerService;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.Output;
import org.apache.flink.streaming.api.operators.Triggerable;
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.tasks.StreamTask;
import org.apache.flink.util.OutputTag;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.joda.time.Instant;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Flink operator for executing {@link DoFn DoFns}.
*
* @param <InputT> the input type of the {@link DoFn}
* @param <OutputT> the output type of the {@link DoFn}
*/
// We use Flink's lifecycle methods to initialize transient fields
@SuppressFBWarnings("SE_TRANSIENT_FIELD_NOT_RESTORED")
@SuppressWarnings({
"rawtypes", // TODO(https://issues.apache.org/jira/browse/BEAM-10556)
"keyfor",
"nullness"
}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
public class DoFnOperator<InputT, OutputT>
extends AbstractStreamOperatorCompat<WindowedValue<OutputT>>
implements OneInputStreamOperator<WindowedValue<InputT>, WindowedValue<OutputT>>,
TwoInputStreamOperator<WindowedValue<InputT>, RawUnionValue, WindowedValue<OutputT>>,
Triggerable<ByteBuffer, TimerData> {
private static final Logger LOG = LoggerFactory.getLogger(DoFnOperator.class);
protected DoFn<InputT, OutputT> doFn;
protected final SerializablePipelineOptions serializedOptions;
protected final TupleTag<OutputT> mainOutputTag;
protected final List<TupleTag<?>> additionalOutputTags;
protected final Collection<PCollectionView<?>> sideInputs;
protected final Map<Integer, PCollectionView<?>> sideInputTagMapping;
protected final WindowingStrategy<?, ?> windowingStrategy;
protected final OutputManagerFactory<OutputT> outputManagerFactory;
protected transient DoFnRunner<InputT, OutputT> doFnRunner;
protected transient PushbackSideInputDoFnRunner<InputT, OutputT> pushbackDoFnRunner;
protected transient BufferingDoFnRunner<InputT, OutputT> bufferingDoFnRunner;
protected transient SideInputHandler sideInputHandler;
protected transient SideInputReader sideInputReader;
protected transient BufferedOutputManager<OutputT> outputManager;
private transient DoFnInvoker<InputT, OutputT> doFnInvoker;
protected transient FlinkStateInternals<?> keyedStateInternals;
protected transient FlinkTimerInternals timerInternals;
protected final String stepName;
private final Coder<WindowedValue<InputT>> windowedInputCoder;
private final Map<TupleTag<?>, Coder<?>> outputCoders;
protected final Coder<?> keyCoder;
final KeySelector<WindowedValue<InputT>, ?> keySelector;
private final TimerInternals.TimerDataCoderV2 timerCoder;
/** Max number of elements to include in a bundle. */
private final long maxBundleSize;
/** Max duration of a bundle. */
private final long maxBundleTimeMills;
private final DoFnSchemaInformation doFnSchemaInformation;
private final Map<String, PCollectionView<?>> sideInputMapping;
/** If true, we must process elements only after a checkpoint is finished. */
private final boolean requiresStableInput;
private final boolean finishBundleBeforeCheckpointing;
/** Stores new finalizations being gathered. */
private transient InMemoryBundleFinalizer bundleFinalizer;
/** Pending bundle finalizations which have not been acknowledged yet. */
private transient LinkedHashMap<Long, List<InMemoryBundleFinalizer.Finalization>>
pendingFinalizations;
/**
* Keep a maximum of 32 bundle finalizations for {@link
* BundleFinalizer.Callback#onBundleSuccess()}.
*/
private static final int MAX_NUMBER_PENDING_BUNDLE_FINALIZATIONS = 32;
protected transient InternalTimerService<TimerData> timerService;
private transient InternalTimeServiceManager<?> timeServiceManagerCompat;
private transient PushedBackElementsHandler<WindowedValue<InputT>> pushedBackElementsHandler;
/** Metrics container for reporting Beam metrics to Flink (null if metrics are disabled). */
transient @Nullable FlinkMetricContainer flinkMetricContainer;
/** Helper class to report the checkpoint duration. */
private transient @Nullable CheckpointStats checkpointStats;
/** A timer that finishes the current bundle after a fixed amount of time. */
private transient ScheduledFuture<?> checkFinishBundleTimer;
/**
* This and the below fields need to be volatile because we use multiple threads to access these.
* (a) the main processing thread (b) a timer thread to finish bundles by a timeout instead of the
* number of element However, we do not need a lock because Flink makes sure to acquire the
* "checkpointing" lock for the main processing but also for timer set via its {@code
* timerService}.
*
* <p>The volatile flag can be removed once https://issues.apache.org/jira/browse/FLINK-12481 has
* been addressed.
*/
private transient volatile boolean bundleStarted;
/** Number of processed elements in the current bundle. */
private transient volatile long elementCount;
/** Time that the last bundle was finished (to set the timer). */
private transient volatile long lastFinishBundleTime;
/** Callback to be executed before the current bundle is started. */
private transient volatile Runnable preBundleCallback;
/** Callback to be executed after the current bundle was finished. */
private transient volatile Runnable bundleFinishedCallback;
// Watermark state.
// Volatile because these can be set in two mutually exclusive threads (see above).
private transient volatile long currentInputWatermark;
private transient volatile long currentSideInputWatermark;
private transient volatile long currentOutputWatermark;
private transient volatile long pushedBackWatermark;
/** Constructor for DoFnOperator. */
public DoFnOperator(
DoFn<InputT, OutputT> doFn,
String stepName,
Coder<WindowedValue<InputT>> inputWindowedCoder,
Map<TupleTag<?>, Coder<?>> outputCoders,
TupleTag<OutputT> mainOutputTag,
List<TupleTag<?>> additionalOutputTags,
OutputManagerFactory<OutputT> outputManagerFactory,
WindowingStrategy<?, ?> windowingStrategy,
Map<Integer, PCollectionView<?>> sideInputTagMapping,
Collection<PCollectionView<?>> sideInputs,
PipelineOptions options,
Coder<?> keyCoder,
KeySelector<WindowedValue<InputT>, ?> keySelector,
DoFnSchemaInformation doFnSchemaInformation,
Map<String, PCollectionView<?>> sideInputMapping) {
this.doFn = doFn;
this.stepName = stepName;
this.windowedInputCoder = inputWindowedCoder;
this.outputCoders = outputCoders;
this.mainOutputTag = mainOutputTag;
this.additionalOutputTags = additionalOutputTags;
this.sideInputTagMapping = sideInputTagMapping;
this.sideInputs = sideInputs;
this.serializedOptions = new SerializablePipelineOptions(options);
this.windowingStrategy = windowingStrategy;
this.outputManagerFactory = outputManagerFactory;
setChainingStrategy(ChainingStrategy.ALWAYS);
this.keyCoder = keyCoder;
this.keySelector = keySelector;
this.timerCoder =
TimerInternals.TimerDataCoderV2.of(windowingStrategy.getWindowFn().windowCoder());
FlinkPipelineOptions flinkOptions = options.as(FlinkPipelineOptions.class);
this.maxBundleSize = flinkOptions.getMaxBundleSize();
Preconditions.checkArgument(maxBundleSize > 0, "Bundle size must be at least 1");
this.maxBundleTimeMills = flinkOptions.getMaxBundleTimeMills();
Preconditions.checkArgument(maxBundleTimeMills > 0, "Bundle time must be at least 1");
this.doFnSchemaInformation = doFnSchemaInformation;
this.sideInputMapping = sideInputMapping;
this.requiresStableInput =
// WindowDoFnOperator does not use a DoFn
doFn != null
&& DoFnSignatures.getSignature(doFn.getClass()).processElement().requiresStableInput();
if (requiresStableInput) {
Preconditions.checkState(
CheckpointingMode.valueOf(flinkOptions.getCheckpointingMode())
== CheckpointingMode.EXACTLY_ONCE,
"Checkpointing mode is not set to exactly once but @RequiresStableInput is used.");
Preconditions.checkState(
flinkOptions.getCheckpointingInterval() > 0,
"No checkpointing configured but pipeline uses @RequiresStableInput");
LOG.warn(
"Enabling stable input for transform {}. Will only process elements at most every {} milliseconds.",
stepName,
flinkOptions.getCheckpointingInterval()
+ Math.max(0, flinkOptions.getMinPauseBetweenCheckpoints()));
}
this.finishBundleBeforeCheckpointing = flinkOptions.getFinishBundleBeforeCheckpointing();
}
// allow overriding this in WindowDoFnOperator because this one dynamically creates
// the DoFn
protected DoFn<InputT, OutputT> getDoFn() {
return doFn;
}
// allow overriding this, for example SplittableDoFnOperator will not create a
// stateful DoFn runner because ProcessFn, which is used for executing a Splittable DoFn
// doesn't play by the normal DoFn rules and WindowDoFnOperator uses LateDataDroppingDoFnRunner
protected DoFnRunner<InputT, OutputT> createWrappingDoFnRunner(
DoFnRunner<InputT, OutputT> wrappedRunner, StepContext stepContext) {
if (keyCoder != null) {
StatefulDoFnRunner.CleanupTimer<InputT> cleanupTimer =
new StatefulDoFnRunner.TimeInternalsCleanupTimer<InputT>(
timerInternals, windowingStrategy) {
@Override
public void setForWindow(InputT input, BoundedWindow window) {
if (!window.equals(GlobalWindow.INSTANCE)) {
// Skip setting a cleanup timer for the global window as these timers
// lead to potentially unbounded state growth in the runner, depending on key
// cardinality. Cleanup for global window will be performed upon arrival of the
// final watermark.
super.setForWindow(input, window);
}
}
};
// we don't know the window type
// @SuppressWarnings({"unchecked", "rawtypes"})
Coder windowCoder = windowingStrategy.getWindowFn().windowCoder();
@SuppressWarnings({"unchecked", "rawtypes"})
StatefulDoFnRunner.StateCleaner<?> stateCleaner =
new StatefulDoFnRunner.StateInternalsStateCleaner<>(
doFn, keyedStateInternals, windowCoder);
return DoFnRunners.defaultStatefulDoFnRunner(
doFn,
getInputCoder(),
wrappedRunner,
stepContext,
windowingStrategy,
cleanupTimer,
stateCleaner,
true /* requiresTimeSortedInput is supported */);
} else {
return doFnRunner;
}
}
@Override
public void setup(
StreamTask<?, ?> containingTask,
StreamConfig config,
Output<StreamRecord<WindowedValue<OutputT>>> output) {
// make sure that FileSystems is initialized correctly
FileSystems.setDefaultPipelineOptions(serializedOptions.get());
super.setup(containingTask, config, output);
}
@Override
public void initializeState(StateInitializationContext context) throws Exception {
super.initializeState(context);
ListStateDescriptor<WindowedValue<InputT>> pushedBackStateDescriptor =
new ListStateDescriptor<>(
"pushed-back-elements",
new CoderTypeSerializer<>(windowedInputCoder, serializedOptions));
if (keySelector != null) {
pushedBackElementsHandler =
KeyedPushedBackElementsHandler.create(
keySelector, getKeyedStateBackend(), pushedBackStateDescriptor);
} else {
ListState<WindowedValue<InputT>> listState =
getOperatorStateBackend().getListState(pushedBackStateDescriptor);
pushedBackElementsHandler = NonKeyedPushedBackElementsHandler.create(listState);
}
currentInputWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis();
currentSideInputWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis();
currentOutputWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis();
sideInputReader = NullSideInputReader.of(sideInputs);
if (!sideInputs.isEmpty()) {
FlinkBroadcastStateInternals sideInputStateInternals =
new FlinkBroadcastStateInternals<>(
getContainingTask().getIndexInSubtaskGroup(),
getOperatorStateBackend(),
serializedOptions);
sideInputHandler = new SideInputHandler(sideInputs, sideInputStateInternals);
sideInputReader = sideInputHandler;
Stream<WindowedValue<InputT>> pushedBack = pushedBackElementsHandler.getElements();
long min =
pushedBack.map(v -> v.getTimestamp().getMillis()).reduce(Long.MAX_VALUE, Math::min);
pushedBackWatermark = min;
} else {
pushedBackWatermark = Long.MAX_VALUE;
}
// StatefulPardo or WindowDoFn
if (keyCoder != null) {
keyedStateInternals =
new FlinkStateInternals<>(
(KeyedStateBackend) getKeyedStateBackend(), keyCoder, serializedOptions);
if (timerService == null) {
timerService =
getInternalTimerService(
"beam-timer", new CoderTypeSerializer<>(timerCoder, serializedOptions), this);
}
timerInternals = new FlinkTimerInternals();
timeServiceManagerCompat = getTimeServiceManagerCompat();
}
outputManager =
outputManagerFactory.create(
output, getLockToAcquireForStateAccessDuringBundles(), getOperatorStateBackend());
}
/**
* Subclasses may provide a lock to ensure that the state backend is not accessed concurrently
* during bundle execution.
*/
protected Lock getLockToAcquireForStateAccessDuringBundles() {
return NoopLock.get();
}
@Override
public void open() throws Exception {
// WindowDoFnOperator need use state and timer to get DoFn.
// So must wait StateInternals and TimerInternals ready.
// This will be called after initializeState()
this.doFn = getDoFn();
FlinkPipelineOptions options = serializedOptions.get().as(FlinkPipelineOptions.class);
doFnInvoker = DoFnInvokers.tryInvokeSetupFor(doFn, options);
StepContext stepContext = new FlinkStepContext();
doFnRunner =
DoFnRunners.simpleRunner(
options,
doFn,
sideInputReader,
outputManager,
mainOutputTag,
additionalOutputTags,
stepContext,
getInputCoder(),
outputCoders,
windowingStrategy,
doFnSchemaInformation,
sideInputMapping);
if (requiresStableInput) {
// put this in front of the root FnRunner before any additional wrappers
doFnRunner =
bufferingDoFnRunner =
BufferingDoFnRunner.create(
doFnRunner,
"stable-input-buffer",
windowedInputCoder,
windowingStrategy.getWindowFn().windowCoder(),
getOperatorStateBackend(),
getKeyedStateBackend(),
options.getNumConcurrentCheckpoints(),
serializedOptions);
}
doFnRunner = createWrappingDoFnRunner(doFnRunner, stepContext);
earlyBindStateIfNeeded();
if (!options.getDisableMetrics()) {
flinkMetricContainer = new FlinkMetricContainer(getRuntimeContext());
doFnRunner = new DoFnRunnerWithMetricsUpdate<>(stepName, doFnRunner, flinkMetricContainer);
String checkpointMetricNamespace = options.getReportCheckpointDuration();
if (checkpointMetricNamespace != null) {
MetricName checkpointMetric =
MetricName.named(checkpointMetricNamespace, "checkpoint_duration");
checkpointStats =
new CheckpointStats(
() ->
flinkMetricContainer
.getMetricsContainer(stepName)
.getDistribution(checkpointMetric));
}
}
elementCount = 0L;
lastFinishBundleTime = getProcessingTimeService().getCurrentProcessingTime();
// Schedule timer to check timeout of finish bundle.
long bundleCheckPeriod = Math.max(maxBundleTimeMills / 2, 1);
checkFinishBundleTimer =
getProcessingTimeService()
.scheduleAtFixedRate(
timestamp -> checkInvokeFinishBundleByTime(), bundleCheckPeriod, bundleCheckPeriod);
if (doFn instanceof SplittableParDoViaKeyedWorkItems.ProcessFn) {
pushbackDoFnRunner =
new ProcessFnRunner<>((DoFnRunner) doFnRunner, sideInputs, sideInputHandler);
} else {
pushbackDoFnRunner =
SimplePushbackSideInputDoFnRunner.create(doFnRunner, sideInputs, sideInputHandler);
}
bundleFinalizer = new InMemoryBundleFinalizer();
pendingFinalizations = new LinkedHashMap<>();
}
private void earlyBindStateIfNeeded() throws IllegalArgumentException, IllegalAccessException {
if (keyCoder != null) {
if (doFn != null) {
DoFnSignature signature = DoFnSignatures.getSignature(doFn.getClass());
FlinkStateInternals.EarlyBinder earlyBinder =
new FlinkStateInternals.EarlyBinder(getKeyedStateBackend(), serializedOptions);
for (DoFnSignature.StateDeclaration value : signature.stateDeclarations().values()) {
StateSpec<?> spec =
(StateSpec<?>) signature.stateDeclarations().get(value.id()).field().get(doFn);
spec.bind(value.id(), earlyBinder);
}
if (doFnRunner instanceof StatefulDoFnRunner) {
((StatefulDoFnRunner<InputT, OutputT, BoundedWindow>) doFnRunner)
.getSystemStateTags()
.forEach(tag -> tag.getSpec().bind(tag.getId(), earlyBinder));
}
}
}
}
@Override
public void dispose() throws Exception {
try {
Optional.ofNullable(flinkMetricContainer)
.ifPresent(FlinkMetricContainer::registerMetricsForPipelineResult);
Optional.ofNullable(checkFinishBundleTimer).ifPresent(timer -> timer.cancel(true));
Workarounds.deleteStaticCaches();
Optional.ofNullable(doFnInvoker).ifPresent(DoFnInvoker::invokeTeardown);
} finally {
// This releases all task's resources. We need to call this last
// to ensure that state, timers, or output buffers can still be
// accessed during finishing the bundle.
super.dispose();
}
}
@Override
public void close() throws Exception {
try {
// This is our last change to block shutdown of this operator while
// there are still remaining processing-time timers. Flink will ignore pending
// processing-time timers when upstream operators have shut down and will also
// shut down this operator with pending processing-time timers.
if (numProcessingTimeTimers() > 0) {
timerInternals.processPendingProcessingTimeTimers();
}
if (numProcessingTimeTimers() > 0) {
throw new RuntimeException(
"There are still "
+ numProcessingTimeTimers()
+ " processing-time timers left, this indicates a bug");
}
// make sure we send a +Inf watermark downstream. It can happen that we receive +Inf
// in processWatermark*() but have holds, so we have to re-evaluate here.
processWatermark(new Watermark(Long.MAX_VALUE));
// Make sure to finish the current bundle
while (bundleStarted) {
invokeFinishBundle();
}
if (currentOutputWatermark < Long.MAX_VALUE) {
throw new RuntimeException(
"There are still watermark holds. Watermark held at " + currentOutputWatermark);
}
} finally {
super.close();
}
// sanity check: these should have been flushed out by +Inf watermarks
if (!sideInputs.isEmpty()) {
List<WindowedValue<InputT>> pushedBackElements =
pushedBackElementsHandler.getElements().collect(Collectors.toList());
if (pushedBackElements.size() > 0) {
String pushedBackString = Joiner.on(",").join(pushedBackElements);
throw new RuntimeException(
"Leftover pushed-back data: " + pushedBackString + ". This indicates a bug.");
}
}
}
public long getEffectiveInputWatermark() {
// hold back by the pushed back values waiting for side inputs
return Math.min(pushedBackWatermark, currentInputWatermark);
}
public long getCurrentOutputWatermark() {
return currentOutputWatermark;
}
protected final void setPreBundleCallback(Runnable callback) {
this.preBundleCallback = callback;
}
protected final void setBundleFinishedCallback(Runnable callback) {
this.bundleFinishedCallback = callback;
}
@Override
public final void processElement(StreamRecord<WindowedValue<InputT>> streamRecord) {
checkInvokeStartBundle();
long oldHold = keyCoder != null ? keyedStateInternals.minWatermarkHoldMs() : -1L;
doFnRunner.processElement(streamRecord.getValue());
checkInvokeFinishBundleByCount();
emitWatermarkIfHoldChanged(oldHold);
}
@Override
public final void processElement1(StreamRecord<WindowedValue<InputT>> streamRecord)
throws Exception {
checkInvokeStartBundle();
Iterable<WindowedValue<InputT>> justPushedBack =
pushbackDoFnRunner.processElementInReadyWindows(streamRecord.getValue());
long min = pushedBackWatermark;
for (WindowedValue<InputT> pushedBackValue : justPushedBack) {
min = Math.min(min, pushedBackValue.getTimestamp().getMillis());
pushedBackElementsHandler.pushBack(pushedBackValue);
}
pushedBackWatermark = min;
checkInvokeFinishBundleByCount();
}
/**
* Add the side input value. Here we are assuming that views have already been materialized and
* are sent over the wire as {@link Iterable}. Subclasses may elect to perform materialization in
* state and receive side input incrementally instead.
*
* @param streamRecord
*/
protected void addSideInputValue(StreamRecord<RawUnionValue> streamRecord) {
@SuppressWarnings("unchecked")
WindowedValue<Iterable<?>> value =
(WindowedValue<Iterable<?>>) streamRecord.getValue().getValue();
PCollectionView<?> sideInput = sideInputTagMapping.get(streamRecord.getValue().getUnionTag());
sideInputHandler.addSideInputValue(sideInput, value);
}
@Override
public final void processElement2(StreamRecord<RawUnionValue> streamRecord) throws Exception {
// we finish the bundle because the newly arrived side-input might
// make a view available that was previously not ready.
// The PushbackSideInputRunner will only reset it's cache of non-ready windows when
// finishing a bundle.
invokeFinishBundle();
checkInvokeStartBundle();
// add the side input, which may cause pushed back elements become eligible for processing
addSideInputValue(streamRecord);
List<WindowedValue<InputT>> newPushedBack = new ArrayList<>();
Iterator<WindowedValue<InputT>> it = pushedBackElementsHandler.getElements().iterator();
while (it.hasNext()) {
WindowedValue<InputT> element = it.next();
// we need to set the correct key in case the operator is
// a (keyed) window operator
setKeyContextElement1(new StreamRecord<>(element));
Iterable<WindowedValue<InputT>> justPushedBack =
pushbackDoFnRunner.processElementInReadyWindows(element);
Iterables.addAll(newPushedBack, justPushedBack);
}
pushedBackElementsHandler.clear();
long min = Long.MAX_VALUE;
for (WindowedValue<InputT> pushedBackValue : newPushedBack) {
min = Math.min(min, pushedBackValue.getTimestamp().getMillis());
pushedBackElementsHandler.pushBack(pushedBackValue);
}
pushedBackWatermark = min;
checkInvokeFinishBundleByCount();
// maybe output a new watermark
processWatermark1(new Watermark(currentInputWatermark));
}
@Override
public final void processWatermark(Watermark mark) throws Exception {
processWatermark1(mark);
}
@Override
public final void processWatermark1(Watermark mark) throws Exception {
// Flush any data buffered during snapshotState().
outputManager.flushBuffer();
// We do the check here because we are guaranteed to at least get the +Inf watermark on the
// main input when the job finishes.
if (currentSideInputWatermark >= BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()) {
// this means we will never see any more side input
// we also do the check here because we might have received the side-input MAX watermark
// before receiving any main-input data
emitAllPushedBackData();
}
currentInputWatermark = mark.getTimestamp();
processInputWatermark(true);
}
private void processInputWatermark(boolean advanceInputWatermark) throws Exception {
long inputWatermarkHold = applyInputWatermarkHold(getEffectiveInputWatermark());
if (keyCoder != null && advanceInputWatermark) {
timeServiceManagerCompat.advanceWatermark(new Watermark(inputWatermarkHold));
}
long potentialOutputWatermark =
applyOutputWatermarkHold(
currentOutputWatermark, computeOutputWatermark(inputWatermarkHold));
maybeEmitWatermark(potentialOutputWatermark);
}
/**
* Allows to apply a hold to the input watermark. By default, just passes the input watermark
* through.
*/
public long applyInputWatermarkHold(long inputWatermark) {
return inputWatermark;
}
/**
* Allows to apply a hold to the output watermark before it is send out. By default, just passes
* the potential output watermark through which will make it the new output watermark.
*
* @param currentOutputWatermark the current output watermark
* @param potentialOutputWatermark The potential new output watermark which can be adjusted, if
* needed. The input watermark hold has already been applied.
* @return The new output watermark which will be emitted.
*/
public long applyOutputWatermarkHold(long currentOutputWatermark, long potentialOutputWatermark) {
return potentialOutputWatermark;
}
private long computeOutputWatermark(long inputWatermarkHold) {
final long potentialOutputWatermark;
if (keyCoder == null) {
potentialOutputWatermark = inputWatermarkHold;
} else {
potentialOutputWatermark =
Math.min(keyedStateInternals.minWatermarkHoldMs(), inputWatermarkHold);
}
return potentialOutputWatermark;
}
private void maybeEmitWatermark(long watermark) {
if (watermark > currentOutputWatermark) {
// Must invoke finishBatch before emit the +Inf watermark otherwise there are some late
// events.
if (watermark >= BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()) {
invokeFinishBundle();
}
LOG.debug("Emitting watermark {}", watermark);
currentOutputWatermark = watermark;
output.emitWatermark(new Watermark(watermark));
// Check if the final watermark was triggered to perform state cleanup for global window
if (keyedStateInternals != null
&& currentOutputWatermark
> adjustTimestampForFlink(GlobalWindow.INSTANCE.maxTimestamp().getMillis())) {
keyedStateInternals.clearGlobalState();
}
}
}
@Override
public final void processWatermark2(Watermark mark) throws Exception {
currentSideInputWatermark = mark.getTimestamp();
if (mark.getTimestamp() >= BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()) {
// this means we will never see any more side input
emitAllPushedBackData();
// maybe output a new watermark
processWatermark1(new Watermark(currentInputWatermark));
}
}
/**
* Emits all pushed-back data. This should be used once we know that there will not be any future
* side input, i.e. that there is no point in waiting.
*/
private void emitAllPushedBackData() throws Exception {
Iterator<WindowedValue<InputT>> it = pushedBackElementsHandler.getElements().iterator();
while (it.hasNext()) {
checkInvokeStartBundle();
WindowedValue<InputT> element = it.next();
// we need to set the correct key in case the operator is
// a (keyed) window operator
setKeyContextElement1(new StreamRecord<>(element));
doFnRunner.processElement(element);
}
pushedBackElementsHandler.clear();
pushedBackWatermark = Long.MAX_VALUE;
}
/**
* Check whether invoke startBundle, if it is, need to output elements that were buffered as part
* of finishing a bundle in snapshot() first.
*
* <p>In order to avoid having {@link DoFnRunner#processElement(WindowedValue)} or {@link
* DoFnRunner#onTimer(String, String, Object, BoundedWindow, Instant, Instant, TimeDomain)} not
* between StartBundle and FinishBundle, this method needs to be called in each processElement and
* each processWatermark and onProcessingTime. Do not need to call in onEventTime, because it has
* been guaranteed in the processWatermark.
*/
private void checkInvokeStartBundle() {
if (!bundleStarted) {
// Flush any data buffered during snapshotState().
outputManager.flushBuffer();
LOG.debug("Starting bundle.");
if (preBundleCallback != null) {
preBundleCallback.run();
}
pushbackDoFnRunner.startBundle();
bundleStarted = true;
}
}
/** Check whether invoke finishBundle by elements count. Called in processElement. */
@SuppressWarnings("NonAtomicVolatileUpdate")
@SuppressFBWarnings("VO_VOLATILE_INCREMENT")
private void checkInvokeFinishBundleByCount() {
// We do not access this statement concurrently but we want to make sure that each thread
// sees the latest value, which is why we use volatile. See the class field section above
// for more information.
//noinspection NonAtomicOperationOnVolatileField
elementCount++;
if (elementCount >= maxBundleSize) {
invokeFinishBundle();
}
}
/** Check whether invoke finishBundle by timeout. */
private void checkInvokeFinishBundleByTime() {
long now = getProcessingTimeService().getCurrentProcessingTime();
if (now - lastFinishBundleTime >= maxBundleTimeMills) {
invokeFinishBundle();
}
}
protected final void invokeFinishBundle() {
if (bundleStarted) {
LOG.debug("Finishing bundle.");
pushbackDoFnRunner.finishBundle();
LOG.debug("Finished bundle. Element count: {}", elementCount);
elementCount = 0L;
lastFinishBundleTime = getProcessingTimeService().getCurrentProcessingTime();
bundleStarted = false;
// callback only after current bundle was fully finalized
// it could start a new bundle, for example resulting from timer processing
if (bundleFinishedCallback != null) {
LOG.debug("Invoking bundle finish callback.");
bundleFinishedCallback.run();
}
}
}
@Override
public void prepareSnapshotPreBarrier(long checkpointId) {
if (finishBundleBeforeCheckpointing) {
// We finish the bundle and flush any pending data.
// This avoids buffering any data as part of snapshotState() below.
while (bundleStarted) {
invokeFinishBundle();
}
}
}
@Override
public final void snapshotState(StateSnapshotContext context) throws Exception {
if (checkpointStats != null) {
checkpointStats.snapshotStart(context.getCheckpointId());
}
if (requiresStableInput) {
// We notify the BufferingDoFnRunner to associate buffered state with this
// snapshot id and start a new buffer for elements arriving after this snapshot.
bufferingDoFnRunner.checkpoint(context.getCheckpointId());
}
int diff = pendingFinalizations.size() - MAX_NUMBER_PENDING_BUNDLE_FINALIZATIONS;
if (diff >= 0) {
for (Iterator<Long> iterator = pendingFinalizations.keySet().iterator(); diff >= 0; diff--) {
iterator.next();
iterator.remove();
}
}
pendingFinalizations.put(context.getCheckpointId(), bundleFinalizer.getAndClearFinalizations());
try {
outputManager.openBuffer();
// Ensure that no new bundle gets started as part of finishing a bundle
while (bundleStarted) {
invokeFinishBundle();
}
outputManager.closeBuffer();
} catch (Exception e) {
// https://jira.apache.org/jira/browse/FLINK-14653
// Any regular exception during checkpointing will be tolerated by Flink because those
// typically do not affect the execution flow. We need to fail hard here because errors
// in bundle execution are application errors which are not related to checkpointing.
throw new Error("Checkpointing failed because bundle failed to finalize.", e);
}
super.snapshotState(context);
}
public BundleFinalizer getBundleFinalizer() {
return bundleFinalizer;
}
@Override
public void notifyCheckpointComplete(long checkpointId) throws Exception {
if (checkpointStats != null) {
checkpointStats.reportCheckpointDuration(checkpointId);
}
if (requiresStableInput) {
// We can now release all buffered data which was held back for
// @RequiresStableInput guarantees.
bufferingDoFnRunner.checkpointCompleted(checkpointId);
}
List<InMemoryBundleFinalizer.Finalization> finalizations =
pendingFinalizations.remove(checkpointId);
if (finalizations != null) {
// confirm all finalizations that were associated with the checkpoint
for (InMemoryBundleFinalizer.Finalization finalization : finalizations) {
finalization.getCallback().onBundleSuccess();
}
}
super.notifyCheckpointComplete(checkpointId);
}
@Override
public void onEventTime(InternalTimer<ByteBuffer, TimerData> timer) {
checkInvokeStartBundle();
fireTimerInternal(timer.getKey(), timer.getNamespace());
}
@Override
public void onProcessingTime(InternalTimer<ByteBuffer, TimerData> timer) {
checkInvokeStartBundle();
fireTimerInternal(timer.getKey(), timer.getNamespace());
}
// allow overriding this in ExecutableStageDoFnOperator to set the key context
protected void fireTimerInternal(ByteBuffer key, TimerData timerData) {
long oldHold = keyCoder != null ? keyedStateInternals.minWatermarkHoldMs() : -1L;