forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-8949] Add dedicated watermarks metric retrieval endpoint (apac…
…he#10238) Without this, watermarks for jobs with a parallelism of >= 160 cannot be displayed correctly and will result in a "/bad-request" error message. The reason is that the watermark for each subtask will be retrieved in a giant metrics request like the following (abbreviated): ``` http://localhost:8081/jobs/32f2205231d280ad105b011198dd9e5f/vertices/8b69eb2c39b9caf941896fdafa7ca05f/metrics?get=0.currentInputWatermark,1.currentInputWatermark,2.currentInputWatermark,3.currentInputWatermark,...,160.currentInputWatermark ``` We debated raising the maximum header length or lifting the header length restriction. Instead, we opted for a separate metrics endpoint which returns the watermarks for the entire job vertex: ``` /jobs/:jobid/vertices/:vertexid/watermarks ```
- Loading branch information
Showing
9 changed files
with
460 additions
and
51 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
99 changes: 99 additions & 0 deletions
99
...in/java/org/apache/flink/runtime/rest/handler/job/metrics/JobVertexWatermarksHandler.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,99 @@ | ||
/* | ||
* 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.executiongraph.AccessExecutionJobVertex; | ||
import org.apache.flink.runtime.executiongraph.AccessExecutionVertex; | ||
import org.apache.flink.runtime.jobgraph.JobVertexID; | ||
import org.apache.flink.runtime.metrics.MetricNames; | ||
import org.apache.flink.runtime.rest.handler.HandlerRequest; | ||
import org.apache.flink.runtime.rest.handler.RestHandlerException; | ||
import org.apache.flink.runtime.rest.handler.job.AbstractJobVertexHandler; | ||
import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphCache; | ||
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.JobVertexMessageParameters; | ||
import org.apache.flink.runtime.rest.messages.job.metrics.JobVertexWatermarksHeaders; | ||
import org.apache.flink.runtime.rest.messages.job.metrics.Metric; | ||
import org.apache.flink.runtime.rest.messages.job.metrics.MetricCollectionResponseBody; | ||
import org.apache.flink.runtime.webmonitor.RestfulGateway; | ||
import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; | ||
|
||
import java.util.ArrayList; | ||
import java.util.Collections; | ||
import java.util.List; | ||
import java.util.Map; | ||
import java.util.concurrent.Executor; | ||
|
||
|
||
/** | ||
* Handler that returns the watermarks given a {@link JobID} and {@link JobVertexID}. | ||
*/ | ||
public class JobVertexWatermarksHandler extends AbstractJobVertexHandler<MetricCollectionResponseBody, JobVertexMessageParameters> { | ||
|
||
private final MetricFetcher metricFetcher; | ||
|
||
public JobVertexWatermarksHandler( | ||
GatewayRetriever<? extends RestfulGateway> leaderRetriever, | ||
Time timeout, | ||
Map<String, String> responseHeaders, | ||
MetricFetcher metricFetcher, | ||
ExecutionGraphCache executionGraphCache, | ||
Executor executor) { | ||
super(leaderRetriever, | ||
timeout, | ||
responseHeaders, | ||
JobVertexWatermarksHeaders.INSTANCE, | ||
executionGraphCache, | ||
executor); | ||
this.metricFetcher = metricFetcher; | ||
} | ||
|
||
@Override | ||
protected MetricCollectionResponseBody handleRequest( | ||
HandlerRequest<EmptyRequestBody, JobVertexMessageParameters> request, | ||
AccessExecutionJobVertex jobVertex) throws RestHandlerException { | ||
|
||
String jobID = request.getPathParameter(JobIDPathParameter.class).toString(); | ||
String taskID = jobVertex.getJobVertexId().toString(); | ||
|
||
metricFetcher.update(); | ||
MetricStore.TaskMetricStore taskMetricStore = metricFetcher.getMetricStore().getTaskMetricStore(jobID, taskID); | ||
if (taskMetricStore == null) { | ||
return new MetricCollectionResponseBody(Collections.emptyList()); | ||
} | ||
|
||
AccessExecutionVertex[] taskVertices = jobVertex.getTaskVertices(); | ||
List<Metric> metrics = new ArrayList<>(taskVertices.length); | ||
|
||
for (AccessExecutionVertex taskVertex : taskVertices) { | ||
String id = taskVertex.getParallelSubtaskIndex() + "." + MetricNames.IO_CURRENT_INPUT_WATERMARK; | ||
String watermarkValue = taskMetricStore.getMetric(id); | ||
if (watermarkValue != null) { | ||
metrics.add(new Metric(id, watermarkValue)); | ||
} | ||
} | ||
|
||
return new MetricCollectionResponseBody(metrics); | ||
} | ||
} |
74 changes: 74 additions & 0 deletions
74
...n/java/org/apache/flink/runtime/rest/messages/job/metrics/JobVertexWatermarksHeaders.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,74 @@ | ||
/* | ||
* 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.HttpMethodWrapper; | ||
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.JobVertexMessageParameters; | ||
import org.apache.flink.runtime.rest.messages.MessageHeaders; | ||
|
||
import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; | ||
|
||
/** | ||
* {@link MessageHeaders} for retrieving watermarks. | ||
*/ | ||
public final class JobVertexWatermarksHeaders implements MessageHeaders<EmptyRequestBody, MetricCollectionResponseBody, JobVertexMessageParameters> { | ||
|
||
public static final JobVertexWatermarksHeaders INSTANCE = new JobVertexWatermarksHeaders(); | ||
|
||
private JobVertexWatermarksHeaders() { | ||
} | ||
|
||
@Override | ||
public HttpMethodWrapper getHttpMethod() { | ||
return HttpMethodWrapper.GET; | ||
} | ||
|
||
@Override | ||
public String getTargetRestEndpointURL() { | ||
return "/jobs/:" + JobIDPathParameter.KEY + "/vertices/:" + JobVertexIdPathParameter.KEY + "/watermarks"; | ||
} | ||
|
||
@Override | ||
public String getDescription() { | ||
return "Returns the watermarks for all subtasks of a task."; | ||
} | ||
|
||
@Override | ||
public Class<MetricCollectionResponseBody> getResponseClass() { | ||
return MetricCollectionResponseBody.class; | ||
} | ||
|
||
@Override | ||
public HttpResponseStatus getResponseStatusCode() { | ||
return HttpResponseStatus.OK; | ||
} | ||
|
||
@Override | ||
public Class<EmptyRequestBody> getRequestClass() { | ||
return EmptyRequestBody.class; | ||
} | ||
|
||
@Override | ||
public JobVertexMessageParameters getUnresolvedMessageParameters() { | ||
return new JobVertexMessageParameters(); | ||
} | ||
} |
Oops, something went wrong.