[FLINK-8141] [flip6] Fix JsonPlan serialization in JobDetailsInfo

The JsonPlan in JobDetailInfo must be serialized as a raw value
to make it parsable for downstream applications.

This closes #5109.
上级 c1ffc114
...@@ -126,7 +126,7 @@ public class JobDetailsHandler extends AbstractExecutionGraphHandler<JobDetailsI ...@@ -126,7 +126,7 @@ public class JobDetailsHandler extends AbstractExecutionGraphHandler<JobDetailsI
executionGraph.getJsonPlan()); executionGraph.getJsonPlan());
} }
public static JobDetailsInfo.JobVertexDetailsInfo createJobVertexDetailsInfo( private static JobDetailsInfo.JobVertexDetailsInfo createJobVertexDetailsInfo(
AccessExecutionJobVertex ejv, AccessExecutionJobVertex ejv,
long now, long now,
JobID jobId, JobID jobId,
......
...@@ -27,10 +27,12 @@ import org.apache.flink.runtime.rest.messages.json.JobIDDeserializer; ...@@ -27,10 +27,12 @@ import org.apache.flink.runtime.rest.messages.json.JobIDDeserializer;
import org.apache.flink.runtime.rest.messages.json.JobIDSerializer; import org.apache.flink.runtime.rest.messages.json.JobIDSerializer;
import org.apache.flink.runtime.rest.messages.json.JobVertexIDDeserializer; import org.apache.flink.runtime.rest.messages.json.JobVertexIDDeserializer;
import org.apache.flink.runtime.rest.messages.json.JobVertexIDSerializer; import org.apache.flink.runtime.rest.messages.json.JobVertexIDSerializer;
import org.apache.flink.runtime.rest.messages.json.RawJsonDeserializer;
import org.apache.flink.util.Preconditions; 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.JsonCreator;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonRawValue;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize; 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 org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonSerialize;
...@@ -103,6 +105,7 @@ public class JobDetailsInfo implements ResponseBody { ...@@ -103,6 +105,7 @@ public class JobDetailsInfo implements ResponseBody {
private final Map<ExecutionState, Integer> jobVerticesPerState; private final Map<ExecutionState, Integer> jobVerticesPerState;
@JsonProperty(FIELD_NAME_JSON_PLAN) @JsonProperty(FIELD_NAME_JSON_PLAN)
@JsonRawValue
private final String jsonPlan; private final String jsonPlan;
@JsonCreator @JsonCreator
...@@ -118,7 +121,7 @@ public class JobDetailsInfo implements ResponseBody { ...@@ -118,7 +121,7 @@ public class JobDetailsInfo implements ResponseBody {
@JsonProperty(FIELD_NAME_TIMESTAMPS) Map<JobStatus, Long> timestamps, @JsonProperty(FIELD_NAME_TIMESTAMPS) Map<JobStatus, Long> timestamps,
@JsonProperty(FIELD_NAME_JOB_VERTEX_INFOS) Collection<JobVertexDetailsInfo> jobVertexInfos, @JsonProperty(FIELD_NAME_JOB_VERTEX_INFOS) Collection<JobVertexDetailsInfo> jobVertexInfos,
@JsonProperty(FIELD_NAME_JOB_VERTICES_PER_STATE) Map<ExecutionState, Integer> jobVerticesPerState, @JsonProperty(FIELD_NAME_JOB_VERTICES_PER_STATE) Map<ExecutionState, Integer> jobVerticesPerState,
@JsonProperty(FIELD_NAME_JSON_PLAN) String jsonPlan) { @JsonProperty(FIELD_NAME_JSON_PLAN) @JsonDeserialize(using = RawJsonDeserializer.class) String jsonPlan) {
this.jobId = Preconditions.checkNotNull(jobId); this.jobId = Preconditions.checkNotNull(jobId);
this.name = Preconditions.checkNotNull(name); this.name = Preconditions.checkNotNull(name);
this.isStoppable = isStoppable; this.isStoppable = isStoppable;
......
/*
* 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.json;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.StdDeserializer;
import java.io.IOException;
/**
* Json deserializer which deserializes raw json.
*/
public final class RawJsonDeserializer extends StdDeserializer<String> {
private static final long serialVersionUID = -4089499607872996396L;
protected RawJsonDeserializer() {
super(String.class);
}
@Override
public String deserialize(JsonParser p, DeserializationContext ctxt) throws IOException {
final JsonNode jsonNode = ctxt.readValue(p, JsonNode.class);
return jsonNode.toString();
}
}
...@@ -44,7 +44,7 @@ public class JobDetailsInfoTest extends RestResponseMarshallingTestBase<JobDetai ...@@ -44,7 +44,7 @@ public class JobDetailsInfoTest extends RestResponseMarshallingTestBase<JobDetai
protected JobDetailsInfo getTestResponseInstance() throws Exception { protected JobDetailsInfo getTestResponseInstance() throws Exception {
final Random random = new Random(); final Random random = new Random();
final int numJobVertexDetailsInfos = 4; final int numJobVertexDetailsInfos = 4;
final String jsonPlan = "{id: \"1234\"}"; final String jsonPlan = "{\"id\":\"1234\"}";
final Map<JobStatus, Long> timestamps = new HashMap<>(JobStatus.values().length); final Map<JobStatus, Long> timestamps = new HashMap<>(JobStatus.values().length);
final Collection<JobDetailsInfo.JobVertexDetailsInfo> jobVertexInfos = new ArrayList<>(numJobVertexDetailsInfos); final Collection<JobDetailsInfo.JobVertexDetailsInfo> jobVertexInfos = new ArrayList<>(numJobVertexDetailsInfos);
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册