Browse Source

YARN-11786. Upgrade hadoop-yarn-server-timelineservice-hbase-tests to Support Trunk Compilation and Remove compatible hadoop version. (#7453)

* Upgrade hadoop-yarn-server-timelineservice-hbase-tests to Support Trunk Compilation and Remove compatible hadoop version.

Co-authored-by: Chris Nauroth <cnauroth@apache.org>
Co-authored-by: Hualong Zhang <hualong.z@hotmail.com>
Reviewed-by: Chris Nauroth <cnauroth@apache.org>
Reviewed-by: Hualong Zhang <hualong.z@hotmail.com>
Signed-off-by: Shilun Fan <slfan1989@apache.org>
slfan1989 1 month ago
parent
commit
4baa167f0a
17 changed files with 912 additions and 217 deletions
  1. 1 1
      BUILDING.txt
  2. 1 1
      hadoop-project/pom.xml
  3. 3 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowRunEntity.java
  4. 3 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/writer/TimelineEntitySetWriter.java
  5. 16 21
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml
  6. 41 40
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/AbstractTimelineReaderHBaseTestBase.java
  7. 123 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/FlowActivityEntityListReader.java
  8. 63 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/FlowActivityEntityReader.java
  9. 124 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/FlowActivityEntitySetReader.java
  10. 62 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/FlowRunEntityReader.java
  11. 73 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/FlowRunEntitySetReader.java
  12. 148 148
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
  13. 122 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityListReader.java
  14. 125 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntitySetReader.java
  15. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
  16. 6 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java
  17. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml

+ 1 - 1
BUILDING.txt

@@ -163,7 +163,7 @@ Maven build goals:
  YARN Application Timeline Service V2 build options:
 
    YARN Timeline Service v.2 chooses Apache HBase as the primary backing storage. The supported
-   version of Apache HBase is 2.5.8.
+   version of Apache HBase is 2.6.1.
 
  Snappy build options:
 

+ 1 - 1
hadoop-project/pom.xml

@@ -222,7 +222,7 @@
     <swagger-annotations-version>1.5.4</swagger-annotations-version>
     <snakeyaml.version>2.0</snakeyaml.version>
     <sshd.version>2.11.0</sshd.version>
-    <hbase.version>2.5.8-hadoop3</hbase.version>
+    <hbase.version>2.6.1-hadoop3</hbase.version>
     <junit.version>4.13.2</junit.version>
     <junit.jupiter.version>5.8.2</junit.jupiter.version>
     <junit.vintage.version>5.8.2</junit.vintage.version>

+ 3 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowRunEntity.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.api.records.timelineservice;
 
 import javax.xml.bind.annotation.XmlElement;
 
+import com.fasterxml.jackson.annotation.JsonInclude;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
@@ -27,6 +28,7 @@ import org.apache.hadoop.classification.InterfaceStability;
  */
 @InterfaceAudience.Public
 @InterfaceStability.Unstable
+@JsonInclude(JsonInclude.Include.NON_NULL)
 public class FlowRunEntity extends HierarchicalTimelineEntity {
   public static final String USER_INFO_KEY =
       TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "USER";
@@ -107,7 +109,7 @@ public class FlowRunEntity extends HierarchicalTimelineEntity {
     addInfo(FLOW_RUN_ID_INFO_KEY, runId);
   }
 
-  public long getStartTime() {
+  public Long getStartTime() {
     return getCreatedTime();
   }
 

+ 3 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/writer/TimelineEntitySetWriter.java

@@ -43,11 +43,13 @@ import java.util.Set;
 public class TimelineEntitySetWriter implements MessageBodyWriter<Set<TimelineEntity>> {
 
   private ObjectMapper objectMapper = new ObjectMapper();
+  private String timelineEntityType =
+      "java.util.Set<org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity>";
 
   @Override
   public boolean isWriteable(Class<?> type, Type genericType,
       Annotation[] annotations, MediaType mediaType) {
-    return true;
+    return timelineEntityType.equals(genericType.getTypeName());
   }
 
   @Override

+ 16 - 21
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml

@@ -89,7 +89,6 @@
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
-      <version>${hbase-compatible-hadoop.version}</version>
       <scope>test</scope>
       <exclusions>
         <exclusion>
@@ -104,10 +103,6 @@
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-hdfs-client</artifactId>
         </exclusion>
-        <exclusion>
-          <groupId>com.sun.jersey</groupId>
-          <artifactId>jersey-json</artifactId>
-        </exclusion>
       </exclusions>
     </dependency>
 
@@ -131,13 +126,22 @@
       <artifactId>junit-platform-launcher</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.junit.vintage</groupId>
+      <artifactId>junit-vintage-engine</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.glassfish.jersey.core</groupId>
+      <artifactId>jersey-server</artifactId>
+      <scope>test</scope>
+    </dependency>
 
     <!-- 'mvn dependency:analyze' fails to detect use of this direct
          dependency -->
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-auth</artifactId>
-      <version>${hbase-compatible-hadoop.version}</version>
       <scope>test</scope>
       <exclusions>
         <exclusion>
@@ -242,13 +246,6 @@
       <scope>runtime</scope>
     </dependency>
 
-    <dependency>
-      <groupId>com.sun.jersey</groupId>
-      <artifactId>jersey-client</artifactId>
-      <version>1.19.4</version>
-      <scope>test</scope>
-    </dependency>
-
     <dependency>
       <groupId>javax.ws.rs</groupId>
       <artifactId>jsr311-api</artifactId>
@@ -369,7 +366,6 @@
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
-      <version>${hbase-compatible-hadoop.version}</version>
       <type>test-jar</type>
       <scope>test</scope>
       <exclusions>
@@ -377,10 +373,6 @@
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-auth</artifactId>
         </exclusion>
-        <exclusion>
-          <groupId>com.sun.jersey</groupId>
-          <artifactId>jersey-json</artifactId>
-        </exclusion>
         <exclusion>
           <groupId>commons-logging</groupId>
           <artifactId>commons-logging</artifactId>
@@ -393,7 +385,6 @@
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-hdfs</artifactId>
-      <version>${hbase-compatible-hadoop.version}</version>
       <scope>test</scope>
       <exclusions>
         <exclusion>
@@ -412,7 +403,6 @@
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-hdfs</artifactId>
-      <version>${hbase-compatible-hadoop.version}</version>
       <type>test-jar</type>
       <scope>test</scope>
       <exclusions>
@@ -430,7 +420,6 @@
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-hdfs-client</artifactId>
-      <version>${hbase-compatible-hadoop.version}</version>
       <scope>test</scope>
     </dependency>
 
@@ -507,6 +496,12 @@
       <artifactId>mockito-inline</artifactId>
       <scope>test</scope>
     </dependency>
+
+    <dependency>
+      <groupId>org.glassfish.jersey.media</groupId>
+      <artifactId>jersey-media-json-jettison</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>

+ 41 - 40
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/AbstractTimelineReaderHBaseTestBase.java

@@ -22,32 +22,29 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
+import javax.ws.rs.client.Client;
+import javax.ws.rs.client.ClientBuilder;
+
 import java.io.IOException;
-import java.lang.reflect.UndeclaredThrowableException;
 import java.net.HttpURLConnection;
 import java.net.URI;
-import java.net.URL;
 import java.util.List;
 
+import javax.ws.rs.core.GenericType;
 import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
 
+import org.apache.hadoop.classification.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.reader.TimelineEntityReader;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.timelineservice.storage.DataGeneratorForTest;
-import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
+import org.glassfish.jersey.client.ClientConfig;
+import org.glassfish.jersey.client.HttpUrlConnectorProvider;
 import org.junit.Assert;
 
-import com.sun.jersey.api.client.Client;
-import com.sun.jersey.api.client.ClientResponse;
-import com.sun.jersey.api.client.ClientResponse.Status;
-import com.sun.jersey.api.client.GenericType;
-import com.sun.jersey.api.client.config.ClientConfig;
-import com.sun.jersey.api.client.config.DefaultClientConfig;
-import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
-import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
-
 /**
  * Test Base for TimelineReaderServer HBase tests.
  */
@@ -109,19 +106,25 @@ public abstract class AbstractTimelineReaderHBaseTestBase {
   }
 
   protected Client createClient() {
-    ClientConfig cfg = new DefaultClientConfig();
-    cfg.getClasses().add(YarnJacksonJaxbJsonProvider.class);
-    return new Client(
-        new URLConnectionClientHandler(new DummyURLConnectionFactory()), cfg);
+    final ClientConfig cc = new ClientConfig();
+    cc.connectorProvider(getHttpURLConnectionFactory());
+    return ClientBuilder.newClient(cc)
+        .register(TimelineEntityReader.class)
+        .register(TimelineEntitySetReader.class)
+        .register(TimelineEntityListReader.class)
+        .register(FlowActivityEntityReader.class)
+        .register(FlowRunEntityReader.class)
+        .register(FlowActivityEntitySetReader.class)
+        .register(FlowActivityEntityListReader.class)
+        .register(FlowRunEntitySetReader.class);
   }
 
-  protected ClientResponse getResponse(Client client, URI uri)
+  protected Response getResponse(Client client, URI uri)
       throws Exception {
-    ClientResponse resp =
-        client.resource(uri).accept(MediaType.APPLICATION_JSON)
-            .type(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+    Response resp =
+        client.target(uri).request(MediaType.APPLICATION_JSON).get();
     if (resp == null || resp.getStatusInfo()
-        .getStatusCode() != ClientResponse.Status.OK.getStatusCode()) {
+        .getStatusCode() != HttpURLConnection.HTTP_OK) {
       String msg = "";
       if (resp != null) {
         msg = String.valueOf(resp.getStatusInfo().getStatusCode());
@@ -132,39 +135,37 @@ public abstract class AbstractTimelineReaderHBaseTestBase {
     return resp;
   }
 
-  protected void verifyHttpResponse(Client client, URI uri, Status status) {
-    ClientResponse resp =
-        client.resource(uri).accept(MediaType.APPLICATION_JSON)
-            .type(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+  protected void verifyHttpResponse(Client client, URI uri, Response.Status status) {
+    Response resp = client.target(uri).request(MediaType.APPLICATION_JSON).get();
     assertNotNull(resp);
     assertTrue("Response from server should have been " + status,
         resp.getStatusInfo().getStatusCode() == status.getStatusCode());
-    System.out.println("Response is: " + resp.getEntity(String.class));
+    System.out.println("Response is: " + resp.readEntity(String.class));
   }
 
   protected List<FlowActivityEntity> verifyFlowEntites(Client client, URI uri,
       int noOfEntities) throws Exception {
-    ClientResponse resp = getResponse(client, uri);
+    Response resp = getResponse(client, uri);
     List<FlowActivityEntity> entities =
-        resp.getEntity(new GenericType<List<FlowActivityEntity>>() {
+        resp.readEntity(new GenericType<List<FlowActivityEntity>>() {
         });
     assertNotNull(entities);
     assertEquals(noOfEntities, entities.size());
     return entities;
   }
 
-  protected static class DummyURLConnectionFactory
-      implements HttpURLConnectionFactory {
-
-    @Override
-    public HttpURLConnection getHttpURLConnection(final URL url)
-        throws IOException {
-      try {
-        return (HttpURLConnection) url.openConnection();
-      } catch (UndeclaredThrowableException e) {
-        throw new IOException(e.getCause());
-      }
-    }
+  @VisibleForTesting
+  protected HttpUrlConnectorProvider getHttpURLConnectionFactory() {
+    return new HttpUrlConnectorProvider().connectionFactory(
+        url -> {
+          HttpURLConnection conn;
+          try {
+            conn =  (HttpURLConnection) url.openConnection();
+          } catch (Exception e) {
+            throw new IOException(e);
+          }
+          return conn;
+        });
   }
 
   protected static HBaseTestingUtility getHBaseTestingUtility() {

+ 123 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/FlowActivityEntityListReader.java

@@ -0,0 +1,123 @@
+/**
+ * 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.hadoop.yarn.server.timelineservice.reader;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+
+import javax.ws.rs.Consumes;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.ext.MessageBodyReader;
+import javax.ws.rs.ext.Provider;
+import java.io.IOException;
+import java.io.InputStream;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Type;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * We have defined a dedicated Reader for `List<FlowActivityEntity>`,
+ * aimed at adapting to the Jersey2 framework
+ * to ensure that JSON can be converted into `List<FlowActivityEntity>`.
+ */
+@Provider
+@Consumes(MediaType.APPLICATION_JSON)
+public class FlowActivityEntityListReader implements MessageBodyReader<List<FlowActivityEntity>> {
+
+  private ObjectMapper objectMapper = new ObjectMapper();
+  private String timelineEntityType =
+      "java.util.List<org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity>";
+
+  @Override
+  public boolean isReadable(Class<?> type, Type genericType,
+      Annotation[] annotations, MediaType mediaType) {
+    return timelineEntityType.equals(genericType.getTypeName());
+  }
+
+  @Override
+  public List<FlowActivityEntity> readFrom(Class<List<FlowActivityEntity>> type,
+      Type genericType, Annotation[] annotations, MediaType mediaType,
+      MultivaluedMap<String, String> httpHeaders,
+      InputStream entityStream) throws IOException, WebApplicationException {
+    List<FlowActivityEntity> flowActivityEntityList = new ArrayList<>();
+
+    JsonNode jsonNode = objectMapper.readTree(entityStream);
+    if (jsonNode.isArray()) {
+      for (JsonNode jNode : jsonNode) {
+        FlowActivityEntity entity = new FlowActivityEntity();
+
+        // Get Identifier
+        JsonNode jnIdentifier = jNode.get("identifier");
+        JsonNode jnType = jnIdentifier.get("type");
+        JsonNode jnId = jnIdentifier.get("id");
+        TimelineEntity.Identifier identifier =
+            new TimelineEntity.Identifier(jnType.asText(), jnId.asText());
+        entity.setIdentifier(identifier);
+
+        // Get Type
+        JsonNode jnAppType = jNode.get("type");
+        entity.setType(jnAppType.asText());
+
+        // Get Createdtime
+        JsonNode jnCreatedTime = jNode.get("createdtime");
+        entity.setCreatedTime(jnCreatedTime.asLong());
+
+        // Get configs
+        JsonNode jnConfigs = jNode.get("configs");
+        if (jnConfigs != null) {
+          Map<String, String> configInfos =
+              objectMapper.treeToValue(jnConfigs, Map.class);
+          entity.setConfigs(configInfos);
+        }
+
+        // Get info
+        JsonNode jnInfos = jNode.get("info");
+        if (jnInfos != null) {
+          Map<String, Object> entityInfos =
+              objectMapper.treeToValue(jnInfos, Map.class);
+          entity.setInfo(entityInfos);
+        }
+
+        // Get BasicInfo
+        entity.setDate(jNode.get("date").asLong());
+        entity.setCluster(jNode.get("cluster").asText());
+        entity.setUser(jNode.get("user").asText());
+        entity.setFlowName(jNode.get("flowName").asText());
+
+        // Get flowRuns
+        JsonNode jnflowRuns = jNode.get("flowRuns");
+        if (jnflowRuns != null) {
+          for (JsonNode jnflow : jnflowRuns) {
+            FlowRunEntity flowRunEntity = objectMapper.treeToValue(jnflow, FlowRunEntity.class);
+            entity.addFlowRun(flowRunEntity);
+          }
+        }
+        flowActivityEntityList.add(entity);
+      }
+    }
+
+    return flowActivityEntityList;
+  }
+}

+ 63 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/FlowActivityEntityReader.java

@@ -0,0 +1,63 @@
+/**
+ * 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.hadoop.yarn.server.timelineservice.reader;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity;
+
+import javax.ws.rs.Consumes;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.ext.MessageBodyReader;
+import javax.ws.rs.ext.Provider;
+import java.io.IOException;
+import java.io.InputStream;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Type;
+
+/**
+ * We have defined a dedicated Reader for FlowActivityEntity,
+ * aimed at adapting to the Jersey2 framework
+ * to ensure that JSON can be converted into FlowActivityEntity.
+ */
+@Provider
+@Consumes(MediaType.APPLICATION_JSON)
+public class FlowActivityEntityReader implements MessageBodyReader<FlowActivityEntity> {
+
+  private ObjectMapper objectMapper = new ObjectMapper();
+
+  @Override
+  public boolean isReadable(Class<?> type, Type genericType,
+      Annotation[] annotations, MediaType mediaType) {
+    return type == FlowActivityEntity.class;
+  }
+
+  @Override
+  public FlowActivityEntity readFrom(Class<FlowActivityEntity> type, Type genericType,
+      Annotation[] annotations, MediaType mediaType, MultivaluedMap<String, String> httpHeaders,
+      InputStream entityStream) throws IOException, WebApplicationException {
+    try {
+      FlowActivityEntity timelineEntity =
+          objectMapper.readValue(entityStream, FlowActivityEntity.class);
+      return timelineEntity;
+    } catch (Exception e) {
+      return new FlowActivityEntity();
+    }
+  }
+}

+ 124 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/FlowActivityEntitySetReader.java

@@ -0,0 +1,124 @@
+/**
+ * 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.hadoop.yarn.server.timelineservice.reader;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+
+import javax.ws.rs.Consumes;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.ext.MessageBodyReader;
+import javax.ws.rs.ext.Provider;
+import java.io.IOException;
+import java.io.InputStream;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Type;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * We have defined a dedicated Reader for `Set<FlowActivityEntity>`,
+ * aimed at adapting to the Jersey2 framework
+ * to ensure that JSON can be converted into `Set<FlowActivityEntity>`.
+ */
+@Provider
+@Consumes(MediaType.APPLICATION_JSON)
+public class FlowActivityEntitySetReader implements MessageBodyReader<Set<FlowActivityEntity>> {
+
+  private ObjectMapper objectMapper = new ObjectMapper();
+  private String timelineEntityType =
+      "java.util.Set<org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity>";
+
+  @Override
+  public boolean isReadable(Class<?> type, Type genericType,
+      Annotation[] annotations, MediaType mediaType) {
+    return timelineEntityType.equals(genericType.getTypeName());
+  }
+
+  @Override
+  public Set<FlowActivityEntity> readFrom(Class<Set<FlowActivityEntity>> type,
+      Type genericType, Annotation[] annotations, MediaType mediaType,
+      MultivaluedMap<String, String> httpHeaders,
+      InputStream entityStream) throws IOException, WebApplicationException {
+    Set<FlowActivityEntity> flowActivityEntitySet = new HashSet<>();
+
+    JsonNode jsonNode = objectMapper.readTree(entityStream);
+    if (jsonNode.isArray()) {
+      for (JsonNode jNode : jsonNode) {
+        FlowActivityEntity entity = new FlowActivityEntity();
+
+        // Get Identifier
+        JsonNode jnIdentifier = jNode.get("identifier");
+        JsonNode jnType = jnIdentifier.get("type");
+        JsonNode jnId = jnIdentifier.get("id");
+        TimelineEntity.Identifier identifier =
+            new TimelineEntity.Identifier(jnType.asText(), jnId.asText());
+        entity.setIdentifier(identifier);
+
+        // Get Type
+        JsonNode jnAppType = jNode.get("type");
+        entity.setType(jnAppType.asText());
+
+        // Get Createdtime
+        JsonNode jnCreatedTime = jNode.get("createdtime");
+        entity.setCreatedTime(jnCreatedTime.asLong());
+
+        // Get configs
+        JsonNode jnConfigs = jNode.get("configs");
+        if (jnConfigs != null) {
+          Map<String, String> configInfos =
+              objectMapper.treeToValue(jnConfigs, Map.class);
+          entity.setConfigs(configInfos);
+        }
+
+        // Get info
+        JsonNode jnInfos = jNode.get("info");
+        if (jnInfos != null) {
+          Map<String, Object> entityInfos =
+              objectMapper.treeToValue(jnInfos, Map.class);
+          entity.setInfo(entityInfos);
+        }
+
+        // Get BasicInfo
+        entity.setDate(jNode.get("date").asLong());
+        entity.setCluster(jNode.get("cluster").asText());
+        entity.setUser(jNode.get("user").asText());
+        entity.setFlowName(jNode.get("flowName").asText());
+
+        // Get flowRuns
+        JsonNode jnflowRuns = jNode.get("flowRuns");
+        if (jnflowRuns != null) {
+          for (JsonNode jnflow : jnflowRuns) {
+            FlowRunEntity flowRunEntity = objectMapper.treeToValue(jnflow, FlowRunEntity.class);
+            entity.addFlowRun(flowRunEntity);
+          }
+        }
+
+        flowActivityEntitySet.add(entity);
+      }
+    }
+
+    return flowActivityEntitySet;
+  }
+}

+ 62 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/FlowRunEntityReader.java

@@ -0,0 +1,62 @@
+/**
+ * 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.hadoop.yarn.server.timelineservice.reader;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
+
+import javax.ws.rs.Consumes;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.ext.MessageBodyReader;
+import javax.ws.rs.ext.Provider;
+import java.io.IOException;
+import java.io.InputStream;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Type;
+
+/**
+ * We have defined a dedicated Reader for FlowRunEntity,
+ * aimed at adapting to the Jersey2 framework
+ * to ensure that JSON can be converted into FlowRunEntity.
+ */
+@Provider
+@Consumes(MediaType.APPLICATION_JSON)
+public class FlowRunEntityReader implements MessageBodyReader<FlowRunEntity> {
+
+  private ObjectMapper objectMapper = new ObjectMapper();
+
+  @Override
+  public boolean isReadable(Class<?> type, Type genericType,
+      Annotation[] annotations, MediaType mediaType) {
+    return type == FlowRunEntity.class;
+  }
+
+  @Override
+  public FlowRunEntity readFrom(Class<FlowRunEntity> type, Type genericType,
+      Annotation[] annotations, MediaType mediaType, MultivaluedMap<String, String> httpHeaders,
+      InputStream entityStream) throws IOException, WebApplicationException {
+    try {
+      FlowRunEntity timelineEntity = objectMapper.readValue(entityStream, FlowRunEntity.class);
+      return timelineEntity;
+    } catch (Exception e) {
+      return new FlowRunEntity();
+    }
+  }
+}

+ 73 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/FlowRunEntitySetReader.java

@@ -0,0 +1,73 @@
+/**
+ * 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.hadoop.yarn.server.timelineservice.reader;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
+
+import javax.ws.rs.Consumes;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.ext.MessageBodyReader;
+import javax.ws.rs.ext.Provider;
+import java.io.IOException;
+import java.io.InputStream;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Type;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * We have defined a dedicated Reader for `Set<FlowActivityEntity>`,
+ * aimed at adapting to the Jersey2 framework
+ * to ensure that JSON can be converted into `Set<FlowActivityEntity>`.
+ */
+@Provider
+@Consumes(MediaType.APPLICATION_JSON)
+public class FlowRunEntitySetReader implements MessageBodyReader<Set<FlowRunEntity>> {
+
+  private ObjectMapper objectMapper = new ObjectMapper();
+  private String timelineEntityType =
+      "java.util.Set<org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity>";
+
+  @Override
+  public boolean isReadable(Class<?> type, Type genericType,
+      Annotation[] annotations, MediaType mediaType) {
+    return timelineEntityType.equals(genericType.getTypeName());
+  }
+
+  @Override
+  public Set<FlowRunEntity> readFrom(Class<Set<FlowRunEntity>> type,
+      Type genericType, Annotation[] annotations, MediaType mediaType,
+      MultivaluedMap<String, String> httpHeaders,
+      InputStream entityStream) throws IOException, WebApplicationException {
+    Set<FlowRunEntity> flowRunEntitySet = new HashSet<>();
+
+    JsonNode jsonNode = objectMapper.readTree(entityStream);
+    if (jsonNode.isArray()) {
+      for (JsonNode jNode : jsonNode) {
+        FlowRunEntity flowRunEntity = objectMapper.treeToValue(jNode, FlowRunEntity.class);
+        flowRunEntitySet.add(flowRunEntity);
+      }
+    }
+
+    return flowRunEntitySet;
+  }
+}

File diff suppressed because it is too large
+ 148 - 148
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java


+ 122 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityListReader.java

@@ -0,0 +1,122 @@
+/**
+ * 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.hadoop.yarn.server.timelineservice.reader;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity.Identifier;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
+
+import javax.ws.rs.Consumes;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.ext.MessageBodyReader;
+import javax.ws.rs.ext.Provider;
+import java.io.IOException;
+import java.io.InputStream;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Type;
+import java.util.*;
+
+/**
+ * We have defined a dedicated Reader for `Set<TimelineEntity>`,
+ * aimed at adapting to the Jersey2 framework
+ * to ensure that JSON can be converted into `Set<TimelineEntity>`.
+ */
+@Provider
+@Consumes(MediaType.APPLICATION_JSON)
+public class TimelineEntityListReader implements MessageBodyReader<List<TimelineEntity>> {
+
+  private ObjectMapper objectMapper = new ObjectMapper();
+  private String timelineEntityType =
+      "java.util.List<org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity>";
+
+  @Override
+  public boolean isReadable(Class<?> type, Type genericType,
+      Annotation[] annotations, MediaType mediaType) {
+    return timelineEntityType.equals(genericType.getTypeName());
+  }
+
+  @Override
+  public List<TimelineEntity> readFrom(Class<List<TimelineEntity>> type,
+      Type genericType, Annotation[] annotations, MediaType mediaType,
+      MultivaluedMap<String, String> httpHeaders,
+      InputStream entityStream) throws IOException, WebApplicationException {
+    List<TimelineEntity> timelineEntityList = new ArrayList<>();
+
+    JsonNode jsonNode = objectMapper.readTree(entityStream);
+    if (jsonNode.isArray()) {
+      for (JsonNode jNode : jsonNode) {
+        TimelineEntity entity = new TimelineEntity();
+
+        // Get Identifier
+        JsonNode jnIdentifier = jNode.get("identifier");
+        JsonNode jnType = jnIdentifier.get("type");
+        JsonNode jnId = jnIdentifier.get("id");
+        Identifier identifier = new Identifier(jnType.asText(), jnId.asText());
+        entity.setIdentifier(identifier);
+
+        // Get Type
+        JsonNode jnAppType = jNode.get("type");
+        entity.setType(jnAppType.asText());
+
+        // Get Createdtime
+        JsonNode jnCreatedTime = jNode.get("createdtime");
+        entity.setCreatedTime(jnCreatedTime.asLong());
+
+        JsonNode jnMetrics = jNode.get("metrics");
+        Set<TimelineMetric> metricSet = new HashSet<>();
+
+        if (jnMetrics.isArray()) {
+          for (JsonNode metric : jnMetrics) {
+            TimelineMetric timelineMetric = objectMapper.treeToValue(metric, TimelineMetric.class);
+            metricSet.add(timelineMetric);
+            System.out.println(metric);
+          }
+        }
+        entity.setMetrics(metricSet);
+
+        // Get configs
+        JsonNode jnConfigs = jNode.get("configs");
+        if (jnConfigs != null) {
+          Map<String, String> configInfos =
+              objectMapper.treeToValue(jnConfigs, Map.class);
+          entity.setConfigs(configInfos);
+        }
+
+        // Get info
+        JsonNode jnInfos = jNode.get("info");
+        if (jnInfos != null) {
+          Map<String, Object> entityInfos =
+              objectMapper.treeToValue(jnInfos, Map.class);
+          entity.setInfo(entityInfos);
+        }
+
+        // Get idprefix
+        JsonNode jnIdprefix = jNode.get("idprefix");
+        entity.setIdPrefix(jnIdprefix.asLong());
+
+        timelineEntityList.add(entity);
+      }
+    }
+
+    return timelineEntityList;
+  }
+}

+ 125 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntitySetReader.java

@@ -0,0 +1,125 @@
+/**
+ * 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.hadoop.yarn.server.timelineservice.reader;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity.Identifier;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
+
+import javax.ws.rs.Consumes;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.ext.MessageBodyReader;
+import javax.ws.rs.ext.Provider;
+import java.io.IOException;
+import java.io.InputStream;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Type;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * We have defined a dedicated Reader for `Set<TimelineEntity>`,
+ * aimed at adapting to the Jersey2 framework
+ * to ensure that JSON can be converted into `Set<TimelineEntity>`.
+ */
+@Provider
+@Consumes(MediaType.APPLICATION_JSON)
+public class TimelineEntitySetReader implements MessageBodyReader<Set<TimelineEntity>> {
+
+  private ObjectMapper objectMapper = new ObjectMapper();
+  private String timelineEntityType =
+      "java.util.Set<org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity>";
+
+  @Override
+  public boolean isReadable(Class<?> type, Type genericType,
+      Annotation[] annotations, MediaType mediaType) {
+    return timelineEntityType.equals(genericType.getTypeName());
+  }
+
+  @Override
+  public Set<TimelineEntity> readFrom(Class<Set<TimelineEntity>> type,
+      Type genericType, Annotation[] annotations, MediaType mediaType,
+      MultivaluedMap<String, String> httpHeaders,
+      InputStream entityStream) throws IOException, WebApplicationException {
+    Set<TimelineEntity> timelineEntitySet = new HashSet<>();
+
+    JsonNode jsonNode = objectMapper.readTree(entityStream);
+    if (jsonNode.isArray()) {
+      for (JsonNode jNode : jsonNode) {
+        TimelineEntity entity = new TimelineEntity();
+
+        // Get Identifier
+        JsonNode jnIdentifier = jNode.get("identifier");
+        JsonNode jnType = jnIdentifier.get("type");
+        JsonNode jnId = jnIdentifier.get("id");
+        Identifier identifier = new Identifier(jnType.asText(), jnId.asText());
+        entity.setIdentifier(identifier);
+
+        // Get Type
+        JsonNode jnAppType = jNode.get("type");
+        entity.setType(jnAppType.asText());
+
+        // Get Createdtime
+        JsonNode jnCreatedTime = jNode.get("createdtime");
+        entity.setCreatedTime(jnCreatedTime.asLong());
+
+        // Get metrics
+        JsonNode jnMetrics = jNode.get("metrics");
+        Set<TimelineMetric> metricSet = new HashSet<>();
+
+        if (jnMetrics.isArray()) {
+          for (JsonNode metric : jnMetrics) {
+            TimelineMetric timelineMetric = objectMapper.treeToValue(metric, TimelineMetric.class);
+            metricSet.add(timelineMetric);
+          }
+        }
+        entity.setMetrics(metricSet);
+
+        // Get configs
+        JsonNode jnConfigs = jNode.get("configs");
+        if (jnConfigs != null) {
+          Map<String, String> configInfos =
+              objectMapper.treeToValue(jnConfigs, Map.class);
+          entity.setConfigs(configInfos);
+        }
+
+        // Get info
+        JsonNode jnInfos = jNode.get("info");
+        if (jnInfos != null) {
+          Map<String, Object> entityInfos =
+              objectMapper.treeToValue(jnInfos, Map.class);
+          entity.setInfo(entityInfos);
+        }
+
+
+        // Get idprefix
+        JsonNode jnIdprefix = jNode.get("idprefix");
+        entity.setIdPrefix(jnIdprefix.asLong());
+
+        timelineEntitySet.add(entity);
+      }
+    }
+
+    return timelineEntitySet;
+  }
+}

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java

@@ -251,7 +251,7 @@ public class TestHBaseStorageFlowRun {
           new TimelineDataToRetrieve());
       assertTrue(TimelineEntityType.YARN_FLOW_RUN.matches(entity.getType()));
       FlowRunEntity flowRun = (FlowRunEntity)entity;
-      assertEquals(minStartTs, flowRun.getStartTime());
+      assertEquals(minStartTs, flowRun.getStartTime().longValue());
       assertEquals(endTs, flowRun.getMaxEndTime());
     } finally {
       if (hbr != null) {

+ 6 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java

@@ -646,7 +646,9 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
   protected void serviceStop() throws Exception {
     boolean isStorageUp = true;
     try {
-      storageMonitor.checkStorageIsUp();
+      if (storageMonitor != null) {
+        storageMonitor.checkStorageIsUp();
+      }
     } catch (IOException e) {
       LOG.warn("Failed to close the timeline tables as Hbase is down", e);
       isStorageUp = false;
@@ -688,7 +690,9 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
         conn.close();
       }
     }
-    storageMonitor.stop();
+    if (storageMonitor != null) {
+      storageMonitor.stop();
+    }
     super.serviceStop();
   }
 

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml

@@ -29,7 +29,6 @@
 
   <properties>
     <hadoop.common.build.dir>${basedir}/../../../../hadoop-common-project/hadoop-common/target</hadoop.common.build.dir>
-    <hbase-compatible-hadoop.version>3.3.6</hbase-compatible-hadoop.version>
   </properties>
 
   <!-- Do not add dependencies here, add them to the POM of the leaf module -->

Some files were not shown because too many files changed in this diff