Github user tillrohrmann commented on a diff in the pull request:
https://github.com/apache/flink/pull/4893#discussion_r149305450
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobVertexBackPressureHandler.java
---
@@ -0,0 +1,134 @@
+/*
+ * 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;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.WebOptions;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.RestHandlerException;
+import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphCache;
+import
org.apache.flink.runtime.rest.handler.legacy.backpressure.BackPressureStatsTracker;
+import
org.apache.flink.runtime.rest.handler.legacy.backpressure.OperatorBackPressureStats;
+import
org.apache.flink.runtime.rest.handler.legacy.backpressure.StackTraceSampleCoordinator;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.JobVertexBackPressureInfo;
+import org.apache.flink.runtime.rest.messages.JobVertexIdPathParameter;
+import org.apache.flink.runtime.rest.messages.JobVertexMessageParameters;
+import org.apache.flink.runtime.rest.messages.MessageHeaders;
+import org.apache.flink.runtime.webmonitor.RestfulGateway;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * Request handler for the job vertex back pressure.
+ */
+public class JobVertexBackPressureHandler extends
AbstractExecutionGraphHandler<JobVertexBackPressureInfo,
JobVertexMessageParameters> {
+ /** Back pressure stats tracker. */
+ private final BackPressureStatsTracker backPressureStatsTracker;
+
+ /** Time after which stats are considered outdated. */
+ private final int refreshInterval;
+
+ public JobVertexBackPressureHandler(
+ CompletableFuture<String> localRestAddress,
+ GatewayRetriever<? extends RestfulGateway>
leaderRetriever,
+ Time timeout,
+ MessageHeaders<EmptyRequestBody,
JobVertexBackPressureInfo, JobVertexMessageParameters> messageHeaders,
+ ExecutionGraphCache executionGraphCache,
+ Executor executor,
+ Configuration clusterConfiguration) {
+ super(localRestAddress, leaderRetriever, timeout,
messageHeaders, executionGraphCache, executor);
+
+ // Back pressure stats tracker config
+ this.refreshInterval =
clusterConfiguration.getInteger(WebOptions.BACKPRESSURE_REFRESH_INTERVAL);
+ this.backPressureStatsTracker = new BackPressureStatsTracker(
+ new StackTraceSampleCoordinator(executor, 60000),
+
clusterConfiguration.getInteger(WebOptions.BACKPRESSURE_CLEANUP_INTERVAL),
+
clusterConfiguration.getInteger(WebOptions.BACKPRESSURE_NUM_SAMPLES),
+
Time.milliseconds(clusterConfiguration.getInteger(WebOptions.BACKPRESSURE_DELAY)));
+ }
+
+ @Override
+ protected JobVertexBackPressureInfo
handleRequest(HandlerRequest<EmptyRequestBody, JobVertexMessageParameters>
request, AccessExecutionGraph executionGraph) throws RestHandlerException {
+ JobVertexID jobVertexID =
request.getPathParameter(JobVertexIdPathParameter.class);
+ AccessExecutionJobVertex accessJobVertex =
executionGraph.getJobVertex(jobVertexID);
+ if (accessJobVertex instanceof ArchivedExecutionJobVertex) {
+ return new JobVertexBackPressureInfo(null, null, null,
null);
--- End diff --
I would not allow to pass in `null` values to the
`JobVertexBackPressureInfo`. Instead I would create a singleton
`EmptyJbVertexBackPressureInfo` which initializes the fields accordingly.
---