|
| 1 | +/** |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | +package org.apache.hadoop.yarn.server.timelineservice.reader; |
| 19 | + |
| 20 | +import com.fasterxml.jackson.databind.JsonNode; |
| 21 | +import com.fasterxml.jackson.databind.ObjectMapper; |
| 22 | +import org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity; |
| 23 | +import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity; |
| 24 | +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; |
| 25 | + |
| 26 | +import javax.ws.rs.Consumes; |
| 27 | +import javax.ws.rs.WebApplicationException; |
| 28 | +import javax.ws.rs.core.MediaType; |
| 29 | +import javax.ws.rs.core.MultivaluedMap; |
| 30 | +import javax.ws.rs.ext.MessageBodyReader; |
| 31 | +import javax.ws.rs.ext.Provider; |
| 32 | +import java.io.IOException; |
| 33 | +import java.io.InputStream; |
| 34 | +import java.lang.annotation.Annotation; |
| 35 | +import java.lang.reflect.Type; |
| 36 | +import java.util.ArrayList; |
| 37 | +import java.util.List; |
| 38 | +import java.util.Map; |
| 39 | + |
| 40 | +/** |
| 41 | + * We have defined a dedicated Reader for `List<FlowActivityEntity>`, |
| 42 | + * aimed at adapting to the Jersey2 framework |
| 43 | + * to ensure that JSON can be converted into `List<FlowActivityEntity>`. |
| 44 | + */ |
| 45 | +@Provider |
| 46 | +@Consumes(MediaType.APPLICATION_JSON) |
| 47 | +public class FlowActivityEntityListReader implements MessageBodyReader<List<FlowActivityEntity>> { |
| 48 | + |
| 49 | + private ObjectMapper objectMapper = new ObjectMapper(); |
| 50 | + private String timelineEntityType = |
| 51 | + "java.util.List<org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity>"; |
| 52 | + |
| 53 | + @Override |
| 54 | + public boolean isReadable(Class<?> type, Type genericType, |
| 55 | + Annotation[] annotations, MediaType mediaType) { |
| 56 | + return timelineEntityType.equals(genericType.getTypeName()); |
| 57 | + } |
| 58 | + |
| 59 | + @Override |
| 60 | + public List<FlowActivityEntity> readFrom(Class<List<FlowActivityEntity>> type, |
| 61 | + Type genericType, Annotation[] annotations, MediaType mediaType, |
| 62 | + MultivaluedMap<String, String> httpHeaders, |
| 63 | + InputStream entityStream) throws IOException, WebApplicationException { |
| 64 | + List<FlowActivityEntity> flowActivityEntityList = new ArrayList<>(); |
| 65 | + |
| 66 | + JsonNode jsonNode = objectMapper.readTree(entityStream); |
| 67 | + if (jsonNode.isArray()) { |
| 68 | + for (JsonNode jNode : jsonNode) { |
| 69 | + FlowActivityEntity entity = new FlowActivityEntity(); |
| 70 | + |
| 71 | + // Get Identifier |
| 72 | + JsonNode jnIdentifier = jNode.get("identifier"); |
| 73 | + JsonNode jnType = jnIdentifier.get("type"); |
| 74 | + JsonNode jnId = jnIdentifier.get("id"); |
| 75 | + TimelineEntity.Identifier identifier = new TimelineEntity.Identifier(jnType.asText(), jnId.asText()); |
| 76 | + entity.setIdentifier(identifier); |
| 77 | + |
| 78 | + // Get Type |
| 79 | + JsonNode jnAppType = jNode.get("type"); |
| 80 | + entity.setType(jnAppType.asText()); |
| 81 | + |
| 82 | + // Get Createdtime |
| 83 | + JsonNode jnCreatedTime = jNode.get("createdtime"); |
| 84 | + entity.setCreatedTime(jnCreatedTime.asLong()); |
| 85 | + |
| 86 | + // Get configs |
| 87 | + JsonNode jnConfigs = jNode.get("configs"); |
| 88 | + if (jnConfigs != null) { |
| 89 | + Map<String, String> configInfos = |
| 90 | + objectMapper.treeToValue(jnConfigs, Map.class); |
| 91 | + entity.setConfigs(configInfos); |
| 92 | + } |
| 93 | + |
| 94 | + // Get info |
| 95 | + JsonNode jnInfos = jNode.get("info"); |
| 96 | + if (jnInfos != null) { |
| 97 | + Map<String, Object> entityInfos = |
| 98 | + objectMapper.treeToValue(jnInfos, Map.class); |
| 99 | + entity.setInfo(entityInfos); |
| 100 | + } |
| 101 | + |
| 102 | + // Get BasicInfo |
| 103 | + entity.setDate(jNode.get("date").asLong()); |
| 104 | + entity.setCluster(jNode.get("cluster").asText()); |
| 105 | + entity.setUser(jNode.get("user").asText()); |
| 106 | + entity.setFlowName(jNode.get("flowName").asText()); |
| 107 | + |
| 108 | + // Get flowRuns |
| 109 | + JsonNode jnflowRuns = jNode.get("flowRuns"); |
| 110 | + if (jnflowRuns != null) { |
| 111 | + for (JsonNode jnflow : jnflowRuns) { |
| 112 | + FlowRunEntity flowRunEntity = objectMapper.treeToValue(jnflow, FlowRunEntity.class); |
| 113 | + entity.addFlowRun(flowRunEntity); |
| 114 | + } |
| 115 | + } |
| 116 | + flowActivityEntityList.add(entity); |
| 117 | + } |
| 118 | + } |
| 119 | + |
| 120 | + return flowActivityEntityList; |
| 121 | + } |
| 122 | +} |
0 commit comments