forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 2
/
MetricFetcher.java
201 lines (176 loc) · 7.37 KB
/
MetricFetcher.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
/*
* 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.runtime.rest.handler.legacy.metrics;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.runtime.instance.Instance;
import org.apache.flink.runtime.jobmaster.JobManagerGateway;
import org.apache.flink.runtime.messages.webmonitor.JobDetails;
import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails;
import org.apache.flink.runtime.metrics.dump.MetricDumpSerialization;
import org.apache.flink.runtime.metrics.dump.MetricQueryService;
import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceGateway;
import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
import org.apache.flink.util.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Executor;
import java.util.stream.Collectors;
import static org.apache.flink.runtime.metrics.dump.MetricDumpSerialization.MetricDumpDeserializer;
/**
* The MetricFetcher can be used to fetch metrics from the JobManager and all registered TaskManagers.
*
* <p>Metrics will only be fetched when {@link MetricFetcher#update()} is called, provided that a sufficient time since
* the last call has passed.
*/
public class MetricFetcher {
private static final Logger LOG = LoggerFactory.getLogger(MetricFetcher.class);
private final GatewayRetriever<JobManagerGateway> retriever;
private final MetricQueryServiceRetriever queryServiceRetriever;
private final Executor executor;
private final Time timeout;
private final MetricStore metrics = new MetricStore();
private final MetricDumpDeserializer deserializer = new MetricDumpDeserializer();
private long lastUpdateTime;
public MetricFetcher(
GatewayRetriever<JobManagerGateway> retriever,
MetricQueryServiceRetriever queryServiceRetriever,
Executor executor,
Time timeout) {
this.retriever = Preconditions.checkNotNull(retriever);
this.queryServiceRetriever = Preconditions.checkNotNull(queryServiceRetriever);
this.executor = Preconditions.checkNotNull(executor);
this.timeout = Preconditions.checkNotNull(timeout);
}
/**
* Returns the MetricStore containing all stored metrics.
*
* @return MetricStore containing all stored metrics;
*/
public MetricStore getMetricStore() {
return metrics;
}
/**
* This method can be used to signal this MetricFetcher that the metrics are still in use and should be updated.
*/
public void update() {
synchronized (this) {
long currentTime = System.currentTimeMillis();
if (currentTime - lastUpdateTime > 10000) { // 10 seconds have passed since the last update
lastUpdateTime = currentTime;
fetchMetrics();
}
}
}
private void fetchMetrics() {
try {
Optional<JobManagerGateway> optJobManagerGateway = retriever.getNow();
if (optJobManagerGateway.isPresent()) {
final JobManagerGateway jobManagerGateway = optJobManagerGateway.get();
/**
* Remove all metrics that belong to a job that is not running and no longer archived.
*/
CompletableFuture<MultipleJobsDetails> jobDetailsFuture = jobManagerGateway.requestJobDetails(true, true, timeout);
jobDetailsFuture.whenCompleteAsync(
(MultipleJobsDetails jobDetails, Throwable throwable) -> {
if (throwable != null) {
LOG.debug("Fetching of JobDetails failed.", throwable);
} else {
ArrayList<String> activeJobs = new ArrayList<>();
for (JobDetails job : jobDetails.getRunning()) {
activeJobs.add(job.getJobId().toString());
}
for (JobDetails job : jobDetails.getFinished()) {
activeJobs.add(job.getJobId().toString());
}
metrics.retainJobs(activeJobs);
}
},
executor);
String jobManagerPath = jobManagerGateway.getAddress();
String jmQueryServicePath = jobManagerPath.substring(0, jobManagerPath.lastIndexOf('/') + 1) + MetricQueryService.METRIC_QUERY_SERVICE_NAME;
retrieveAndQueryMetrics(jmQueryServicePath);
/**
* We first request the list of all registered task managers from the job manager, and then
* request the respective metric dump from each task manager.
*
* <p>All stored metrics that do not belong to a registered task manager will be removed.
*/
CompletableFuture<Collection<Instance>> taskManagersFuture = jobManagerGateway.requestTaskManagerInstances(timeout);
taskManagersFuture.whenCompleteAsync(
(Collection<Instance> taskManagers, Throwable throwable) -> {
if (throwable != null) {
LOG.debug("Fetching list of registered TaskManagers failed.", throwable);
} else {
List<String> activeTaskManagers = taskManagers.stream().map(
taskManagerInstance -> {
final String taskManagerAddress = taskManagerInstance.getTaskManagerGateway().getAddress();
final String tmQueryServicePath = taskManagerAddress.substring(0, taskManagerAddress.lastIndexOf('/') + 1) + MetricQueryService.METRIC_QUERY_SERVICE_NAME + "_" + taskManagerInstance.getTaskManagerID().getResourceIdString();
retrieveAndQueryMetrics(tmQueryServicePath);
return taskManagerInstance.getId().toString();
}).collect(Collectors.toList());
metrics.retainTaskManagers(activeTaskManagers);
}
},
executor);
}
} catch (Exception e) {
LOG.warn("Exception while fetching metrics.", e);
}
}
/**
* Retrieves and queries the specified QueryServiceGateway.
*
* @param queryServicePath specifying the QueryServiceGateway
*/
private void retrieveAndQueryMetrics(String queryServicePath) {
final CompletableFuture<MetricQueryServiceGateway> queryServiceGatewayFuture = queryServiceRetriever.retrieveService(queryServicePath);
queryServiceGatewayFuture.whenCompleteAsync(
(MetricQueryServiceGateway queryServiceGateway, Throwable t) -> {
if (t != null) {
LOG.debug("Could not retrieve QueryServiceGateway.", t);
} else {
queryMetrics(queryServiceGateway);
}
},
executor);
}
/**
* Query the metrics from the given QueryServiceGateway.
*
* @param queryServiceGateway to query for metrics
*/
private void queryMetrics(final MetricQueryServiceGateway queryServiceGateway) {
queryServiceGateway
.queryMetrics(timeout)
.whenCompleteAsync(
(MetricDumpSerialization.MetricSerializationResult result, Throwable t) -> {
if (t != null) {
LOG.debug("Fetching metrics failed.", t);
} else {
metrics.addAll(deserializer.deserialize(result));
}
},
executor);
}
}