Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-7368][metrics] Make MetricStore ThreadSafe class #4840

Closed
wants to merge 2 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Expand Up @@ -130,71 +130,68 @@ private String writeTaskManagersJson(Collection<Instance> instances, Map<String,
// only send metrics when only one task manager requests them.
if (pathParams.containsKey(TASK_MANAGER_ID_KEY)) {
fetcher.update();
final MetricStore metricStore = fetcher.getMetricStore();

synchronized (metricStore) {
MetricStore.TaskManagerMetricStore metrics = metricStore.getTaskManagerMetricStore(instance.getId().toString());
if (metrics != null) {
gen.writeObjectFieldStart("metrics");
long heapUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Heap.Used", "0"));
long heapCommitted = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Heap.Committed", "0"));
long heapTotal = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Heap.Max", "0"));

gen.writeNumberField("heapCommitted", heapCommitted);
gen.writeNumberField("heapUsed", heapUsed);
gen.writeNumberField("heapMax", heapTotal);

long nonHeapUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.NonHeap.Used", "0"));
long nonHeapCommitted = Long.valueOf(metrics.getMetric("Status.JVM.Memory.NonHeap.Committed", "0"));
long nonHeapTotal = Long.valueOf(metrics.getMetric("Status.JVM.Memory.NonHeap.Max", "0"));

gen.writeNumberField("nonHeapCommitted", nonHeapCommitted);
gen.writeNumberField("nonHeapUsed", nonHeapUsed);
gen.writeNumberField("nonHeapMax", nonHeapTotal);

gen.writeNumberField("totalCommitted", heapCommitted + nonHeapCommitted);
gen.writeNumberField("totalUsed", heapUsed + nonHeapUsed);
gen.writeNumberField("totalMax", heapTotal + nonHeapTotal);

long directCount = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Direct.Count", "0"));
long directUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Direct.MemoryUsed", "0"));
long directMax = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Direct.TotalCapacity", "0"));

gen.writeNumberField("directCount", directCount);
gen.writeNumberField("directUsed", directUsed);
gen.writeNumberField("directMax", directMax);

long mappedCount = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Mapped.Count", "0"));
long mappedUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Mapped.MemoryUsed", "0"));
long mappedMax = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Mapped.TotalCapacity", "0"));

gen.writeNumberField("mappedCount", mappedCount);
gen.writeNumberField("mappedUsed", mappedUsed);
gen.writeNumberField("mappedMax", mappedMax);

long memorySegmentsAvailable = Long.valueOf(metrics.getMetric("Status.Network.AvailableMemorySegments", "0"));
long memorySegmentsTotal = Long.valueOf(metrics.getMetric("Status.Network.TotalMemorySegments", "0"));

gen.writeNumberField("memorySegmentsAvailable", memorySegmentsAvailable);
gen.writeNumberField("memorySegmentsTotal", memorySegmentsTotal);

gen.writeArrayFieldStart("garbageCollectors");

for (String gcName : metrics.garbageCollectorNames) {
String count = metrics.getMetric("Status.JVM.GarbageCollector." + gcName + ".Count", null);
String time = metrics.getMetric("Status.JVM.GarbageCollector." + gcName + ".Time", null);
if (count != null && time != null) {
gen.writeStartObject();
gen.writeStringField("name", gcName);
gen.writeNumberField("count", Long.valueOf(count));
gen.writeNumberField("time", Long.valueOf(time));
gen.writeEndObject();
}
}

gen.writeEndArray();
gen.writeEndObject();
MetricStore.TaskManagerMetricStore metrics = fetcher.getMetricStore().getTaskManagerMetricStore(instance.getId().toString());
if (metrics != null) {
gen.writeObjectFieldStart("metrics");
long heapUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Heap.Used", "0"));
long heapCommitted = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Heap.Committed", "0"));
long heapTotal = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Heap.Max", "0"));

gen.writeNumberField("heapCommitted", heapCommitted);
gen.writeNumberField("heapUsed", heapUsed);
gen.writeNumberField("heapMax", heapTotal);

long nonHeapUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.NonHeap.Used", "0"));
long nonHeapCommitted = Long.valueOf(metrics.getMetric("Status.JVM.Memory.NonHeap.Committed", "0"));
long nonHeapTotal = Long.valueOf(metrics.getMetric("Status.JVM.Memory.NonHeap.Max", "0"));

gen.writeNumberField("nonHeapCommitted", nonHeapCommitted);
gen.writeNumberField("nonHeapUsed", nonHeapUsed);
gen.writeNumberField("nonHeapMax", nonHeapTotal);

gen.writeNumberField("totalCommitted", heapCommitted + nonHeapCommitted);
gen.writeNumberField("totalUsed", heapUsed + nonHeapUsed);
gen.writeNumberField("totalMax", heapTotal + nonHeapTotal);

long directCount = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Direct.Count", "0"));
long directUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Direct.MemoryUsed", "0"));
long directMax = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Direct.TotalCapacity", "0"));

gen.writeNumberField("directCount", directCount);
gen.writeNumberField("directUsed", directUsed);
gen.writeNumberField("directMax", directMax);

long mappedCount = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Mapped.Count", "0"));
long mappedUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Mapped.MemoryUsed", "0"));
long mappedMax = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Mapped.TotalCapacity", "0"));

gen.writeNumberField("mappedCount", mappedCount);
gen.writeNumberField("mappedUsed", mappedUsed);
gen.writeNumberField("mappedMax", mappedMax);

long memorySegmentsAvailable = Long.valueOf(metrics.getMetric("Status.Network.AvailableMemorySegments", "0"));
long memorySegmentsTotal = Long.valueOf(metrics.getMetric("Status.Network.TotalMemorySegments", "0"));

gen.writeNumberField("memorySegmentsAvailable", memorySegmentsAvailable);
gen.writeNumberField("memorySegmentsTotal", memorySegmentsTotal);

gen.writeArrayFieldStart("garbageCollectors");

for (String gcName : metrics.garbageCollectorNames) {
String count = metrics.getMetric("Status.JVM.GarbageCollector." + gcName + ".Count", null);
String time = metrics.getMetric("Status.JVM.GarbageCollector." + gcName + ".Time", null);
if (count != null && time != null) {
gen.writeStartObject();
gen.writeStringField("name", gcName);
gen.writeNumberField("count", Long.valueOf(count));
gen.writeNumberField("time", Long.valueOf(time));
gen.writeEndObject();
}
}

gen.writeEndArray();
gen.writeEndObject();
}
}

Expand Down
Expand Up @@ -87,54 +87,48 @@ private String getMetricsValues(Map<String, String> pathParams, String requested
*/
return "";
}
MetricStore metricStore = fetcher.getMetricStore();
synchronized (metricStore) {
Map<String, String> metrics = getMapFor(pathParams, metricStore);
if (metrics == null) {
return "";
}
String[] requestedMetrics = requestedMetricsList.split(",");

StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);

gen.writeStartArray();
for (String requestedMetric : requestedMetrics) {
Object metricValue = metrics.get(requestedMetric);
if (metricValue != null) {
gen.writeStartObject();
gen.writeStringField("id", requestedMetric);
gen.writeStringField("value", metricValue.toString());
gen.writeEndObject();
}
}
gen.writeEndArray();

gen.close();
return writer.toString();
Map<String, String> metrics = getMapFor(pathParams, fetcher.getMetricStore());
if (metrics == null) {
return "";
}
}
String[] requestedMetrics = requestedMetricsList.split(",");

private String getAvailableMetricsList(Map<String, String> pathParams) throws IOException {
MetricStore metricStore = fetcher.getMetricStore();
synchronized (metricStore) {
Map<String, String> metrics = getMapFor(pathParams, metricStore);
if (metrics == null) {
return "";
}
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);

gen.writeStartArray();
for (String m : metrics.keySet()) {
gen.writeStartArray();
for (String requestedMetric : requestedMetrics) {
Object metricValue = metrics.get(requestedMetric);
if (metricValue != null) {
gen.writeStartObject();
gen.writeStringField("id", m);
gen.writeStringField("id", requestedMetric);
gen.writeStringField("value", metricValue.toString());
gen.writeEndObject();
}
gen.writeEndArray();
}
gen.writeEndArray();

gen.close();
return writer.toString();
}

gen.close();
return writer.toString();
private String getAvailableMetricsList(Map<String, String> pathParams) throws IOException {
Map<String, String> metrics = getMapFor(pathParams, fetcher.getMetricStore());
if (metrics == null) {
return "";
}
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);

gen.writeStartArray();
for (String m : metrics.keySet()) {
gen.writeStartObject();
gen.writeStringField("id", m);
gen.writeEndObject();
}
gen.writeEndArray();

gen.close();
return writer.toString();
}
}
Expand Up @@ -47,7 +47,7 @@ public String[] getPaths() {

@Override
protected Map<String, String> getMapFor(Map<String, String> pathParams, MetricStore metrics) {
MetricStore.JobManagerMetricStore jobManager = metrics.getJobManagerMetricStore();
MetricStore.ComponentMetricStore jobManager = metrics.getJobManagerMetricStore();
if (jobManager == null) {
return null;
} else {
Expand Down
Expand Up @@ -47,7 +47,7 @@ public String[] getPaths() {

@Override
protected Map<String, String> getMapFor(Map<String, String> pathParams, MetricStore metrics) {
MetricStore.JobMetricStore job = metrics.getJobMetricStore(pathParams.get(PARAMETER_JOB_ID));
MetricStore.ComponentMetricStore job = metrics.getJobMetricStore(pathParams.get(PARAMETER_JOB_ID));
return job != null
? job.metrics
: null;
Expand Down
Expand Up @@ -47,7 +47,7 @@ public String[] getPaths() {

@Override
protected Map<String, String> getMapFor(Map<String, String> pathParams, MetricStore metrics) {
MetricStore.TaskMetricStore task = metrics.getTaskMetricStore(
MetricStore.ComponentMetricStore task = metrics.getTaskMetricStore(
pathParams.get(JobMetricsHandler.PARAMETER_JOB_ID),
pathParams.get(PARAMETER_VERTEX_ID));
return task != null
Expand Down
Expand Up @@ -23,7 +23,6 @@
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.MetricDump;
import org.apache.flink.runtime.metrics.dump.MetricDumpSerialization;
import org.apache.flink.runtime.metrics.dump.MetricQueryService;
import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
Expand Down Expand Up @@ -112,16 +111,14 @@ private void fetchMetrics() {
if (throwable != null) {
LOG.debug("Fetching of JobDetails failed.", throwable);
} else {
ArrayList<String> toRetain = new ArrayList<>();
ArrayList<String> activeJobs = new ArrayList<>();
for (JobDetails job : jobDetails.getRunning()) {
toRetain.add(job.getJobId().toString());
activeJobs.add(job.getJobId().toString());
}
for (JobDetails job : jobDetails.getFinished()) {
toRetain.add(job.getJobId().toString());
}
synchronized (metrics) {
metrics.jobs.keySet().retainAll(toRetain);
activeJobs.add(job.getJobId().toString());
}
metrics.retainJobs(activeJobs);
}
},
executor);
Expand Down Expand Up @@ -154,9 +151,7 @@ private void fetchMetrics() {
return taskManagerInstance.getId().toString();
}).collect(Collectors.toList());

synchronized (metrics) {
metrics.taskManagers.keySet().retainAll(activeTaskManagers);
}
metrics.retainTaskManagers(activeTaskManagers);
}
},
executor);
Expand Down Expand Up @@ -198,12 +193,7 @@ private void queryMetrics(final MetricQueryServiceGateway queryServiceGateway) {
if (t != null) {
LOG.debug("Fetching metrics failed.", t);
} else {
List<MetricDump> dumpedMetrics = deserializer.deserialize(result);
synchronized (metrics) {
for (MetricDump metric : dumpedMetrics) {
metrics.add(metric);
}
}
metrics.addAll(deserializer.deserialize(result));
}
},
executor);
Expand Down