-
Notifications
You must be signed in to change notification settings - Fork 4
/
FlinkConfig.scala
323 lines (269 loc) · 10.4 KB
/
FlinkConfig.scala
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
package io.epiphanous.flinkrunner.model
import com.typesafe.config.{ConfigFactory, ConfigObject}
import com.typesafe.scalalogging.LazyLogging
import io.epiphanous.flinkrunner.{FlinkRunnerFactory, SEE}
import org.apache.flink.api.java.utils.ParameterTool
import org.apache.flink.contrib.streaming.state.{
PredefinedOptions,
RocksDBStateBackend
}
import org.apache.flink.runtime.state.filesystem.FsStateBackend
import org.apache.flink.streaming.api.TimeCharacteristic
import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
import java.io.File
import java.time.Duration
import java.util.{Properties, List => JList, Map => JMap}
import scala.collection.JavaConverters._
import scala.util.{Failure, Success, Try}
@SerialVersionUID(1544548116L)
class FlinkConfig(
args: Array[String],
factory: FlinkRunnerFactory[_],
sources: Map[String, Seq[Array[Byte]]] = Map.empty,
optConfig: Option[String] = None)
extends LazyLogging
with Serializable {
val (jobName, jobArgs, jobParams) = {
val (n, a) = args match {
case Array(opt, _*) if opt.startsWith("-") => ("help", args)
case Array("help", _*) => ("help", args.tail)
case Array(jn, "help", _*) => (jn, Array("--help") ++ args.tail)
case Array(jn, _*) => (jn, args.tail)
case _ => ("help", args)
}
(n, a, ParameterTool.fromArgs(a))
}
val _config = {
val sc =
Seq(ConfigFactory.load(), ConfigFactory.load("flink-runner.conf"))
val ocf =
if (jobParams.has("config"))
Some(ConfigFactory.parseFile(new File(jobParams.get("config"))))
else None
val ocs = optConfig.map(ConfigFactory.parseString)
// precedence in config is from right to left...
(ocs ++ ocf ++ sc).foldRight(ConfigFactory.empty())((z, c) =>
z.withFallback(c)
)
}
def getCollectionSource(name: String) =
sources.getOrElse(
name,
throw new RuntimeException(s"missing collection source $name")
)
val systemName = _config.getString("system.name")
val jobs = _config.getObject("jobs").unwrapped().keySet().asScala.toSet
def getJobConfig(name: String) = _config.getConfig(s"jobs.$name")
private def _s(path: String): (String, String) = {
val jpath = _j(path)
if (jobParams.has(jpath)) ("a", jpath)
else if (_config.hasPath(jpath)) ("c", jpath)
else if (jobParams.has(path)) ("a", path)
else ("c", path)
}
private def _j(path: String) = s"jobs.$jobName.$path"
def getObject(path: String): ConfigObject = {
val jpath = _j(path)
if (_config.hasPath(jpath)) _config.getObject(jpath)
else _config.getObject(path)
}
def getObjectOption(path: String): Option[ConfigObject] =
Try(getObject(path)).toOption
def getString(path: String): String =
_s(path) match {
case ("a", p) => jobParams.get(p)
case (_, p) => _config.getString(p)
}
def getStringList(path: String): List[String] =
_s(path) match {
case ("a", p) => jobParams.get(p).split("[, ]+").toList
case (_, p) => _config.getStringList(p).asScala.toList
}
def getInt(path: String): Int =
_s(path) match {
case ("a", p) => jobParams.getInt(p)
case (_, p) => _config.getInt(p)
}
def getLong(path: String): Long =
_s(path) match {
case ("a", p) => jobParams.getLong(p)
case (_, p) => _config.getLong(p)
}
def getBoolean(path: String): Boolean =
_s(path) match {
case ("a", p) => jobParams.getBoolean(p)
case (_, p) => _config.getBoolean(p)
}
def getDouble(path: String): Double =
_s(path) match {
case ("a", p) => jobParams.getDouble(p)
case (_, p) => _config.getDouble(p)
}
def getDuration(path: String): Duration =
_s(path) match {
case ("a", p) =>
ConfigFactory
.parseString(s"$p = ${jobParams.get(p)}")
.getDuration(p)
case (_, p) => _config.getDuration(p)
}
def getProperties(path: String): Properties = {
val p = new Properties()
def flatten(key: String, value: Object): Unit = {
val pkey = if (key.isEmpty) key else s"$key."
value match {
case map: JMap[String, Object] @unchecked =>
map.asScala.foreach { case (k, v) => flatten(s"$pkey$k", v) }
case list: JList[Object] @unchecked =>
list.asScala.zipWithIndex.foreach { case (v, i) =>
flatten(s"$pkey$i", v)
}
case v =>
p.put(key, v.toString)
() // force unit return
}
}
(_s(path) match {
case ("a", p) =>
Some(
ConfigFactory
.parseString(s"$p = ${jobParams.get(p)}")
.getObject(p)
)
case (_, p) =>
if (_config.hasPath(p)) Some(_config.getObject(p)) else None
}) match {
case Some(c) => flatten("", c.unwrapped())
case None => // noop
}
p
}
def _classInstance[T](path: String): T =
Class
.forName(getString(path))
.getDeclaredConstructor()
.newInstance()
.asInstanceOf[T]
// def getJobInstance = factory.getJobInstance(jobName, this)
def getDeserializationSchema(name: String) =
factory.getDeserializationSchema(name, this)
def getKafkaDeserializationSchema(name: String) =
factory.getKafkaDeserializationSchema(name, this)
def getKinesisDeserializationSchema(name: String) =
factory.getKinesisDeserializationSchema(name, this)
def getSerializationSchema(name: String) =
factory.getSerializationSchema(name, this)
def getKafkaSerializationSchema(name: String) =
factory.getKafkaSerializationSchema(name, this)
def getKinesisSerializationSchema(name: String) =
factory.getKinesisSerializationSchema(name, this)
def getEncoder(name: String) = factory.getEncoder(name, this)
def getAddToJdbcBatchFunction(name: String) =
factory.getAddToJdbcBatchFunction(name, this)
def getBucketAssigner(name: String) =
factory.getBucketAssigner(name, this)
def getAvroCoder(name: String) = factory.getAvroCoder(name, this)
def getSourceConfig(name: String): SourceConfig =
SourceConfig(name, this)
def getSinkConfig(name: String): SinkConfig = SinkConfig(name, this)
def getSourceNames: Seq[String] =
if (sources.nonEmpty) sources.keySet.toSeq
else
Try(getStringList("source.names")) match {
case Success(sn) => sn
case Failure(_) =>
getObject("sources").unwrapped().keySet().asScala.toSeq
}
def getSinkNames: Seq[String] =
Try(getStringList("sink.names")) match {
case Success(sn) => sn
case Failure(_) =>
getObject("sinks").unwrapped().keySet().asScala.toSeq
}
lazy val environment = getString("environment")
lazy val isDev = environment.startsWith("dev")
lazy val isStage = environment.startsWith("stag")
lazy val isProd = environment.startsWith("prod")
def configureStreamExecutionEnvironment: SEE = {
val env =
if (isDev)
StreamExecutionEnvironment.createLocalEnvironment(1)
else
StreamExecutionEnvironment.getExecutionEnvironment
// use event time
env.setStreamTimeCharacteristic(timeCharacteristic)
// set parallelism
env.setParallelism(globalParallelism)
// configure check-pointing and state backend
if (checkpointInterval > 0) {
env.enableCheckpointing(checkpointInterval)
env.getCheckpointConfig.setMinPauseBetweenCheckpoints(
checkpointMinPause.toMillis
)
env.getCheckpointConfig.setMaxConcurrentCheckpoints(
checkpointMaxConcurrent
)
val backend = if (stateBackend == "rocksdb") {
logger.info(s"Using ROCKS DB state backend at $checkpointUrl")
val rocksBackend =
new RocksDBStateBackend(checkpointUrl, checkpointIncremental)
if (checkpointFlash)
rocksBackend.setPredefinedOptions(
PredefinedOptions.FLASH_SSD_OPTIMIZED
)
rocksBackend
} else {
logger.info(s"Using FILE SYSTEM state backend at $checkpointUrl")
new FsStateBackend(checkpointUrl)
}
/* this deprecation is annoying; its due to rocksdb's state backend
extending AbstractStateBackend which is deprecated */
env.setStateBackend(backend)
}
env
}
def getTimeCharacteristic(tc: String): TimeCharacteristic = {
tc.toLowerCase
.replaceFirst("\\s*time$", "") match {
case "event" => TimeCharacteristic.EventTime
case "processing" => TimeCharacteristic.ProcessingTime
case "ingestion" => TimeCharacteristic.IngestionTime
case unknown =>
throw new RuntimeException(
s"Unknown time.characteristic setting: '$unknown'"
)
}
}
lazy val timeCharacteristic = getTimeCharacteristic(
getString("time.characteristic")
)
def getWatermarkStrategy(ws: String) =
ws.toLowerCase.replaceAll("[^a-z]", "") match {
case "boundedlateness" => "bounded lateness"
case "boundedoutoforderness" => "bounded out of orderness"
case "ascendingtimestamps" => "ascending timestamps"
case "monotonictimestamps" => "ascending timestamps"
case unknown =>
throw new RuntimeException(
s"Unknown watermark.strategy setting: '$unknown'"
)
}
lazy val watermarkStrategy = getWatermarkStrategy(
getString("watermark.strategy")
)
lazy val systemHelp = _config.getString("system.help")
lazy val jobHelp = getString("help")
lazy val jobDescription = getString("description")
lazy val globalParallelism = getInt("global.parallelism")
lazy val checkpointInterval = getLong("checkpoint.interval")
lazy val checkpointMinPause = getDuration("checkpoint.min.pause")
lazy val checkpointMaxConcurrent = getInt("checkpoint.max.concurrent")
lazy val checkpointUrl = getString("checkpoint.url")
lazy val checkpointFlash = getBoolean("checkpoint.flash")
lazy val stateBackend = getString("state.backend").toLowerCase
lazy val checkpointIncremental = getBoolean("checkpoint.incremental")
lazy val showPlan = getBoolean("show.plan")
lazy val mockEdges = isDev && getBoolean("mock.edges")
lazy val maxLateness = getDuration("max.lateness")
lazy val maxIdleness = getDuration("max.idleness")
}