-
Notifications
You must be signed in to change notification settings - Fork 374
/
ScalingMetricCollector.java
507 lines (439 loc) · 22.3 KB
/
ScalingMetricCollector.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
/*
* 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.autoscaler;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.JobStatus;
import org.apache.flink.autoscaler.config.AutoScalerOptions;
import org.apache.flink.autoscaler.exceptions.NotReadyException;
import org.apache.flink.autoscaler.metrics.CollectedMetricHistory;
import org.apache.flink.autoscaler.metrics.CollectedMetrics;
import org.apache.flink.autoscaler.metrics.FlinkMetric;
import org.apache.flink.autoscaler.metrics.MetricNotFoundException;
import org.apache.flink.autoscaler.metrics.ScalingMetric;
import org.apache.flink.autoscaler.metrics.ScalingMetrics;
import org.apache.flink.autoscaler.state.AutoScalerStateStore;
import org.apache.flink.autoscaler.topology.IOMetrics;
import org.apache.flink.autoscaler.topology.JobTopology;
import org.apache.flink.client.program.rest.RestClusterClient;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
import org.apache.flink.runtime.rest.messages.JobIDPathParameter;
import org.apache.flink.runtime.rest.messages.JobVertexIdPathParameter;
import org.apache.flink.runtime.rest.messages.job.JobDetailsInfo;
import org.apache.flink.runtime.rest.messages.job.metrics.AggregatedMetric;
import org.apache.flink.runtime.rest.messages.job.metrics.AggregatedSubtaskMetricsHeaders;
import org.apache.flink.runtime.rest.messages.job.metrics.AggregatedSubtaskMetricsParameters;
import org.apache.flink.runtime.rest.messages.job.metrics.Metric;
import org.apache.flink.util.Preconditions;
import lombok.SneakyThrows;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.time.Clock;
import java.time.Duration;
import java.time.Instant;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.SortedMap;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import java.util.function.Supplier;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.apache.flink.autoscaler.metrics.ScalingHistoryUtils.updateVertexList;
import static org.apache.flink.autoscaler.utils.AutoScalerUtils.excludeVerticesFromScaling;
import static org.apache.flink.autoscaler.utils.DateTimeUtils.readable;
import static org.apache.flink.util.Preconditions.checkState;
/** Metric collector using flink rest api. */
public abstract class ScalingMetricCollector<KEY, Context extends JobAutoScalerContext<KEY>> {
private static final Logger LOG = LoggerFactory.getLogger(ScalingMetricCollector.class);
private final Map<KEY, Map<JobVertexID, Map<String, FlinkMetric>>> availableVertexMetricNames =
new ConcurrentHashMap<>();
private final Map<KEY, SortedMap<Instant, CollectedMetrics>> histories =
new ConcurrentHashMap<>();
protected final Map<KEY, Boolean> jobsWithGcMetrics = new ConcurrentHashMap<KEY, Boolean>();
private Clock clock = Clock.systemDefaultZone();
public CollectedMetricHistory updateMetrics(
Context ctx, AutoScalerStateStore<KEY, Context> stateStore) throws Exception {
var jobKey = ctx.getJobKey();
var conf = ctx.getConfiguration();
var now = clock.instant();
var metricHistory =
histories.computeIfAbsent(
jobKey,
(k) -> {
try {
return stateStore.getCollectedMetrics(ctx);
} catch (Exception exception) {
throw new RuntimeException(
"Get evaluated metrics failed.", exception);
}
});
var jobDetailsInfo =
getJobDetailsInfo(ctx, conf.get(AutoScalerOptions.FLINK_CLIENT_TIMEOUT));
var jobRunningTs = getJobRunningTs(jobDetailsInfo);
// We detect job change compared to our collected metrics by checking against the earliest
// metric timestamp
if (!metricHistory.isEmpty() && jobRunningTs.isAfter(metricHistory.firstKey())) {
LOG.info("Job updated at {}. Clearing metrics.", readable(jobRunningTs));
stateStore.removeCollectedMetrics(ctx);
cleanup(ctx.getJobKey());
metricHistory.clear();
}
var topology = getJobTopology(ctx, stateStore, jobDetailsInfo);
var stableTime = jobRunningTs.plus(conf.get(AutoScalerOptions.STABILIZATION_INTERVAL));
final boolean isStabilizing = now.isBefore(stableTime);
// Calculate timestamp when the metric windows is full
var metricWindowSize = getMetricWindowSize(conf);
var windowFullTime =
getWindowFullTime(metricHistory.tailMap(stableTime), now, metricWindowSize);
// The filtered list of metrics we want to query for each vertex
var filteredVertexMetricNames = queryFilteredMetricNames(ctx, topology, isStabilizing);
// Aggregated job vertex metrics collected from Flink based on the filtered metric names
var collectedVertexMetrics = queryAllAggregatedMetrics(ctx, filteredVertexMetricNames);
var collectedJmMetrics = queryJmMetrics(ctx);
var collectedTmMetrics = queryTmMetrics(ctx);
// The computed scaling metrics based on the collected aggregated vertex metrics
var scalingMetrics =
convertToScalingMetrics(
jobKey,
collectedVertexMetrics,
collectedJmMetrics,
collectedTmMetrics,
topology,
conf);
// Add scaling metrics to history if they were computed successfully
metricHistory.put(now, scalingMetrics);
if (isStabilizing) {
LOG.info("Stabilizing until {}", readable(stableTime));
stateStore.storeCollectedMetrics(ctx, metricHistory);
return new CollectedMetricHistory(topology, Collections.emptySortedMap(), jobRunningTs);
}
var collectedMetrics = new CollectedMetricHistory(topology, metricHistory, jobRunningTs);
if (now.isBefore(windowFullTime)) {
LOG.info("Metric window not full until {}", readable(windowFullTime));
} else {
collectedMetrics.setFullyCollected(true);
// Trim metrics outside the metric window from metrics history
metricHistory.headMap(now.minus(metricWindowSize)).clear();
}
stateStore.storeCollectedMetrics(ctx, metricHistory);
return collectedMetrics;
}
protected abstract Map<FlinkMetric, Metric> queryJmMetrics(Context ctx) throws Exception;
protected abstract Map<FlinkMetric, AggregatedMetric> queryTmMetrics(Context ctx)
throws Exception;
protected Duration getMetricWindowSize(Configuration conf) {
return conf.get(AutoScalerOptions.METRICS_WINDOW);
}
private static Instant getWindowFullTime(
SortedMap<Instant, CollectedMetrics> metricsAfterStable,
Instant now,
Duration metricWindowSize) {
return metricsAfterStable.isEmpty()
? now.plus(metricWindowSize)
: metricsAfterStable.firstKey().plus(metricWindowSize);
}
@VisibleForTesting
protected Instant getJobRunningTs(JobDetailsInfo jobDetailsInfo) {
final Map<JobStatus, Long> timestamps = jobDetailsInfo.getTimestamps();
final Long runningTs = timestamps.get(JobStatus.RUNNING);
checkState(runningTs != null, "Unable to find when the job was switched to RUNNING.");
return Instant.ofEpochMilli(runningTs);
}
protected JobTopology getJobTopology(
Context ctx,
AutoScalerStateStore<KEY, Context> stateStore,
JobDetailsInfo jobDetailsInfo)
throws Exception {
var t = getJobTopology(jobDetailsInfo);
Set<JobVertexID> vertexSet = Set.copyOf(t.getVerticesInTopologicalOrder());
updateVertexList(stateStore, ctx, clock.instant(), vertexSet);
updateKafkaSourceMaxParallelisms(ctx, jobDetailsInfo.getJobId(), t);
excludeVerticesFromScaling(ctx.getConfiguration(), t.getFinishedVertices());
return t;
}
@VisibleForTesting
@SneakyThrows
protected JobTopology getJobTopology(JobDetailsInfo jobDetailsInfo) {
Map<JobVertexID, Integer> maxParallelismMap =
jobDetailsInfo.getJobVertexInfos().stream()
.collect(
Collectors.toMap(
JobDetailsInfo.JobVertexDetailsInfo::getJobVertexID,
JobDetailsInfo.JobVertexDetailsInfo::getMaxParallelism));
// Flink 1.17 introduced a strange behaviour where the JobDetailsInfo#getJsonPlan()
// doesn't return the jsonPlan correctly but it wraps with RawJson{json='plan'}
var rawPlan = jobDetailsInfo.getJsonPlan();
var json = rawPlan.substring("RawJson{json='".length(), rawPlan.length() - "'}".length());
var metrics = new HashMap<JobVertexID, IOMetrics>();
var finished = new HashSet<JobVertexID>();
jobDetailsInfo
.getJobVertexInfos()
.forEach(
d -> {
if (d.getExecutionState() == ExecutionState.FINISHED) {
finished.add(d.getJobVertexID());
}
metrics.put(
d.getJobVertexID(), IOMetrics.from(d.getJobVertexMetrics()));
});
return JobTopology.fromJsonPlan(json, maxParallelismMap, metrics, finished);
}
private void updateKafkaSourceMaxParallelisms(Context ctx, JobID jobId, JobTopology topology)
throws Exception {
try (var restClient = ctx.getRestClusterClient()) {
var partitionRegex = Pattern.compile("^.*\\.partition\\.\\d+\\.currentOffset$");
for (var vertexInfo : topology.getVertexInfos().values()) {
if (vertexInfo.getInputs().isEmpty()) {
var sourceVertex = vertexInfo.getId();
var numPartitions =
queryAggregatedMetricNames(restClient, jobId, sourceVertex).stream()
.filter(partitionRegex.asMatchPredicate())
.count();
if (numPartitions > 0) {
LOG.debug(
"Updating source {} max parallelism based on available partitions to {}",
sourceVertex,
numPartitions);
topology.updateMaxParallelism(sourceVertex, (int) numPartitions);
}
}
}
}
}
/**
* Given a series of collected Flink vertex metrics we compute the scaling metrics for each job
* vertex.
*/
private CollectedMetrics convertToScalingMetrics(
KEY jobKey,
Map<JobVertexID, Map<FlinkMetric, AggregatedMetric>> collectedMetrics,
Map<FlinkMetric, Metric> collectedJmMetrics,
Map<FlinkMetric, AggregatedMetric> collectedTmMetrics,
JobTopology jobTopology,
Configuration conf) {
var out = new HashMap<JobVertexID, Map<ScalingMetric, Double>>();
var finishedVertices = jobTopology.getFinishedVertices();
if (!finishedVertices.isEmpty()) {
collectedMetrics = new HashMap<>(collectedMetrics);
for (JobVertexID v : finishedVertices) {
collectedMetrics.put(v, FlinkMetric.FINISHED_METRICS);
}
}
collectedMetrics.forEach(
(jobVertexID, vertexFlinkMetrics) -> {
LOG.debug(
"Calculating vertex scaling metrics for {} from {}",
jobVertexID,
vertexFlinkMetrics);
var vertexScalingMetrics = new HashMap<ScalingMetric, Double>();
out.put(jobVertexID, vertexScalingMetrics);
if (jobTopology.isSource(jobVertexID)) {
ScalingMetrics.computeLagMetrics(vertexFlinkMetrics, vertexScalingMetrics);
}
ScalingMetrics.computeLoadMetrics(
jobVertexID,
vertexFlinkMetrics,
vertexScalingMetrics,
jobTopology.get(jobVertexID).getIoMetrics(),
conf);
var metricHistory =
histories.getOrDefault(jobKey, Collections.emptySortedMap());
ScalingMetrics.computeDataRateMetrics(
jobVertexID,
vertexFlinkMetrics,
vertexScalingMetrics,
jobTopology,
conf,
observedTprAvg(
jobVertexID,
metricHistory,
conf.get(
AutoScalerOptions
.OBSERVED_TRUE_PROCESSING_RATE_MIN_OBSERVATIONS)));
vertexScalingMetrics
.entrySet()
.forEach(e -> e.setValue(ScalingMetrics.roundMetric(e.getValue())));
LOG.debug(
"Vertex scaling metrics for {}: {}", jobVertexID, vertexScalingMetrics);
});
var globalMetrics =
ScalingMetrics.computeGlobalMetrics(collectedJmMetrics, collectedTmMetrics, conf);
LOG.debug("Global metrics: {}", globalMetrics);
return new CollectedMetrics(out, globalMetrics);
}
private static Supplier<Double> observedTprAvg(
JobVertexID jobVertexID,
SortedMap<Instant, CollectedMetrics> metricHistory,
int minObservations) {
return () ->
ScalingMetricEvaluator.getAverage(
ScalingMetric.OBSERVED_TPR, jobVertexID, metricHistory, minObservations);
}
private Map<JobVertexID, Map<String, FlinkMetric>> queryFilteredMetricNames(
Context ctx, JobTopology topology, boolean isStabilizing) {
try {
return queryFilteredMetricNames(ctx, topology);
} catch (MetricNotFoundException e) {
if (isStabilizing) {
throw new NotReadyException(e);
}
throw e;
}
}
/** Query the available metric names for each job vertex. */
protected Map<JobVertexID, Map<String, FlinkMetric>> queryFilteredMetricNames(
Context ctx, JobTopology topology) {
var vertices = topology.getVerticesInTopologicalOrder();
var names =
availableVertexMetricNames.compute(
ctx.getJobKey(),
(k, previousMetricNames) -> {
if (previousMetricNames != null
&& previousMetricNames
.keySet()
.equals(topology.getVertexInfos().keySet())) {
var newMetricNames = new HashMap<>(previousMetricNames);
var sourceMetricNames =
queryFilteredMetricNames(
ctx,
topology,
vertices.stream().filter(topology::isSource));
newMetricNames.putAll(sourceMetricNames);
return newMetricNames;
}
// Query all metric names
return queryFilteredMetricNames(ctx, topology, vertices.stream());
});
names.keySet().removeAll(topology.getFinishedVertices());
return names;
}
@SneakyThrows
private Map<JobVertexID, Map<String, FlinkMetric>> queryFilteredMetricNames(
Context ctx, JobTopology topology, Stream<JobVertexID> vertexStream) {
try (var restClient = ctx.getRestClusterClient()) {
return vertexStream
.filter(v -> !topology.getFinishedVertices().contains(v))
.collect(
Collectors.toMap(
v -> v,
v ->
getFilteredVertexMetricNames(
restClient, ctx.getJobID(), v, topology)));
}
}
/** Query and filter metric names for a given job vertex. */
Map<String, FlinkMetric> getFilteredVertexMetricNames(
RestClusterClient<?> restClient,
JobID jobID,
JobVertexID jobVertexID,
JobTopology topology) {
var allMetricNames = queryAggregatedMetricNames(restClient, jobID, jobVertexID);
var filteredMetrics = new HashMap<String, FlinkMetric>();
var requiredMetrics = new HashSet<FlinkMetric>();
requiredMetrics.add(FlinkMetric.BUSY_TIME_PER_SEC);
if (topology.isSource(jobVertexID)) {
requiredMetrics.add(FlinkMetric.BACKPRESSURE_TIME_PER_SEC);
requiredMetrics.add(FlinkMetric.SOURCE_TASK_NUM_RECORDS_IN_PER_SEC);
requiredMetrics.add(FlinkMetric.SOURCE_TASK_NUM_RECORDS_IN);
// Pending records metric won't be available for some sources.
// The Kafka source, for instance, lazily initializes this metric on receiving
// the first record. If this is a fresh topic or no new data has been read since
// the last checkpoint, the pendingRecords metrics won't be available. Also, legacy
// sources do not have this metric.
List<String> pendingRecordsMetric = FlinkMetric.PENDING_RECORDS.findAll(allMetricNames);
if (pendingRecordsMetric.isEmpty()) {
LOG.warn(
"pendingRecords metric for {} could not be found. Either a legacy source or an idle source. Assuming no pending records.",
jobVertexID);
}
pendingRecordsMetric.forEach(m -> filteredMetrics.put(m, FlinkMetric.PENDING_RECORDS));
FlinkMetric.SOURCE_TASK_NUM_RECORDS_OUT
.findAny(allMetricNames)
.ifPresent(
m -> filteredMetrics.put(m, FlinkMetric.SOURCE_TASK_NUM_RECORDS_OUT));
}
for (FlinkMetric flinkMetric : requiredMetrics) {
Optional<String> flinkMetricName = flinkMetric.findAny(allMetricNames);
if (flinkMetricName.isPresent()) {
// Add actual Flink metric name to list
filteredMetrics.put(flinkMetricName.get(), flinkMetric);
} else {
throw new MetricNotFoundException(flinkMetric, jobVertexID);
}
}
return filteredMetrics;
}
@VisibleForTesting
@SneakyThrows
protected Collection<String> queryAggregatedMetricNames(
RestClusterClient<?> restClient, JobID jobID, JobVertexID jobVertexID) {
var parameters = new AggregatedSubtaskMetricsParameters();
var pathIt = parameters.getPathParameters().iterator();
((JobIDPathParameter) pathIt.next()).resolve(jobID);
((JobVertexIdPathParameter) pathIt.next()).resolve(jobVertexID);
return restClient
.sendRequest(
AggregatedSubtaskMetricsHeaders.getInstance(),
parameters,
EmptyRequestBody.getInstance())
.get()
.getMetrics()
.stream()
.map(AggregatedMetric::getId)
.collect(Collectors.toSet());
}
protected abstract Map<JobVertexID, Map<FlinkMetric, AggregatedMetric>>
queryAllAggregatedMetrics(
Context ctx,
Map<JobVertexID, Map<String, FlinkMetric>> filteredVertexMetricNames);
public JobDetailsInfo getJobDetailsInfo(
JobAutoScalerContext<KEY> context, Duration clientTimeout) throws Exception {
try (var restClient = context.getRestClusterClient()) {
return restClient
.getJobDetails(context.getJobID())
.get(clientTimeout.toSeconds(), TimeUnit.SECONDS);
}
}
public void cleanup(KEY jobKey) {
histories.remove(jobKey);
availableVertexMetricNames.remove(jobKey);
jobsWithGcMetrics.remove(jobKey);
}
@VisibleForTesting
protected void setClock(Clock clock) {
this.clock = Preconditions.checkNotNull(clock);
}
@VisibleForTesting
protected Map<KEY, Map<JobVertexID, Map<String, FlinkMetric>>> getAvailableVertexMetricNames() {
return availableVertexMetricNames;
}
@VisibleForTesting
protected Map<KEY, SortedMap<Instant, CollectedMetrics>> getHistories() {
return histories;
}
}