-
Notifications
You must be signed in to change notification settings - Fork 4.2k
/
FlinkPipelineOptions.java
189 lines (150 loc) · 6.92 KB
/
FlinkPipelineOptions.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
/*
* 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;
import com.fasterxml.jackson.annotation.JsonIgnore;
import java.util.List;
import org.apache.beam.sdk.options.ApplicationNameOptions;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.StreamingOptions;
import org.apache.flink.runtime.state.StateBackend;
import org.apache.flink.streaming.api.CheckpointingMode;
/** Options which can be used to configure a Flink PipelineRunner. */
public interface FlinkPipelineOptions
extends PipelineOptions, ApplicationNameOptions, StreamingOptions {
String AUTO = "[auto]";
/**
* List of local files to make available to workers.
*
* <p>Jars are placed on the worker's classpath.
*
* <p>The default value is the list of jars from the main program's classpath.
*/
@Description(
"Jar-Files to send to all workers and put on the classpath. "
+ "The default value is all files from the classpath.")
List<String> getFilesToStage();
void setFilesToStage(List<String> value);
/**
* The url of the Flink JobManager on which to execute pipelines. This can either be the the
* address of a cluster JobManager, in the form "host:port" or one of the special Strings
* "[local]", "[collection]" or "[auto]". "[local]" will start a local Flink Cluster in the JVM,
* "[collection]" will execute the pipeline on Java Collections while "[auto]" will let the system
* decide where to execute the pipeline based on the environment.
*/
@Description(
"Address of the Flink Master where the Pipeline should be executed. Can"
+ " either be of the form \"host:port\" or one of the special values [local], "
+ "[collection] or [auto].")
@Default.String(AUTO)
String getFlinkMaster();
void setFlinkMaster(String value);
@Description(
"The degree of parallelism to be used when distributing operations onto workers. "
+ "If the parallelism is not set, the configured Flink default is used, or 1 if none can be found.")
@Default.Integer(-1)
Integer getParallelism();
void setParallelism(Integer value);
@Description(
"The interval between consecutive checkpoints (i.e. snapshots of the current"
+ "pipeline state used for fault tolerance).")
@Default.Long(-1L)
Long getCheckpointingInterval();
void setCheckpointingInterval(Long interval);
@Description("The checkpointing mode that defines consistency guarantee.")
@Default.Enum("EXACTLY_ONCE")
CheckpointingMode getCheckpointingMode();
void setCheckpointingMode(CheckpointingMode mode);
@Description("The maximum time that a checkpoint may take before being discarded.")
@Default.Long(-1L)
Long getCheckpointTimeoutMillis();
void setCheckpointTimeoutMillis(Long checkpointTimeoutMillis);
@Description("The minimal pause before the next checkpoint is triggered.")
@Default.Long(-1L)
Long getMinPauseBetweenCheckpoints();
void setMinPauseBetweenCheckpoints(Long minPauseInterval);
@Description(
"Sets the number of times that failed tasks are re-executed. "
+ "A value of zero effectively disables fault tolerance. A value of -1 indicates "
+ "that the system default value (as defined in the configuration) should be used.")
@Default.Integer(-1)
Integer getNumberOfExecutionRetries();
void setNumberOfExecutionRetries(Integer retries);
@Description(
"Sets the delay between executions. A value of {@code -1} "
+ "indicates that the default value should be used.")
@Default.Long(-1L)
Long getExecutionRetryDelay();
void setExecutionRetryDelay(Long delay);
@Description("Sets the behavior of reusing objects.")
@Default.Boolean(false)
Boolean getObjectReuse();
void setObjectReuse(Boolean reuse);
/**
* State backend to store Beam's state during computation. Note: Only applicable when executing in
* streaming mode.
*/
@Description(
"Sets the state backend to use in streaming mode. "
+ "Otherwise the default is read from the Flink config.")
@JsonIgnore
StateBackend getStateBackend();
void setStateBackend(StateBackend stateBackend);
@Description("Enable/disable Beam metrics in Flink Runner")
@Default.Boolean(true)
Boolean getEnableMetrics();
void setEnableMetrics(Boolean enableMetrics);
/** Enables or disables externalized checkpoints. */
@Description(
"Enables or disables externalized checkpoints. "
+ "Works in conjunction with CheckpointingInterval")
@Default.Boolean(false)
Boolean isExternalizedCheckpointsEnabled();
void setExternalizedCheckpointsEnabled(Boolean externalCheckpoints);
@Description("Sets the behavior of externalized checkpoints on cancellation.")
@Default.Boolean(false)
Boolean getRetainExternalizedCheckpointsOnCancellation();
void setRetainExternalizedCheckpointsOnCancellation(Boolean retainOnCancellation);
@Description("The maximum number of elements in a bundle.")
@Default.Long(1000)
Long getMaxBundleSize();
void setMaxBundleSize(Long size);
@Description("The maximum time to wait before finalising a bundle (in milliseconds).")
@Default.Long(1000)
Long getMaxBundleTimeMills();
void setMaxBundleTimeMills(Long time);
/**
* Whether to shutdown sources when their watermark reaches {@code +Inf}. For production use cases
* you want this to be disabled because Flink will currently (versions {@literal <=} 1.5) stop
* doing checkpoints when any operator (which includes sources) is finished.
*
* <p>Please see <a href="https://issues.apache.org/jira/browse/FLINK-2491">FLINK-2491</a> for
* progress on this issue.
*/
@Description("If set, shutdown sources when their watermark reaches +Inf.")
@Default.Boolean(false)
Boolean isShutdownSourcesOnFinalWatermark();
void setShutdownSourcesOnFinalWatermark(Boolean shutdownOnFinalWatermark);
@Description(
"Interval in milliseconds for sending latency tracking marks from the sources to the sinks. "
+ "Interval value <= 0 disables the feature.")
@Default.Long(0)
Long getLatencyTrackingInterval();
void setLatencyTrackingInterval(Long interval);
}