Skip to content

Commit

Permalink
Merge 916596e into d519eb7
Browse files Browse the repository at this point in the history
  • Loading branch information
yew1eb committed Apr 23, 2018
2 parents d519eb7 + 916596e commit 254571a
Show file tree
Hide file tree
Showing 6 changed files with 372 additions and 2 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,82 @@
/*
* 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.AccessExecutionJobVertex;
import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
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.JobVertexMessageParameters;
import org.apache.flink.runtime.rest.messages.MessageHeaders;
import org.apache.flink.runtime.rest.messages.job.SubtasksAllAccumulatorsInfo;
import org.apache.flink.runtime.rest.messages.job.UserAccumulator;
import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
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.Map;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Executor;

/**
* Request handler for the subtasks all accumulators.
*/
public class SubtasksAllAccumulatorsHandler extends AbstractJobVertexHandler<SubtasksAllAccumulatorsInfo, JobVertexMessageParameters> {

public SubtasksAllAccumulatorsHandler(CompletableFuture<String> localRestAddress, GatewayRetriever<? extends RestfulGateway> leaderRetriever, Time timeout, Map<String, String> responseHeaders, MessageHeaders<EmptyRequestBody, SubtasksAllAccumulatorsInfo, JobVertexMessageParameters> messageHeaders, ExecutionGraphCache executionGraphCache, Executor executor) {
super(localRestAddress, leaderRetriever, timeout, responseHeaders, messageHeaders, executionGraphCache, executor);
}

@Override
protected SubtasksAllAccumulatorsInfo handleRequest(HandlerRequest<EmptyRequestBody, JobVertexMessageParameters> request, AccessExecutionJobVertex jobVertex) throws RestHandlerException {
JobVertexID jobVertexId = jobVertex.getJobVertexId();
int parallelism = jobVertex.getParallelism();

final List<SubtasksAllAccumulatorsInfo.SubtaskAccumulatorsInfo> subtaskAccumulatorsInfos = new ArrayList<>();

for (AccessExecutionVertex vertex : jobVertex.getTaskVertices()) {
TaskManagerLocation location = vertex.getCurrentAssignedResourceLocation();
String locationString = location == null ? "(unassigned)" : location.getHostname();

StringifiedAccumulatorResult[] accs = vertex.getCurrentExecutionAttempt().getUserAccumulatorsStringified();
List<UserAccumulator> userAccumulators = new ArrayList<>(accs.length);
for (StringifiedAccumulatorResult acc : accs) {
userAccumulators.add(new UserAccumulator(acc.getName(), acc.getType(), acc.getValue()));
}

subtaskAccumulatorsInfos.add(
new SubtasksAllAccumulatorsInfo.SubtaskAccumulatorsInfo(
vertex.getCurrentExecutionAttempt().getParallelSubtaskIndex(),
vertex.getCurrentExecutionAttempt().getAttemptNumber(),
locationString,
userAccumulators
));
}

return new SubtasksAllAccumulatorsInfo(jobVertexId, parallelism, subtaskAccumulatorsInfos);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -97,15 +97,14 @@ public static String createSubtasksAccumulatorsJson(AccessExecutionJobVertex job

gen.writeArrayFieldStart("subtasks");

int num = 0;
for (AccessExecutionVertex vertex : jobVertex.getTaskVertices()) {

TaskManagerLocation location = vertex.getCurrentAssignedResourceLocation();
String locationString = location == null ? "(unassigned)" : location.getHostname();

gen.writeStartObject();

gen.writeNumberField("subtask", num++);
gen.writeNumberField("subtask", vertex.getCurrentExecutionAttempt().getParallelSubtaskIndex());
gen.writeNumberField("attempt", vertex.getCurrentExecutionAttempt().getAttemptNumber());
gen.writeStringField("host", locationString);

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,75 @@
/*
* 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.SubtasksAllAccumulatorsHandler;
import org.apache.flink.runtime.rest.messages.job.SubtasksAllAccumulatorsInfo;

import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;

/**
* Message headers for the {@link SubtasksAllAccumulatorsHandler}.
*/
public class SubtasksAllAccumulatorsHandlers implements MessageHeaders<EmptyRequestBody, SubtasksAllAccumulatorsInfo, JobVertexMessageParameters> {

private static final SubtasksAllAccumulatorsHandlers INSTANCE = new SubtasksAllAccumulatorsHandlers();

public static final String URL = "/jobs" +
"/:" + JobIDPathParameter.KEY +
"/vertices" +
"/:" + JobVertexIdPathParameter.KEY +
"/subtasks/accumulators";

private SubtasksAllAccumulatorsHandlers() {}

@Override
public Class<EmptyRequestBody> getRequestClass() {
return EmptyRequestBody.class;
}

@Override
public Class<SubtasksAllAccumulatorsInfo> getResponseClass() {
return SubtasksAllAccumulatorsInfo.class;
}

@Override
public HttpResponseStatus getResponseStatusCode() {
return HttpResponseStatus.OK;
}

@Override
public JobVertexMessageParameters getUnresolvedMessageParameters() {
return new JobVertexMessageParameters();
}

@Override
public HttpMethodWrapper getHttpMethod() {
return HttpMethodWrapper.GET;
}

@Override
public String getTargetRestEndpointURL() {
return URL;
}

public static SubtasksAllAccumulatorsHandlers getInstance() {
return INSTANCE;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,144 @@
/*
* 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;

import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.rest.handler.job.SubtasksAllAccumulatorsHandler;
import org.apache.flink.runtime.rest.messages.ResponseBody;
import org.apache.flink.runtime.rest.messages.json.JobVertexIDDeserializer;
import org.apache.flink.runtime.rest.messages.json.JobVertexIDSerializer;
import org.apache.flink.util.Preconditions;

import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonSerialize;

import java.util.Collection;
import java.util.Objects;

/**
* Response type of the {@link SubtasksAllAccumulatorsHandler}.
*/
public class SubtasksAllAccumulatorsInfo implements ResponseBody {

public static final String FIELD_NAME_JOB_VERTEX_ID = "id";
public static final String FIELD_NAME_PARALLELISM = "parallelism";
public static final String FILED_NMAE_SUBTASKS = "subtasks";

@JsonProperty(FIELD_NAME_JOB_VERTEX_ID)
@JsonSerialize(using = JobVertexIDSerializer.class)
private final JobVertexID jobVertexId;

@JsonProperty(FIELD_NAME_PARALLELISM)
private final int parallelism;

@JsonProperty(FILED_NMAE_SUBTASKS)
private final Collection<SubtaskAccumulatorsInfo> subtaskAccumulatorsInfos;

@JsonCreator
public SubtasksAllAccumulatorsInfo(
@JsonDeserialize(using = JobVertexIDDeserializer.class) @JsonProperty(FIELD_NAME_JOB_VERTEX_ID) JobVertexID jobVertexId,
@JsonProperty(FIELD_NAME_PARALLELISM) int parallelism,
@JsonProperty(FILED_NMAE_SUBTASKS) Collection<SubtaskAccumulatorsInfo> subtaskAccumulatorsInfos) {
this.jobVertexId = Preconditions.checkNotNull(jobVertexId);
this.parallelism = parallelism;
this.subtaskAccumulatorsInfos = Preconditions.checkNotNull(subtaskAccumulatorsInfos);
}

@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
SubtasksAllAccumulatorsInfo that = (SubtasksAllAccumulatorsInfo) o;
return Objects.equals(jobVertexId, that.jobVertexId) &&
parallelism == that.parallelism &&
Objects.equals(subtaskAccumulatorsInfos, that.subtaskAccumulatorsInfos);
}

@Override
public int hashCode() {
return Objects.hash(jobVertexId, parallelism, subtaskAccumulatorsInfos);
}

// ---------------------------------------------------
// Static inner classes
// ---------------------------------------------------

/**
* Detailed information about subtask accumulators.
*/
public static class SubtaskAccumulatorsInfo {
public static final String FIELD_NAME_SUBTASK_INDEX = "subtask";
public static final String FIELD_NAME_ATTEMPT_NUM = "attempt";
public static final String FIELD_NAME_HOST = "host";
public static final String FIELD_NAME_USER_ACCUMULATORS = "user-accumulators";


@JsonProperty(FIELD_NAME_SUBTASK_INDEX)
private final int subtaskIndex;

@JsonProperty(FIELD_NAME_ATTEMPT_NUM)
private final int attemptNum;

@JsonProperty(FIELD_NAME_HOST)
private final String host;

@JsonProperty(FIELD_NAME_USER_ACCUMULATORS)
private final Collection<UserAccumulator> userAccumulators;

@JsonCreator
public SubtaskAccumulatorsInfo(
@JsonProperty(FIELD_NAME_SUBTASK_INDEX) int subtaskIndex,
@JsonProperty(FIELD_NAME_ATTEMPT_NUM) int attemptNum,
@JsonProperty(FIELD_NAME_HOST) String host,
@JsonProperty(FIELD_NAME_USER_ACCUMULATORS) Collection<UserAccumulator> userAccumulators) {

this.subtaskIndex = subtaskIndex;
this.attemptNum = attemptNum;
this.host = Preconditions.checkNotNull(host);
this.userAccumulators = Preconditions.checkNotNull(userAccumulators);
}

@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
SubtaskAccumulatorsInfo that = (SubtaskAccumulatorsInfo) o;
return subtaskIndex == that.subtaskIndex &&
attemptNum == that.attemptNum &&
Objects.equals(host, that.host) &&
Objects.equals(userAccumulators, that.userAccumulators);
}

@Override
public int hashCode() {
return Objects.hash(subtaskIndex, attemptNum, host, userAccumulators);
}

}
}
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@
import org.apache.flink.runtime.rest.handler.job.SubtaskCurrentAttemptDetailsHandler;
import org.apache.flink.runtime.rest.handler.job.SubtaskExecutionAttemptAccumulatorsHandler;
import org.apache.flink.runtime.rest.handler.job.SubtaskExecutionAttemptDetailsHandler;
import org.apache.flink.runtime.rest.handler.job.SubtasksAllAccumulatorsHandler;
import org.apache.flink.runtime.rest.handler.job.SubtasksTimesHandler;
import org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointConfigHandler;
import org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointStatisticDetailsHandler;
Expand Down Expand Up @@ -94,6 +95,7 @@
import org.apache.flink.runtime.rest.messages.JobVertexDetailsHeaders;
import org.apache.flink.runtime.rest.messages.JobVertexTaskManagersHeaders;
import org.apache.flink.runtime.rest.messages.JobsOverviewHeaders;
import org.apache.flink.runtime.rest.messages.SubtasksAllAccumulatorsHandlers;
import org.apache.flink.runtime.rest.messages.SubtasksTimesHeaders;
import org.apache.flink.runtime.rest.messages.TerminationModeQueryParameter;
import org.apache.flink.runtime.rest.messages.YarnCancelJobTerminationHeaders;
Expand Down Expand Up @@ -318,6 +320,15 @@ protected List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initiali
executionGraphCache,
executor);

SubtasksAllAccumulatorsHandler subtasksAllAccumulatorsHandler = new SubtasksAllAccumulatorsHandler(
restAddressFuture,
leaderRetriever,
timeout,
responseHeaders,
SubtasksAllAccumulatorsHandlers.getInstance(),
executionGraphCache,
executor);

TaskManagersHandler taskManagersHandler = new TaskManagersHandler(
restAddressFuture,
leaderRetriever,
Expand Down Expand Up @@ -575,6 +586,7 @@ protected List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initiali
handlers.add(Tuple2.of(TaskCheckpointStatisticsHeaders.getInstance(), taskCheckpointStatisticDetailsHandler));
handlers.add(Tuple2.of(JobExceptionsHeaders.getInstance(), jobExceptionsHandler));
handlers.add(Tuple2.of(JobVertexAccumulatorsHeaders.getInstance(), jobVertexAccumulatorsHandler));
handlers.add(Tuple2.of(JobVertexAccumulatorsHeaders.getInstance(), subtasksAllAccumulatorsHandler));
handlers.add(Tuple2.of(JobDetailsHeaders.getInstance(), jobDetailsHandler));
handlers.add(Tuple2.of(JobAccumulatorsHeaders.getInstance(), jobAccumulatorsHandler));
handlers.add(Tuple2.of(TaskManagersHeaders.getInstance(), taskManagersHandler));
Expand Down
Loading

0 comments on commit 254571a

Please sign in to comment.