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-7855] [flip6] Port JobVertexAccumulatorsHandler to REST endpoint
- Loading branch information
1 parent
142ff78
commit 1348839
Showing
5 changed files
with
330 additions
and
0 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
78 changes: 78 additions & 0 deletions
78
...src/main/java/org/apache/flink/runtime/rest/handler/job/JobVertexAccumulatorsHandler.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,78 @@ | ||
/* | ||
* 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.runtime.accumulators.StringifiedAccumulatorResult; | ||
import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; | ||
import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; | ||
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.messages.EmptyRequestBody; | ||
import org.apache.flink.runtime.rest.messages.JobMessageParameters; | ||
import org.apache.flink.runtime.rest.messages.JobVertexAccumulatorsInfo; | ||
import org.apache.flink.runtime.rest.messages.JobVertexIdPathParameter; | ||
import org.apache.flink.runtime.rest.messages.MessageHeaders; | ||
import org.apache.flink.runtime.webmonitor.RestfulGateway; | ||
import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; | ||
|
||
import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; | ||
|
||
import java.util.ArrayList; | ||
import java.util.List; | ||
import java.util.concurrent.CompletableFuture; | ||
import java.util.concurrent.Executor; | ||
|
||
/** | ||
* Request handler for the job vertex accumulators. | ||
*/ | ||
public class JobVertexAccumulatorsHandler extends AbstractExecutionGraphHandler<JobVertexAccumulatorsInfo, JobMessageParameters> { | ||
|
||
public JobVertexAccumulatorsHandler( | ||
CompletableFuture<String> localRestAddress, | ||
GatewayRetriever<? extends RestfulGateway> leaderRetriever, | ||
Time timeout, | ||
MessageHeaders<EmptyRequestBody, JobVertexAccumulatorsInfo, JobMessageParameters> messageHeaders, | ||
ExecutionGraphCache executionGraphCache, | ||
Executor executor) { | ||
super(localRestAddress, leaderRetriever, timeout, messageHeaders, executionGraphCache, executor); | ||
} | ||
|
||
@Override | ||
protected JobVertexAccumulatorsInfo handleRequest(HandlerRequest<EmptyRequestBody, JobMessageParameters> request, AccessExecutionGraph executionGraph) throws RestHandlerException { | ||
JobVertexID jobVertexID = request.getPathParameter(JobVertexIdPathParameter.class); | ||
AccessExecutionJobVertex jobVertex = executionGraph.getJobVertex(jobVertexID); | ||
if (null != jobVertex) { | ||
StringifiedAccumulatorResult[] accs = jobVertex.getAggregatedUserAccumulatorsStringified(); | ||
List<JobVertexAccumulatorsInfo.UserAccumulator> userAccumulatorList = new ArrayList<>(); | ||
for (StringifiedAccumulatorResult acc : accs) { | ||
userAccumulatorList.add(new JobVertexAccumulatorsInfo.UserAccumulator( | ||
acc.getName(), | ||
acc.getType(), | ||
acc.getValue())); | ||
} | ||
|
||
return new JobVertexAccumulatorsInfo(jobVertex.getJobVertexId().toString(), userAccumulatorList); | ||
} else { | ||
throw new RestHandlerException("There is no accumulator for vertex " + jobVertexID + '.', HttpResponseStatus.NOT_FOUND); | ||
} | ||
} | ||
} |
70 changes: 70 additions & 0 deletions
70
...me/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexAccumulatorsHeaders.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,70 @@ | ||
/* | ||
* 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; | ||
|
||
import org.apache.flink.runtime.rest.HttpMethodWrapper; | ||
import org.apache.flink.runtime.rest.handler.job.JobVertexAccumulatorsHandler; | ||
|
||
import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; | ||
|
||
/** | ||
* Message headers for the {@link JobVertexAccumulatorsHandler}. | ||
*/ | ||
public class JobVertexAccumulatorsHeaders implements MessageHeaders<EmptyRequestBody, JobVertexAccumulatorsInfo, JobMessageParameters> { | ||
|
||
private static final JobVertexAccumulatorsHeaders INSTANCE = new JobVertexAccumulatorsHeaders(); | ||
|
||
public static final String URL = "/jobs/:jobid/vertices/:vertexid/accumulators"; | ||
|
||
private JobVertexAccumulatorsHeaders() {} | ||
|
||
@Override | ||
public Class<EmptyRequestBody> getRequestClass() { | ||
return EmptyRequestBody.class; | ||
} | ||
|
||
@Override | ||
public Class<JobVertexAccumulatorsInfo> getResponseClass() { | ||
return JobVertexAccumulatorsInfo.class; | ||
} | ||
|
||
@Override | ||
public HttpResponseStatus getResponseStatusCode() { | ||
return HttpResponseStatus.OK; | ||
} | ||
|
||
@Override | ||
public JobMessageParameters getUnresolvedMessageParameters() { | ||
return new JobMessageParameters(); | ||
} | ||
|
||
@Override | ||
public HttpMethodWrapper getHttpMethod() { | ||
return HttpMethodWrapper.GET; | ||
} | ||
|
||
@Override | ||
public String getTargetRestEndpointURL() { | ||
return URL; | ||
} | ||
|
||
public static JobVertexAccumulatorsHeaders getInstance() { | ||
return INSTANCE; | ||
} | ||
} |
120 changes: 120 additions & 0 deletions
120
...ntime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexAccumulatorsInfo.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,120 @@ | ||
/* | ||
* 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; | ||
|
||
import org.apache.flink.runtime.rest.handler.job.JobVertexAccumulatorsHandler; | ||
|
||
import com.fasterxml.jackson.annotation.JsonCreator; | ||
import com.fasterxml.jackson.annotation.JsonProperty; | ||
|
||
import java.util.List; | ||
import java.util.Objects; | ||
|
||
/** | ||
* Response type of the {@link JobVertexAccumulatorsHandler}. | ||
*/ | ||
public class JobVertexAccumulatorsInfo implements ResponseBody { | ||
|
||
public static final String FIELD_NAME_ID = "id"; | ||
public static final String FIELD_NAME_USER_ACCUMULATORS = "user-accumulators"; | ||
|
||
@JsonProperty(FIELD_NAME_ID) | ||
private String id; | ||
|
||
@JsonProperty(FIELD_NAME_USER_ACCUMULATORS) | ||
private List<UserAccumulator> userAccumulatorList; | ||
|
||
@JsonCreator | ||
public JobVertexAccumulatorsInfo( | ||
@JsonProperty(FIELD_NAME_ID) String id, | ||
@JsonProperty(FIELD_NAME_USER_ACCUMULATORS) List<UserAccumulator> userAccumulatorList) { | ||
this.id = id; | ||
this.userAccumulatorList = userAccumulatorList; | ||
} | ||
|
||
@Override | ||
public boolean equals(Object o) { | ||
if (this == o) { | ||
return true; | ||
} | ||
if (o == null || getClass() != o.getClass()) { | ||
return false; | ||
} | ||
JobVertexAccumulatorsInfo that = (JobVertexAccumulatorsInfo) o; | ||
return Objects.equals(id, that.id) && | ||
Objects.equals(userAccumulatorList, that.userAccumulatorList); | ||
} | ||
|
||
@Override | ||
public int hashCode() { | ||
return Objects.hash(id, userAccumulatorList); | ||
} | ||
|
||
//--------------------------------------------------------------------------------- | ||
// Static helper classes | ||
//--------------------------------------------------------------------------------- | ||
|
||
/** | ||
* Json serializer for the {@link JobVertexAccumulatorsInfo}. | ||
*/ | ||
public static final class UserAccumulator { | ||
|
||
public static final String FIELD_NAME_ACC_NAME = "name"; | ||
public static final String FIELD_NAME_ACC_TYPE = "type"; | ||
public static final String FIELD_NAME_ACC_VALUE = "value"; | ||
|
||
@JsonProperty(FIELD_NAME_ACC_NAME) | ||
private String name; | ||
|
||
@JsonProperty(FIELD_NAME_ACC_TYPE) | ||
private String type; | ||
|
||
@JsonProperty(FIELD_NAME_ACC_VALUE) | ||
private String value; | ||
|
||
@JsonCreator | ||
public UserAccumulator( | ||
@JsonProperty(FIELD_NAME_ACC_NAME) String name, | ||
@JsonProperty(FIELD_NAME_ACC_TYPE) String type, | ||
@JsonProperty(FIELD_NAME_ACC_VALUE) String value) { | ||
this.name = name; | ||
this.type = type; | ||
this.value = value; | ||
} | ||
|
||
@Override | ||
public boolean equals(Object o) { | ||
if (this == o) { | ||
return true; | ||
} | ||
if (o == null || getClass() != o.getClass()) { | ||
return false; | ||
} | ||
UserAccumulator that = (UserAccumulator) o; | ||
return Objects.equals(name, that.name) && | ||
Objects.equals(type, that.type) && | ||
Objects.equals(value, that.value); | ||
} | ||
|
||
@Override | ||
public int hashCode() { | ||
return Objects.hash(name, type, value); | ||
} | ||
} | ||
} |
51 changes: 51 additions & 0 deletions
51
...e/src/test/java/org/apache/flink/runtime/rest/messages/JobVertexAccumulatorsInfoTest.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,51 @@ | ||
/* | ||
* 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; | ||
|
||
import java.util.ArrayList; | ||
import java.util.List; | ||
|
||
/** | ||
* Tests that the {@link JobVertexAccumulatorsInfo} can be marshalled and unmarshalled. | ||
*/ | ||
public class JobVertexAccumulatorsInfoTest extends RestResponseMarshallingTestBase<JobVertexAccumulatorsInfo> { | ||
@Override | ||
protected Class<JobVertexAccumulatorsInfo> getTestResponseClass() { | ||
return JobVertexAccumulatorsInfo.class; | ||
} | ||
|
||
@Override | ||
protected JobVertexAccumulatorsInfo getTestResponseInstance() throws Exception { | ||
List<JobVertexAccumulatorsInfo.UserAccumulator> userAccumulatorList = new ArrayList<>(); | ||
userAccumulatorList.add(new JobVertexAccumulatorsInfo.UserAccumulator( | ||
"test name1", | ||
"test type1", | ||
"test value1")); | ||
userAccumulatorList.add(new JobVertexAccumulatorsInfo.UserAccumulator( | ||
"test name2", | ||
"test type2", | ||
"test value2")); | ||
userAccumulatorList.add(new JobVertexAccumulatorsInfo.UserAccumulator( | ||
"test name3", | ||
"test type3", | ||
"test value3")); | ||
|
||
return new JobVertexAccumulatorsInfo("testId", userAccumulatorList); | ||
} | ||
} |