未验证 提交 49a89960 编写于 作者: G gyao 提交者: Till Rohrmann

[FLIP-7716][flip6] Migrate JobManagerMetricsHandler to new RestServerEndpoint

Migrate logic in
org.apache.flink.runtime.rest.handler.legacy.metrics.JobManagerMetricsHandler
to new handler and add new handler to DispatcherRestEndpoint.

[FLINK-7716][Javadoc] Deprecate method MetricStore#getJobManager().

There is a semantically equivalent method in MetricStore.

This closes #5083.
上级 cb85640b
......@@ -45,6 +45,7 @@ import org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointStatistic
import org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointStatsCache;
import org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointingStatisticsHandler;
import org.apache.flink.runtime.rest.handler.job.checkpoints.TaskCheckpointStatisticDetailsHandler;
import org.apache.flink.runtime.rest.handler.job.metrics.JobManagerMetricsHandler;
import org.apache.flink.runtime.rest.handler.job.metrics.JobVertexMetricsHandler;
import org.apache.flink.runtime.rest.handler.job.metrics.TaskManagerMetricsHandler;
import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphCache;
......@@ -69,6 +70,7 @@ import org.apache.flink.runtime.rest.messages.checkpoints.CheckpointStatisticDet
import org.apache.flink.runtime.rest.messages.checkpoints.CheckpointingStatisticsHeaders;
import org.apache.flink.runtime.rest.messages.checkpoints.TaskCheckpointStatisticsHeaders;
import org.apache.flink.runtime.rest.messages.job.JobDetailsHeaders;
import org.apache.flink.runtime.rest.messages.job.metrics.JobManagerMetricsHeaders;
import org.apache.flink.runtime.rest.messages.job.metrics.JobVertexMetricsHeaders;
import org.apache.flink.runtime.rest.messages.job.metrics.TaskManagerMetricsHeaders;
import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerDetailsHeaders;
......@@ -324,6 +326,13 @@ public class DispatcherRestEndpoint extends RestServerEndpoint {
responseHeaders,
metricFetcher);
final JobManagerMetricsHandler jobManagerMetricsHandler = new JobManagerMetricsHandler(
restAddressFuture,
leaderRetriever,
timeout,
responseHeaders,
metricFetcher);
final File tmpDir = restConfiguration.getTmpDir();
Optional<StaticFileServerHandler<DispatcherGateway>> optWebContent;
......@@ -361,6 +370,7 @@ public class DispatcherRestEndpoint extends RestServerEndpoint {
handlers.add(Tuple2.of(SubtasksTimesHeaders.getInstance(), subtasksTimesHandler));
handlers.add(Tuple2.of(JobVertexMetricsHeaders.getInstance(), jobVertexMetricsHandler));
handlers.add(Tuple2.of(TaskManagerMetricsHeaders.getInstance(), taskManagerMetricsHandler));
handlers.add(Tuple2.of(JobManagerMetricsHeaders.getInstance(), jobManagerMetricsHandler));
optWebContent.ifPresent(
webContent -> handlers.add(Tuple2.of(WebContentHandlerSpecification.getInstance(), webContent)));
......
/*
* 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.time.Time;
import org.apache.flink.runtime.dispatcher.DispatcherGateway;
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.job.metrics.JobManagerMetricsHeaders;
import org.apache.flink.runtime.rest.messages.job.metrics.JobManagerMetricsMessageParameters;
import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
import javax.annotation.Nullable;
import java.util.Map;
import java.util.concurrent.CompletableFuture;
/**
* Handler that returns JobManager metrics.
*/
public class JobManagerMetricsHandler extends AbstractMetricsHandler<JobManagerMetricsMessageParameters> {
public JobManagerMetricsHandler(
final CompletableFuture<String> localRestAddress,
final GatewayRetriever<DispatcherGateway> leaderRetriever,
final Time timeout,
final Map<String, String> headers,
final MetricFetcher metricFetcher) {
super(localRestAddress, leaderRetriever, timeout, headers, JobManagerMetricsHeaders.getInstance(),
metricFetcher);
}
@Nullable
@Override
protected MetricStore.ComponentMetricStore getComponentMetricStore(
final HandlerRequest<EmptyRequestBody, JobManagerMetricsMessageParameters> request,
final MetricStore metricStore) {
return metricStore.getJobManagerMetricStore();
}
}
......@@ -162,6 +162,10 @@ public class MetricStore {
return unmodifiableMap(taskManagers);
}
/**
* @deprecated Use semantically equivalent {@link #getJobManagerMetricStore()}.
*/
@Deprecated
public synchronized ComponentMetricStore getJobManager() {
return ComponentMetricStore.unmodifiable(jobManager);
}
......
/*
* 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.messages.job.metrics;
import org.apache.flink.runtime.rest.messages.MessageHeaders;
/**
* {@link MessageHeaders} for
* {@link org.apache.flink.runtime.rest.handler.job.metrics.JobManagerMetricsHandler}.
*/
public final class JobManagerMetricsHeaders extends
AbstractMetricsHeaders<JobManagerMetricsMessageParameters> {
private static final JobManagerMetricsHeaders INSTANCE = new JobManagerMetricsHeaders();
private JobManagerMetricsHeaders() {
}
@Override
public JobManagerMetricsMessageParameters getUnresolvedMessageParameters() {
return new JobManagerMetricsMessageParameters();
}
@Override
public String getTargetRestEndpointURL() {
return "/jobmanager/metrics";
}
public static JobManagerMetricsHeaders getInstance() {
return INSTANCE;
}
}
/*
* 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.messages.job.metrics;
import org.apache.flink.runtime.rest.messages.MessageParameters;
import org.apache.flink.runtime.rest.messages.MessagePathParameter;
import org.apache.flink.runtime.rest.messages.MessageQueryParameter;
import java.util.Collection;
import java.util.Collections;
/**
* {@link MessageParameters} for
* {@link org.apache.flink.runtime.rest.handler.job.metrics.JobManagerMetricsHandler}.
*/
public class JobManagerMetricsMessageParameters extends MessageParameters {
private final MetricsFilterParameter metricsFilterParameter = new MetricsFilterParameter();
@Override
public Collection<MessagePathParameter<?>> getPathParameters() {
return Collections.emptyList();
}
@Override
public Collection<MessageQueryParameter<?>> getQueryParameters() {
return Collections.singleton(metricsFilterParameter);
}
}
/*
* 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.runtime.metrics.dump.QueryScopeInfo;
import java.util.Collections;
import java.util.Map;
/**
* Tests for {@link JobManagerMetricsHandler}.
*/
public class JobManagerMetricsHandlerTest extends MetricsHandlerTestBase<JobManagerMetricsHandler> {
@Override
JobManagerMetricsHandler getMetricsHandler() {
return new JobManagerMetricsHandler(
TEST_REST_ADDRESS,
leaderRetriever,
TIMEOUT,
TEST_HEADERS,
mockMetricFetcher);
}
@Override
QueryScopeInfo getQueryScopeInfo() {
return new QueryScopeInfo.JobManagerQueryScopeInfo();
}
@Override
Map<String, String> getPathParameters() {
return Collections.emptyMap();
}
}
/*
* 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.messages.job.metrics;
import org.junit.Test;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.instanceOf;
import static org.junit.Assert.assertThat;
/**
* Tests for {@link JobManagerMetricsHeaders}.
*/
public class JobManagerMetricsHeadersTest {
private final JobManagerMetricsHeaders jobManagerMetricsHeaders =
JobManagerMetricsHeaders.getInstance();
@Test
public void testUrl() {
assertThat(jobManagerMetricsHeaders.getTargetRestEndpointURL(), equalTo("/jobmanager/metrics"));
}
@Test
public void testMessageParameters() {
assertThat(jobManagerMetricsHeaders.getUnresolvedMessageParameters(), instanceOf
(JobManagerMetricsMessageParameters.class));
}
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册