[ https://issues.apache.org/jira/browse/FLINK-8370?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16437393#comment-16437393 ]
ASF GitHub Bot commented on FLINK-8370: --------------------------------------- Github user GJL commented on a diff in the pull request: https://github.com/apache/flink/pull/5805#discussion_r181411022 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/metrics/AggregatingSubtasksMetricsHandler.java --- @@ -0,0 +1,129 @@ +/* + * 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.job.metrics; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.rest.handler.HandlerRequest; +import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcher; +import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricStore; +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.metrics.AggregatedSubtaskMetricsHeaders; +import org.apache.flink.runtime.rest.messages.job.metrics.AggregatedSubtaskMetricsParameters; +import org.apache.flink.runtime.rest.messages.job.metrics.SubtasksFilterQueryParameter; +import org.apache.flink.runtime.webmonitor.RestfulGateway; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; +import org.apache.flink.util.UnionIterator; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; + +/** + * Request handler that returns, aggregated across subtasks, a list of all available metrics or the values + * for a set of metrics. + * + * <p>Specific subtasks can be selected for aggregation by specifying a comma-separated list of integer ranges. + * {@code /metrics?get=X,Y&subtasks=0-2,4-5} + */ +public class AggregatingSubtasksMetricsHandler extends AbstractAggregatingMetricsHandler<AggregatedSubtaskMetricsParameters> { + + public AggregatingSubtasksMetricsHandler( + CompletableFuture<String> localRestAddress, + GatewayRetriever<? extends RestfulGateway> leaderRetriever, + Time timeout, + Map<String, String> responseHeaders, + Executor executor, + MetricFetcher<?> fetcher) { + super(localRestAddress, leaderRetriever, timeout, responseHeaders, AggregatedSubtaskMetricsHeaders.getInstance(), executor, fetcher); + } + + @Override + Collection<? extends MetricStore.ComponentMetricStore> getStores(MetricStore store, HandlerRequest<EmptyRequestBody, AggregatedSubtaskMetricsParameters> request) { + JobID jobID = request.getPathParameter(JobIDPathParameter.class); + JobVertexID taskID = request.getPathParameter(JobVertexIdPathParameter.class); + + Collection<String> subtaskRanges = request.getQueryParameter(SubtasksFilterQueryParameter.class); + if (subtaskRanges.isEmpty()) { + return store.getTaskMetricStore(jobID.toString(), taskID.toString()).getAllSubtaskMetricStores(); + } else { + Iterable<Integer> subtasks = getIntegerRangeFromString(subtaskRanges); + Collection<MetricStore.ComponentMetricStore> subtaskStores = new ArrayList<>(8); + for (int subtask : subtasks) { + subtaskStores.add(store.getSubtaskMetricStore(jobID.toString(), taskID.toString(), subtask)); + } + return subtaskStores; + } + } + + private Iterable<Integer> getIntegerRangeFromString(Collection<String> ranges) { + UnionIterator<Integer> iterators = new UnionIterator<>(); + + for (String rawRange : ranges) { + try { + Iterator<Integer> rangeIterator; + String range = rawRange.trim(); + int dashIdx = range.indexOf('-'); + if (dashIdx == -1) { + // only one value in range: + rangeIterator = Collections.singleton(Integer.valueOf(range)).iterator(); + } else { + // evaluate range + final int start = Integer.valueOf(range.substring(0, dashIdx)); + final int end = Integer.valueOf(range.substring(dashIdx + 1, range.length())); + rangeIterator = new Iterator<Integer>() { + int i = start; + + @Override + public boolean hasNext() { + return i <= end; + } + + @Override + public Integer next() { + if (hasNext()) { + return i++; + } else { + throw new NoSuchElementException(); + } + } + + @Override + public void remove() { + throw new UnsupportedOperationException("Remove not supported"); --- End diff -- That's already the default implementation for that interface. > Port AbstractAggregatingMetricsHandler to RestServerEndpoint > ------------------------------------------------------------ > > Key: FLINK-8370 > URL: https://issues.apache.org/jira/browse/FLINK-8370 > Project: Flink > Issue Type: Sub-task > Components: REST > Affects Versions: 1.5.0 > Reporter: Gary Yao > Assignee: Chesnay Schepler > Priority: Blocker > Labels: flip-6 > Fix For: 1.5.0 > > > Port subclasses of > {{org.apache.flink.runtime.rest.handler.legacy.metrics.AbstractAggregatingMetricsHandler}} > to new FLIP-6 {{RestServerEndpoint}}. > The following handlers need to be migrated: > * {{AggregatingJobsMetricsHandler}} > * {{AggregatingSubtasksMetricsHandler}} > * {{AggregatingTaskManagersMetricsHandler}} > New handlers should then be registered in {{WebMonitorEndpoint}}. -- This message was sent by Atlassian JIRA (v7.6.3#76005)