This is an automated email from the ASF dual-hosted git repository. chesnay pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/master by this push: new 8ee66535879 [FLINK-31735][docs] Document 'plan' field as object 8ee66535879 is described below commit 8ee66535879ce90882f0320b0c0d8ab7ef44ed37 Author: Chesnay Schepler <ches...@apache.org> AuthorDate: Thu Apr 6 17:21:58 2023 +0200 [FLINK-31735][docs] Document 'plan' field as object --- docs/layouts/shortcodes/generated/rest_v1_dispatcher.html | 9 ++++++--- docs/static/generated/rest_v1_dispatcher.yml | 2 +- flink-runtime-web/src/test/resources/rest_api_v1.snapshot | 12 ++++++++---- .../flink/runtime/rest/handler/job/JobDetailsHandler.java | 3 ++- .../org/apache/flink/runtime/rest/messages/JobPlanInfo.java | 12 +++++++++++- .../flink/runtime/rest/messages/job/JobDetailsInfo.java | 11 ++++------- .../flink/runtime/rest/messages/job/JobDetailsInfoTest.java | 3 ++- 7 files changed, 34 insertions(+), 18 deletions(-) diff --git a/docs/layouts/shortcodes/generated/rest_v1_dispatcher.html b/docs/layouts/shortcodes/generated/rest_v1_dispatcher.html index f64a73b508a..4d446c68a06 100644 --- a/docs/layouts/shortcodes/generated/rest_v1_dispatcher.html +++ b/docs/layouts/shortcodes/generated/rest_v1_dispatcher.html @@ -509,7 +509,8 @@ Using 'curl' you can upload a jar via 'curl -X POST -H "Expect:" -F "jarfile=@pa "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:JobPlanInfo", "properties" : { "plan" : { - "type" : "any" + "type" : "object", + "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:JobPlanInfo:RawJson" } } }</code></pre> @@ -1184,7 +1185,8 @@ Using 'curl' you can upload a jar via 'curl -X POST -H "Expect:" -F "jarfile=@pa "type" : "integer" }, "plan" : { - "type" : "string" + "type" : "object", + "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:JobPlanInfo:RawJson" }, "start-time" : { "type" : "integer" @@ -2949,7 +2951,8 @@ Using 'curl' you can upload a jar via 'curl -X POST -H "Expect:" -F "jarfile=@pa "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:JobPlanInfo", "properties" : { "plan" : { - "type" : "any" + "type" : "object", + "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:JobPlanInfo:RawJson" } } }</code></pre> diff --git a/docs/static/generated/rest_v1_dispatcher.yml b/docs/static/generated/rest_v1_dispatcher.yml index de43e28c617..7b0a1fac3cc 100644 --- a/docs/static/generated/rest_v1_dispatcher.yml +++ b/docs/static/generated/rest_v1_dispatcher.yml @@ -2337,7 +2337,7 @@ components: type: integer format: int64 plan: - type: string + $ref: '#/components/schemas/RawJson' start-time: type: integer format: int64 diff --git a/flink-runtime-web/src/test/resources/rest_api_v1.snapshot b/flink-runtime-web/src/test/resources/rest_api_v1.snapshot index 25d20a764b3..04f092a2f8b 100644 --- a/flink-runtime-web/src/test/resources/rest_api_v1.snapshot +++ b/flink-runtime-web/src/test/resources/rest_api_v1.snapshot @@ -335,7 +335,8 @@ "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:JobPlanInfo", "properties" : { "plan" : { - "type" : "any" + "type" : "object", + "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:JobPlanInfo:RawJson" } } } @@ -399,7 +400,8 @@ "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:JobPlanInfo", "properties" : { "plan" : { - "type" : "any" + "type" : "object", + "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:JobPlanInfo:RawJson" } } } @@ -967,7 +969,8 @@ } }, "plan" : { - "type" : "string" + "type" : "object", + "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:JobPlanInfo:RawJson" } } } @@ -2227,7 +2230,8 @@ "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:JobPlanInfo", "properties" : { "plan" : { - "type" : "any" + "type" : "object", + "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:JobPlanInfo:RawJson" } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobDetailsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobDetailsHandler.java index f38a7688ab9..fcc3771232b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobDetailsHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobDetailsHandler.java @@ -34,6 +34,7 @@ import org.apache.flink.runtime.rest.handler.util.MutableIOMetrics; import org.apache.flink.runtime.rest.messages.EmptyRequestBody; import org.apache.flink.runtime.rest.messages.JobIDPathParameter; import org.apache.flink.runtime.rest.messages.JobMessageParameters; +import org.apache.flink.runtime.rest.messages.JobPlanInfo; import org.apache.flink.runtime.rest.messages.MessageHeaders; import org.apache.flink.runtime.rest.messages.ResponseBody; import org.apache.flink.runtime.rest.messages.job.JobDetailsInfo; @@ -152,7 +153,7 @@ public class JobDetailsHandler timestamps, jobVertexInfos, jobVerticesPerStateMap, - executionGraph.getJsonPlan()); + new JobPlanInfo.RawJson(executionGraph.getJsonPlan())); } private static JobDetailsInfo.JobVertexDetailsInfo createJobVertexDetailsInfo( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobPlanInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobPlanInfo.java index 183c33a27d5..d82c9ec4657 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobPlanInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobPlanInfo.java @@ -27,11 +27,14 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonPro import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; 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.JavaType; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonMappingException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.SerializerProvider; 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.deser.std.StdDeserializer; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.jsonFormatVisitors.JsonFormatVisitorWrapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ser.std.StdSerializer; import java.io.IOException; @@ -87,7 +90,7 @@ public class JobPlanInfo implements ResponseBody { public static final class RawJson { private final String json; - private RawJson(String json) { + public RawJson(String json) { this.json = json; } @@ -134,6 +137,13 @@ public class JobPlanInfo implements ResponseBody { throws IOException { jsonGenerator.writeRawValue(jobPlanInfo.json); } + + @Override + public void acceptJsonFormatVisitor(JsonFormatVisitorWrapper visitor, JavaType typeHint) + throws JsonMappingException { + // this ensures the type is documented as "object" in the documentation + visitor.expectObjectFormat(typeHint); + } } /** Json deserializer for the {@link RawJson}. */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobDetailsInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobDetailsInfo.java index bd665e0afdb..6b624090b56 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobDetailsInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobDetailsInfo.java @@ -22,19 +22,18 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.rest.messages.JobPlanInfo; import org.apache.flink.runtime.rest.messages.ResponseBody; import org.apache.flink.runtime.rest.messages.job.metrics.IOMetricsInfo; 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.JobVertexIDDeserializer; 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.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore; 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.JsonSerialize; @@ -112,8 +111,7 @@ public class JobDetailsInfo implements ResponseBody { private final Map<ExecutionState, Integer> jobVerticesPerState; @JsonProperty(FIELD_NAME_JSON_PLAN) - @JsonRawValue - private final String jsonPlan; + private final JobPlanInfo.RawJson jsonPlan; @JsonCreator public JobDetailsInfo( @@ -132,8 +130,7 @@ public class JobDetailsInfo implements ResponseBody { Collection<JobVertexDetailsInfo> jobVertexInfos, @JsonProperty(FIELD_NAME_JOB_VERTICES_PER_STATE) Map<ExecutionState, Integer> jobVerticesPerState, - @JsonProperty(FIELD_NAME_JSON_PLAN) @JsonDeserialize(using = RawJsonDeserializer.class) - String jsonPlan) { + @JsonProperty(FIELD_NAME_JSON_PLAN) JobPlanInfo.RawJson jsonPlan) { this.jobId = Preconditions.checkNotNull(jobId); this.name = Preconditions.checkNotNull(name); this.isStoppable = isStoppable; @@ -253,7 +250,7 @@ public class JobDetailsInfo implements ResponseBody { @JsonIgnore public String getJsonPlan() { - return jsonPlan; + return jsonPlan.toString(); } // --------------------------------------------------- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/JobDetailsInfoTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/JobDetailsInfoTest.java index 8c207c5b069..1c281cf24e0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/JobDetailsInfoTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/JobDetailsInfoTest.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.rest.messages.JobPlanInfo; import org.apache.flink.runtime.rest.messages.RestResponseMarshallingTestBase; import org.apache.flink.runtime.rest.messages.job.metrics.IOMetricsInfo; @@ -76,7 +77,7 @@ public class JobDetailsInfoTest extends RestResponseMarshallingTestBase<JobDetai timestamps, jobVertexInfos, jobVerticesPerState, - jsonPlan); + new JobPlanInfo.RawJson(jsonPlan)); } private JobDetailsInfo.JobVertexDetailsInfo createJobVertexDetailsInfo(Random random) {