forked from apache/flink
/
StreamExecutionEnvironment.java
2352 lines (2153 loc) · 109 KB
/
StreamExecutionEnvironment.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.flink.streaming.api.environment;
import org.apache.flink.annotation.Experimental;
import org.apache.flink.annotation.Internal;
import org.apache.flink.annotation.Public;
import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.InvalidProgramException;
import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.common.RuntimeExecutionMode;
import org.apache.flink.api.common.cache.DistributedCache;
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
import org.apache.flink.api.common.functions.InvalidTypesException;
import org.apache.flink.api.common.io.FileInputFormat;
import org.apache.flink.api.common.io.FilePathFilter;
import org.apache.flink.api.common.io.InputFormat;
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.connector.source.Boundedness;
import org.apache.flink.api.connector.source.Source;
import org.apache.flink.api.connector.source.lib.NumberSequenceSource;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.api.java.ClosureCleaner;
import org.apache.flink.api.java.Utils;
import org.apache.flink.api.java.io.TextInputFormat;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.api.java.typeutils.MissingTypeInfo;
import org.apache.flink.api.java.typeutils.PojoTypeInfo;
import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.CoreOptions;
import org.apache.flink.configuration.DeploymentOptions;
import org.apache.flink.configuration.ExecutionOptions;
import org.apache.flink.configuration.PipelineOptions;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.configuration.RestOptions;
import org.apache.flink.core.execution.DefaultExecutorServiceLoader;
import org.apache.flink.core.execution.DetachedJobExecutionResult;
import org.apache.flink.core.execution.JobClient;
import org.apache.flink.core.execution.JobListener;
import org.apache.flink.core.execution.PipelineExecutor;
import org.apache.flink.core.execution.PipelineExecutorFactory;
import org.apache.flink.core.execution.PipelineExecutorServiceLoader;
import org.apache.flink.core.fs.Path;
import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
import org.apache.flink.runtime.state.StateBackend;
import org.apache.flink.runtime.state.StateBackendLoader;
import org.apache.flink.streaming.api.CheckpointingMode;
import org.apache.flink.streaming.api.TimeCharacteristic;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.DataStreamSource;
import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
import org.apache.flink.streaming.api.functions.source.ContinuousFileMonitoringFunction;
import org.apache.flink.streaming.api.functions.source.ContinuousFileReaderOperatorFactory;
import org.apache.flink.streaming.api.functions.source.FileMonitoringFunction;
import org.apache.flink.streaming.api.functions.source.FileProcessingMode;
import org.apache.flink.streaming.api.functions.source.FileReadFunction;
import org.apache.flink.streaming.api.functions.source.FromElementsFunction;
import org.apache.flink.streaming.api.functions.source.FromIteratorFunction;
import org.apache.flink.streaming.api.functions.source.FromSplittableIteratorFunction;
import org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction;
import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
import org.apache.flink.streaming.api.functions.source.SocketTextStreamFunction;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.streaming.api.functions.source.StatefulSequenceSource;
import org.apache.flink.streaming.api.functions.source.TimestampedFileInputSplit;
import org.apache.flink.streaming.api.graph.StreamGraph;
import org.apache.flink.streaming.api.graph.StreamGraphGenerator;
import org.apache.flink.streaming.api.graph.StreamingJobGraphGenerator;
import org.apache.flink.streaming.api.operators.StreamSource;
import org.apache.flink.util.DynamicCodeLoadingException;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.FlinkException;
import org.apache.flink.util.InstantiationUtil;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.SplittableIterator;
import org.apache.flink.util.StringUtils;
import org.apache.flink.util.WrappingRuntimeException;
import com.esotericsoftware.kryo.Serializer;
import javax.annotation.Nullable;
import java.io.IOException;
import java.io.Serializable;
import java.net.URI;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import static org.apache.flink.util.Preconditions.checkNotNull;
/**
* The StreamExecutionEnvironment is the context in which a streaming program is executed. A {@link
* LocalStreamEnvironment} will cause execution in the current JVM, a {@link
* RemoteStreamEnvironment} will cause execution on a remote setup.
*
* <p>The environment provides methods to control the job execution (such as setting the parallelism
* or the fault tolerance/checkpointing parameters) and to interact with the outside world (data
* access).
*
* @see org.apache.flink.streaming.api.environment.LocalStreamEnvironment
* @see org.apache.flink.streaming.api.environment.RemoteStreamEnvironment
*/
@Public
public class StreamExecutionEnvironment {
/** The default name to use for a streaming job if no other name has been specified. */
public static final String DEFAULT_JOB_NAME = "Flink Streaming Job";
/** The time characteristic that is used if none other is set. */
private static final TimeCharacteristic DEFAULT_TIME_CHARACTERISTIC =
TimeCharacteristic.EventTime;
/**
* The environment of the context (local by default, cluster if invoked through command line).
*/
private static StreamExecutionEnvironmentFactory contextEnvironmentFactory = null;
/** The ThreadLocal used to store {@link StreamExecutionEnvironmentFactory}. */
private static final ThreadLocal<StreamExecutionEnvironmentFactory>
threadLocalContextEnvironmentFactory = new ThreadLocal<>();
/** The default parallelism used when creating a local environment. */
private static int defaultLocalParallelism = Runtime.getRuntime().availableProcessors();
// ------------------------------------------------------------------------
/** The execution configuration for this environment. */
private final ExecutionConfig config = new ExecutionConfig();
/** Settings that control the checkpointing behavior. */
private final CheckpointConfig checkpointCfg = new CheckpointConfig();
protected final List<Transformation<?>> transformations = new ArrayList<>();
private long bufferTimeout = StreamingJobGraphGenerator.UNDEFINED_NETWORK_BUFFER_TIMEOUT;
protected boolean isChainingEnabled = true;
/** The state backend used for storing k/v state and state snapshots. */
private StateBackend defaultStateBackend;
/** The default savepoint directory used by the job. */
private Path defaultSavepointDirectory;
/** The time characteristic used by the data streams. */
private TimeCharacteristic timeCharacteristic = DEFAULT_TIME_CHARACTERISTIC;
protected final List<Tuple2<String, DistributedCache.DistributedCacheEntry>> cacheFile =
new ArrayList<>();
private final PipelineExecutorServiceLoader executorServiceLoader;
private final Configuration configuration;
private final ClassLoader userClassloader;
private final List<JobListener> jobListeners = new ArrayList<>();
// --------------------------------------------------------------------------------------------
// Constructor and Properties
// --------------------------------------------------------------------------------------------
public StreamExecutionEnvironment() {
this(new Configuration());
// unfortunately, StreamExecutionEnvironment always (implicitly) had a public constructor.
// This constructor is not useful because the execution environment cannot be used for
// execution. We're keeping this to appease the binary compatibiliy checks.
}
/**
* Creates a new {@link StreamExecutionEnvironment} that will use the given {@link
* Configuration} to configure the {@link PipelineExecutor}.
*/
@PublicEvolving
public StreamExecutionEnvironment(final Configuration configuration) {
this(configuration, null);
}
/**
* Creates a new {@link StreamExecutionEnvironment} that will use the given {@link
* Configuration} to configure the {@link PipelineExecutor}.
*
* <p>In addition, this constructor allows specifying the user code {@link ClassLoader}.
*/
@PublicEvolving
public StreamExecutionEnvironment(
final Configuration configuration, final ClassLoader userClassloader) {
this(new DefaultExecutorServiceLoader(), configuration, userClassloader);
}
/**
* Creates a new {@link StreamExecutionEnvironment} that will use the given {@link
* Configuration} to configure the {@link PipelineExecutor}.
*
* <p>In addition, this constructor allows specifying the {@link PipelineExecutorServiceLoader}
* and user code {@link ClassLoader}.
*/
@PublicEvolving
public StreamExecutionEnvironment(
final PipelineExecutorServiceLoader executorServiceLoader,
final Configuration configuration,
final ClassLoader userClassloader) {
this.executorServiceLoader = checkNotNull(executorServiceLoader);
this.configuration = new Configuration(checkNotNull(configuration));
this.userClassloader =
userClassloader == null ? getClass().getClassLoader() : userClassloader;
// the configuration of a job or an operator can be specified at the following places:
// i) at the operator level using e.g. parallelism using the
// SingleOutputStreamOperator.setParallelism().
// ii) programmatically by using e.g. the env.setRestartStrategy() method
// iii) in the configuration passed here
//
// if specified in multiple places, the priority order is the above.
//
// Given this, it is safe to overwrite the execution config default values here because all
// other ways assume
// that the env is already instantiated so they will overwrite the value passed here.
this.configure(this.configuration, this.userClassloader);
}
protected Configuration getConfiguration() {
return this.configuration;
}
protected ClassLoader getUserClassloader() {
return userClassloader;
}
/** Gets the config object. */
public ExecutionConfig getConfig() {
return config;
}
/**
* Get the list of cached files that were registered for distribution among the task managers.
*/
public List<Tuple2<String, DistributedCache.DistributedCacheEntry>> getCachedFiles() {
return cacheFile;
}
/** Gets the config JobListeners. */
@PublicEvolving
public List<JobListener> getJobListeners() {
return jobListeners;
}
/**
* Sets the parallelism for operations executed through this environment. Setting a parallelism
* of x here will cause all operators (such as map, batchReduce) to run with x parallel
* instances. This method overrides the default parallelism for this environment. The {@link
* LocalStreamEnvironment} uses by default a value equal to the number of hardware contexts (CPU
* cores / threads). When executing the program via the command line client from a JAR file, the
* default degree of parallelism is the one configured for that setup.
*
* @param parallelism The parallelism
*/
public StreamExecutionEnvironment setParallelism(int parallelism) {
config.setParallelism(parallelism);
return this;
}
/**
* Sets the runtime execution mode for the application (see {@link RuntimeExecutionMode}). This
* is equivalent to setting the {@code execution.runtime-mode} in your application's
* configuration file.
*
* <p>We recommend users to NOT use this method but set the {@code execution.runtime-mode} using
* the command-line when submitting the application. Keeping the application code
* configuration-free allows for more flexibility as the same application will be able to be
* executed in any execution mode.
*
* @param executionMode the desired execution mode.
* @return The execution environment of your application.
*/
@PublicEvolving
public StreamExecutionEnvironment setRuntimeMode(final RuntimeExecutionMode executionMode) {
checkNotNull(executionMode);
configuration.set(ExecutionOptions.RUNTIME_MODE, executionMode);
return this;
}
/**
* Sets the maximum degree of parallelism defined for the program. The upper limit (inclusive)
* is Short.MAX_VALUE.
*
* <p>The maximum degree of parallelism specifies the upper limit for dynamic scaling. It also
* defines the number of key groups used for partitioned state.
*
* @param maxParallelism Maximum degree of parallelism to be used for the program., with {@code
* 0 < maxParallelism <= 2^15 - 1}.
*/
public StreamExecutionEnvironment setMaxParallelism(int maxParallelism) {
Preconditions.checkArgument(
maxParallelism > 0
&& maxParallelism <= KeyGroupRangeAssignment.UPPER_BOUND_MAX_PARALLELISM,
"maxParallelism is out of bounds 0 < maxParallelism <= "
+ KeyGroupRangeAssignment.UPPER_BOUND_MAX_PARALLELISM
+ ". Found: "
+ maxParallelism);
config.setMaxParallelism(maxParallelism);
return this;
}
/**
* Gets the parallelism with which operation are executed by default. Operations can
* individually override this value to use a specific parallelism.
*
* @return The parallelism used by operations, unless they override that value.
*/
public int getParallelism() {
return config.getParallelism();
}
/**
* Gets the maximum degree of parallelism defined for the program.
*
* <p>The maximum degree of parallelism specifies the upper limit for dynamic scaling. It also
* defines the number of key groups used for partitioned state.
*
* @return Maximum degree of parallelism
*/
public int getMaxParallelism() {
return config.getMaxParallelism();
}
/**
* Sets the maximum time frequency (milliseconds) for the flushing of the output buffers. By
* default the output buffers flush frequently to provide low latency and to aid smooth
* developer experience. Setting the parameter can result in three logical modes:
*
* <ul>
* <li>A positive integer triggers flushing periodically by that integer
* <li>0 triggers flushing after every record thus minimizing latency
* <li>-1 triggers flushing only when the output buffer is full thus maximizing throughput
* </ul>
*
* @param timeoutMillis The maximum time between two output flushes.
*/
public StreamExecutionEnvironment setBufferTimeout(long timeoutMillis) {
if (timeoutMillis < -1) {
throw new IllegalArgumentException("Timeout of buffer must be non-negative or -1");
}
this.bufferTimeout = timeoutMillis;
return this;
}
/**
* Gets the maximum time frequency (milliseconds) for the flushing of the output buffers. For
* clarification on the extremal values see {@link #setBufferTimeout(long)}.
*
* @return The timeout of the buffer.
*/
public long getBufferTimeout() {
return this.bufferTimeout;
}
/**
* Disables operator chaining for streaming operators. Operator chaining allows non-shuffle
* operations to be co-located in the same thread fully avoiding serialization and
* de-serialization.
*
* @return StreamExecutionEnvironment with chaining disabled.
*/
@PublicEvolving
public StreamExecutionEnvironment disableOperatorChaining() {
this.isChainingEnabled = false;
return this;
}
/**
* Returns whether operator chaining is enabled.
*
* @return {@code true} if chaining is enabled, false otherwise.
*/
@PublicEvolving
public boolean isChainingEnabled() {
return isChainingEnabled;
}
// ------------------------------------------------------------------------
// Checkpointing Settings
// ------------------------------------------------------------------------
/**
* Gets the checkpoint config, which defines values like checkpoint interval, delay between
* checkpoints, etc.
*
* @return The checkpoint config.
*/
public CheckpointConfig getCheckpointConfig() {
return checkpointCfg;
}
/**
* Enables checkpointing for the streaming job. The distributed state of the streaming dataflow
* will be periodically snapshotted. In case of a failure, the streaming dataflow will be
* restarted from the latest completed checkpoint. This method selects {@link
* CheckpointingMode#EXACTLY_ONCE} guarantees.
*
* <p>The job draws checkpoints periodically, in the given interval. The state will be stored in
* the configured state backend.
*
* <p>NOTE: Checkpointing iterative streaming dataflows in not properly supported at the moment.
* For that reason, iterative jobs will not be started if used with enabled checkpointing. To
* override this mechanism, use the {@link #enableCheckpointing(long, CheckpointingMode,
* boolean)} method.
*
* @param interval Time interval between state checkpoints in milliseconds.
*/
public StreamExecutionEnvironment enableCheckpointing(long interval) {
checkpointCfg.setCheckpointInterval(interval);
return this;
}
/**
* Enables checkpointing for the streaming job. The distributed state of the streaming dataflow
* will be periodically snapshotted. In case of a failure, the streaming dataflow will be
* restarted from the latest completed checkpoint.
*
* <p>The job draws checkpoints periodically, in the given interval. The system uses the given
* {@link CheckpointingMode} for the checkpointing ("exactly once" vs "at least once"). The
* state will be stored in the configured state backend.
*
* <p>NOTE: Checkpointing iterative streaming dataflows in not properly supported at the moment.
* For that reason, iterative jobs will not be started if used with enabled checkpointing. To
* override this mechanism, use the {@link #enableCheckpointing(long, CheckpointingMode,
* boolean)} method.
*
* @param interval Time interval between state checkpoints in milliseconds.
* @param mode The checkpointing mode, selecting between "exactly once" and "at least once"
* guaranteed.
*/
public StreamExecutionEnvironment enableCheckpointing(long interval, CheckpointingMode mode) {
checkpointCfg.setCheckpointingMode(mode);
checkpointCfg.setCheckpointInterval(interval);
return this;
}
/**
* Enables checkpointing for the streaming job. The distributed state of the streaming dataflow
* will be periodically snapshotted. In case of a failure, the streaming dataflow will be
* restarted from the latest completed checkpoint.
*
* <p>The job draws checkpoints periodically, in the given interval. The state will be stored in
* the configured state backend.
*
* <p>NOTE: Checkpointing iterative streaming dataflows in not properly supported at the moment.
* If the "force" parameter is set to true, the system will execute the job nonetheless.
*
* @param interval Time interval between state checkpoints in millis.
* @param mode The checkpointing mode, selecting between "exactly once" and "at least once"
* guaranteed.
* @param force If true checkpointing will be enabled for iterative jobs as well.
* @deprecated Use {@link #enableCheckpointing(long, CheckpointingMode)} instead. Forcing
* checkpoints will be removed in the future.
*/
@Deprecated
@SuppressWarnings("deprecation")
@PublicEvolving
public StreamExecutionEnvironment enableCheckpointing(
long interval, CheckpointingMode mode, boolean force) {
checkpointCfg.setCheckpointingMode(mode);
checkpointCfg.setCheckpointInterval(interval);
checkpointCfg.setForceCheckpointing(force);
return this;
}
/**
* Enables checkpointing for the streaming job. The distributed state of the streaming dataflow
* will be periodically snapshotted. In case of a failure, the streaming dataflow will be
* restarted from the latest completed checkpoint. This method selects {@link
* CheckpointingMode#EXACTLY_ONCE} guarantees.
*
* <p>The job draws checkpoints periodically, in the default interval. The state will be stored
* in the configured state backend.
*
* <p>NOTE: Checkpointing iterative streaming dataflows in not properly supported at the moment.
* For that reason, iterative jobs will not be started if used with enabled checkpointing. To
* override this mechanism, use the {@link #enableCheckpointing(long, CheckpointingMode,
* boolean)} method.
*
* @deprecated Use {@link #enableCheckpointing(long)} instead.
*/
@Deprecated
@PublicEvolving
public StreamExecutionEnvironment enableCheckpointing() {
checkpointCfg.setCheckpointInterval(500);
return this;
}
/**
* Returns the checkpointing interval or -1 if checkpointing is disabled.
*
* <p>Shorthand for {@code getCheckpointConfig().getCheckpointInterval()}.
*
* @return The checkpointing interval or -1
*/
public long getCheckpointInterval() {
return checkpointCfg.getCheckpointInterval();
}
/**
* Returns whether checkpointing is force-enabled.
*
* @deprecated Forcing checkpoints will be removed in future version.
*/
@Deprecated
@SuppressWarnings("deprecation")
@PublicEvolving
public boolean isForceCheckpointing() {
return checkpointCfg.isForceCheckpointing();
}
/** Returns whether Unaligned Checkpoints are enabled. */
@PublicEvolving
public boolean isUnalignedCheckpointsEnabled() {
return checkpointCfg.isUnalignedCheckpointsEnabled();
}
/** Returns whether Unaligned Checkpoints are force-enabled. */
@PublicEvolving
public boolean isForceUnalignedCheckpoints() {
return checkpointCfg.isForceUnalignedCheckpoints();
}
/**
* Returns the checkpointing mode (exactly-once vs. at-least-once).
*
* <p>Shorthand for {@code getCheckpointConfig().getCheckpointingMode()}.
*
* @return The checkpoint mode
*/
public CheckpointingMode getCheckpointingMode() {
return checkpointCfg.getCheckpointingMode();
}
/**
* Sets the state backend that describes how to store operator. It defines the data structures
* that hold state during execution (for example hash tables, RocksDB, or other data stores).
*
* <p>State managed by the state backend includes both keyed state that is accessible on {@link
* org.apache.flink.streaming.api.datastream.KeyedStream keyed streams}, as well as state
* maintained directly by the user code that implements {@link
* org.apache.flink.streaming.api.checkpoint.CheckpointedFunction CheckpointedFunction}.
*
* <p>The {@link org.apache.flink.runtime.state.hashmap.HashMapStateBackend} maintains state in
* heap memory, as objects. It is lightweight without extra dependencies, but is limited to JVM
* heap memory.
*
* <p>In contrast, the {@code EmbeddedRocksDBStateBackend} stores its state in an embedded
* {@code RocksDB} instance. This state backend can store very large state that exceeds memory
* and spills to local disk. All key/value state (including windows) is stored in the key/value
* index of RocksDB.
*
* <p>In both cases, fault tolerance is managed via the jobs {@link
* org.apache.flink.runtime.state.CheckpointStorage} which configures how and where state
* backends persist during a checkpoint.
*
* @return This StreamExecutionEnvironment itself, to allow chaining of function calls.
* @see #getStateBackend()
* @see CheckpointConfig#setCheckpointStorage( org.apache.flink.runtime.state.CheckpointStorage)
*/
@PublicEvolving
public StreamExecutionEnvironment setStateBackend(StateBackend backend) {
this.defaultStateBackend = Preconditions.checkNotNull(backend);
return this;
}
/**
* Gets the state backend that defines how to store and checkpoint state.
*
* @see #setStateBackend(StateBackend)
*/
@PublicEvolving
public StateBackend getStateBackend() {
return defaultStateBackend;
}
/**
* Sets the default savepoint directory, where savepoints will be written to if no is explicitly
* provided when triggered.
*
* @return This StreamExecutionEnvironment itself, to allow chaining of function calls.
* @see #getDefaultSavepointDirectory()
*/
@PublicEvolving
public StreamExecutionEnvironment setDefaultSavepointDirectory(String savepointDirectory) {
Preconditions.checkNotNull(savepointDirectory);
return setDefaultSavepointDirectory(new Path(savepointDirectory));
}
/**
* Sets the default savepoint directory, where savepoints will be written to if no is explicitly
* provided when triggered.
*
* @return This StreamExecutionEnvironment itself, to allow chaining of function calls.
* @see #getDefaultSavepointDirectory()
*/
@PublicEvolving
public StreamExecutionEnvironment setDefaultSavepointDirectory(URI savepointDirectory) {
Preconditions.checkNotNull(savepointDirectory);
return setDefaultSavepointDirectory(new Path(savepointDirectory));
}
/**
* Sets the default savepoint directory, where savepoints will be written to if no is explicitly
* provided when triggered.
*
* @return This StreamExecutionEnvironment itself, to allow chaining of function calls.
* @see #getDefaultSavepointDirectory()
*/
@PublicEvolving
public StreamExecutionEnvironment setDefaultSavepointDirectory(Path savepointDirectory) {
this.defaultSavepointDirectory = Preconditions.checkNotNull(savepointDirectory);
return this;
}
/**
* Gets the default savepoint directory for this Job.
*
* @see #setDefaultSavepointDirectory(Path)
*/
@Nullable
@PublicEvolving
public Path getDefaultSavepointDirectory() {
return defaultSavepointDirectory;
}
/**
* Sets the restart strategy configuration. The configuration specifies which restart strategy
* will be used for the execution graph in case of a restart.
*
* @param restartStrategyConfiguration Restart strategy configuration to be set
*/
@PublicEvolving
public void setRestartStrategy(
RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration) {
config.setRestartStrategy(restartStrategyConfiguration);
}
/**
* Returns the specified restart strategy configuration.
*
* @return The restart strategy configuration to be used
*/
@PublicEvolving
public RestartStrategies.RestartStrategyConfiguration getRestartStrategy() {
return config.getRestartStrategy();
}
/**
* Sets the number of times that failed tasks are re-executed. A value of zero effectively
* disables fault tolerance. A value of {@code -1} indicates that the system default value (as
* defined in the configuration) should be used.
*
* @param numberOfExecutionRetries The number of times the system will try to re-execute failed
* tasks.
* @deprecated This method will be replaced by {@link #setRestartStrategy}. The {@link
* RestartStrategies#fixedDelayRestart(int, Time)} contains the number of execution retries.
*/
@Deprecated
@PublicEvolving
public void setNumberOfExecutionRetries(int numberOfExecutionRetries) {
config.setNumberOfExecutionRetries(numberOfExecutionRetries);
}
/**
* Gets the number of times the system will try to re-execute failed tasks. A value of {@code
* -1} indicates that the system default value (as defined in the configuration) should be used.
*
* @return The number of times the system will try to re-execute failed tasks.
* @deprecated This method will be replaced by {@link #getRestartStrategy}.
*/
@Deprecated
@PublicEvolving
public int getNumberOfExecutionRetries() {
return config.getNumberOfExecutionRetries();
}
// --------------------------------------------------------------------------------------------
// Registry for types and serializers
// --------------------------------------------------------------------------------------------
/**
* Adds a new Kryo default serializer to the Runtime.
*
* <p>Note that the serializer instance must be serializable (as defined by
* java.io.Serializable), because it may be distributed to the worker nodes by java
* serialization.
*
* @param type The class of the types serialized with the given serializer.
* @param serializer The serializer to use.
*/
public <T extends Serializer<?> & Serializable> void addDefaultKryoSerializer(
Class<?> type, T serializer) {
config.addDefaultKryoSerializer(type, serializer);
}
/**
* Adds a new Kryo default serializer to the Runtime.
*
* @param type The class of the types serialized with the given serializer.
* @param serializerClass The class of the serializer to use.
*/
public void addDefaultKryoSerializer(
Class<?> type, Class<? extends Serializer<?>> serializerClass) {
config.addDefaultKryoSerializer(type, serializerClass);
}
/**
* Registers the given type with a Kryo Serializer.
*
* <p>Note that the serializer instance must be serializable (as defined by
* java.io.Serializable), because it may be distributed to the worker nodes by java
* serialization.
*
* @param type The class of the types serialized with the given serializer.
* @param serializer The serializer to use.
*/
public <T extends Serializer<?> & Serializable> void registerTypeWithKryoSerializer(
Class<?> type, T serializer) {
config.registerTypeWithKryoSerializer(type, serializer);
}
/**
* Registers the given Serializer via its class as a serializer for the given type at the
* KryoSerializer.
*
* @param type The class of the types serialized with the given serializer.
* @param serializerClass The class of the serializer to use.
*/
@SuppressWarnings("rawtypes")
public void registerTypeWithKryoSerializer(
Class<?> type, Class<? extends Serializer> serializerClass) {
config.registerTypeWithKryoSerializer(type, serializerClass);
}
/**
* Registers the given type with the serialization stack. If the type is eventually serialized
* as a POJO, then the type is registered with the POJO serializer. If the type ends up being
* serialized with Kryo, then it will be registered at Kryo to make sure that only tags are
* written.
*
* @param type The class of the type to register.
*/
public void registerType(Class<?> type) {
if (type == null) {
throw new NullPointerException("Cannot register null type class.");
}
TypeInformation<?> typeInfo = TypeExtractor.createTypeInfo(type);
if (typeInfo instanceof PojoTypeInfo) {
config.registerPojoType(type);
} else {
config.registerKryoType(type);
}
}
// --------------------------------------------------------------------------------------------
// Time characteristic
// --------------------------------------------------------------------------------------------
/**
* Sets the time characteristic for all streams create from this environment, e.g., processing
* time, event time, or ingestion time.
*
* <p>If you set the characteristic to IngestionTime of EventTime this will set a default
* watermark update interval of 200 ms. If this is not applicable for your application you
* should change it using {@link ExecutionConfig#setAutoWatermarkInterval(long)}.
*
* @param characteristic The time characteristic.
* @deprecated In Flink 1.12 the default stream time characteristic has been changed to {@link
* TimeCharacteristic#EventTime}, thus you don't need to call this method for enabling
* event-time support anymore. Explicitly using processing-time windows and timers works in
* event-time mode. If you need to disable watermarks, please use {@link
* ExecutionConfig#setAutoWatermarkInterval(long)}. If you are using {@link
* TimeCharacteristic#IngestionTime}, please manually set an appropriate {@link
* WatermarkStrategy}. If you are using generic "time window" operations (for example {@link
* org.apache.flink.streaming.api.datastream.KeyedStream#timeWindow(org.apache.flink.streaming.api.windowing.time.Time)}
* that change behaviour based on the time characteristic, please use equivalent operations
* that explicitly specify processing time or event time.
*/
@PublicEvolving
@Deprecated
public void setStreamTimeCharacteristic(TimeCharacteristic characteristic) {
this.timeCharacteristic = Preconditions.checkNotNull(characteristic);
if (characteristic == TimeCharacteristic.ProcessingTime) {
getConfig().setAutoWatermarkInterval(0);
} else {
getConfig().setAutoWatermarkInterval(200);
}
}
/**
* Gets the time characteristic.
*
* @deprecated See {@link #setStreamTimeCharacteristic(TimeCharacteristic)} for deprecation
* notice.
*/
@PublicEvolving
@Deprecated
public TimeCharacteristic getStreamTimeCharacteristic() {
return timeCharacteristic;
}
/**
* Sets all relevant options contained in the {@link ReadableConfig} such as e.g. {@link
* StreamPipelineOptions#TIME_CHARACTERISTIC}. It will reconfigure {@link
* StreamExecutionEnvironment}, {@link ExecutionConfig} and {@link CheckpointConfig}.
*
* <p>It will change the value of a setting only if a corresponding option was set in the {@code
* configuration}. If a key is not present, the current value of a field will remain untouched.
*
* @param configuration a configuration to read the values from
* @param classLoader a class loader to use when loading classes
*/
@PublicEvolving
public void configure(ReadableConfig configuration, ClassLoader classLoader) {
configuration
.getOptional(StreamPipelineOptions.TIME_CHARACTERISTIC)
.ifPresent(this::setStreamTimeCharacteristic);
Optional.ofNullable(loadStateBackend(configuration, classLoader))
.ifPresent(this::setStateBackend);
configuration
.getOptional(PipelineOptions.OPERATOR_CHAINING)
.ifPresent(c -> this.isChainingEnabled = c);
configuration
.getOptional(ExecutionOptions.BUFFER_TIMEOUT)
.ifPresent(t -> this.setBufferTimeout(t.toMillis()));
configuration
.getOptional(DeploymentOptions.JOB_LISTENERS)
.ifPresent(listeners -> registerCustomListeners(classLoader, listeners));
configuration
.getOptional(PipelineOptions.CACHED_FILES)
.ifPresent(
f -> {
this.cacheFile.clear();
this.cacheFile.addAll(DistributedCache.parseCachedFilesFromString(f));
});
configuration
.getOptional(ExecutionOptions.RUNTIME_MODE)
.ifPresent(
runtimeMode ->
this.configuration.set(ExecutionOptions.RUNTIME_MODE, runtimeMode));
configuration
.getOptional(ExecutionOptions.SORT_INPUTS)
.ifPresent(
sortInputs ->
this.getConfiguration()
.set(ExecutionOptions.SORT_INPUTS, sortInputs));
configuration
.getOptional(ExecutionOptions.USE_BATCH_STATE_BACKEND)
.ifPresent(
sortInputs ->
this.getConfiguration()
.set(ExecutionOptions.USE_BATCH_STATE_BACKEND, sortInputs));
configuration
.getOptional(PipelineOptions.NAME)
.ifPresent(jobName -> this.getConfiguration().set(PipelineOptions.NAME, jobName));
config.configure(configuration, classLoader);
checkpointCfg.configure(configuration);
}
private void registerCustomListeners(
final ClassLoader classLoader, final List<String> listeners) {
for (String listener : listeners) {
try {
final JobListener jobListener =
InstantiationUtil.instantiate(listener, JobListener.class, classLoader);
jobListeners.add(jobListener);
} catch (FlinkException e) {
throw new WrappingRuntimeException("Could not load JobListener : " + listener, e);
}
}
}
private StateBackend loadStateBackend(ReadableConfig configuration, ClassLoader classLoader) {
try {
return StateBackendLoader.loadStateBackendFromConfig(configuration, classLoader, null);
} catch (DynamicCodeLoadingException | IOException e) {
throw new WrappingRuntimeException(e);
}
}
// --------------------------------------------------------------------------------------------
// Data stream creations
// --------------------------------------------------------------------------------------------
/**
* Creates a new data stream that contains a sequence of numbers. This is a parallel source, if
* you manually set the parallelism to {@code 1} (using {@link
* org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator#setParallelism(int)})
* the generated sequence of elements is in order.
*
* @param from The number to start at (inclusive)
* @param to The number to stop at (inclusive)
* @return A data stream, containing all number in the [from, to] interval
* @deprecated Use {@link #fromSequence(long, long)} instead to create a new data stream that
* contains {@link org.apache.flink.api.connector.source.lib.NumberSequenceSource}.
*/
@Deprecated
public DataStreamSource<Long> generateSequence(long from, long to) {
if (from > to) {
throw new IllegalArgumentException(
"Start of sequence must not be greater than the end");
}
return addSource(new StatefulSequenceSource(from, to), "Sequence Source (Deprecated)");
}
/**
* Creates a new data stream that contains a sequence of numbers (longs) and is useful for
* testing and for cases that just need a stream of N events of any kind.
*
* <p>The generated source splits the sequence into as many parallel sub-sequences as there are
* parallel source readers. Each sub-sequence will be produced in order. If the parallelism is
* limited to one, the source will produce one sequence in order.
*
* <p>This source is always bounded. For very long sequences (for example over the entire domain
* of long integer values), you may consider executing the application in a streaming manner
* because of the end bound that is pretty far away.
*
* <p>Use {@link #fromSource(Source, WatermarkStrategy, String)} together with {@link
* NumberSequenceSource} if you required more control over the created sources. For example, if
* you want to set a {@link WatermarkStrategy}.
*
* @param from The number to start at (inclusive)
* @param to The number to stop at (inclusive)
*/
public DataStreamSource<Long> fromSequence(long from, long to) {
if (from > to) {
throw new IllegalArgumentException(
"Start of sequence must not be greater than the end");
}
return fromSource(
new NumberSequenceSource(from, to),
WatermarkStrategy.noWatermarks(),
"Sequence Source");
}
/**
* Creates a new data stream that contains the given elements. The elements must all be of the
* same type, for example, all of the {@link String} or {@link Integer}.
*
* <p>The framework will try and determine the exact type from the elements. In case of generic
* elements, it may be necessary to manually supply the type information via {@link
* #fromCollection(java.util.Collection, org.apache.flink.api.common.typeinfo.TypeInformation)}.
*
* <p>Note that this operation will result in a non-parallel data stream source, i.e. a data
* stream source with a degree of parallelism one.
*
* @param data The array of elements to create the data stream from.
* @param <OUT> The type of the returned data stream
* @return The data stream representing the given array of elements
*/
@SafeVarargs
public final <OUT> DataStreamSource<OUT> fromElements(OUT... data) {
if (data.length == 0) {
throw new IllegalArgumentException(
"fromElements needs at least one element as argument");
}
TypeInformation<OUT> typeInfo;
try {
typeInfo = TypeExtractor.getForObject(data[0]);
} catch (Exception e) {
throw new RuntimeException(
"Could not create TypeInformation for type "
+ data[0].getClass().getName()