Browse Source

YARN-5229. Refactor #isApplicationEntity and #getApplicationEvent from HBaseTimelineWriterImpl. (Vrushali C via gtcarrera9)

Li Lu 9 years ago
parent
commit
e72e6260da

+ 28 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/ApplicationEntity.java

@@ -49,4 +49,32 @@ public class ApplicationEntity extends HierarchicalTimelineEntity {
   public void setQueue(String queue) {
     addInfo(QUEUE_INFO_KEY, queue);
   }
+
+  /**
+   * Checks if the input TimelineEntity object is an ApplicationEntity.
+   *
+   * @param te TimelineEntity object.
+   * @return true if input is an ApplicationEntity, false otherwise
+   */
+  public static boolean isApplicationEntity(TimelineEntity te) {
+    return (te == null ? false
+        : te.getType().equals(TimelineEntityType.YARN_APPLICATION.toString()));
+  }
+
+  /**
+   * @param te TimelineEntity object.
+   * @param eventId event with this id needs to be fetched
+   * @return TimelineEvent if TimelineEntity contains the desired event.
+   */
+  public static TimelineEvent getApplicationEvent(TimelineEntity te,
+      String eventId) {
+    if (isApplicationEntity(te)) {
+      for (TimelineEvent event : te.getEvents()) {
+        if (event.getId().equals(eventId)) {
+          return event;
+        }
+      }
+    }
+    return null;
+  }
 }

+ 71 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestApplicationEntity.java

@@ -0,0 +1,71 @@
+/**
+ * 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.api.records.timelineservice;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
+
+/**
+ * Various tests for the ApplicationEntity class.
+ *
+ */
+public class TestApplicationEntity {
+
+  @Test
+  public void testIsApplicationEntity() {
+    TimelineEntity te = new TimelineEntity();
+    te.setType(TimelineEntityType.YARN_APPLICATION.toString());
+    assertTrue(ApplicationEntity.isApplicationEntity(te));
+
+    te = null;
+    assertEquals(false, ApplicationEntity.isApplicationEntity(te));
+
+    te = new TimelineEntity();
+    te.setType(TimelineEntityType.YARN_CLUSTER.toString());
+    assertEquals(false, ApplicationEntity.isApplicationEntity(te));
+  }
+
+  @Test
+  public void testGetApplicationEvent() {
+    TimelineEntity te = null;
+    TimelineEvent tEvent = ApplicationEntity.getApplicationEvent(te,
+        "no event");
+    assertEquals(null, tEvent);
+
+    te = new TimelineEntity();
+    te.setType(TimelineEntityType.YARN_APPLICATION.toString());
+    TimelineEvent event = new TimelineEvent();
+    event.setId("start_event");
+    event.setTimestamp(System.currentTimeMillis());
+    te.addEvent(event);
+    tEvent = ApplicationEntity.getApplicationEvent(te, "start_event");
+    assertEquals(event, tEvent);
+
+    te = new TimelineEntity();
+    te.setType(TimelineEntityType.YARN_CLUSTER.toString());
+    event = new TimelineEvent();
+    event.setId("start_event_cluster");
+    event.setTimestamp(System.currentTimeMillis());
+    te.addEvent(event);
+    tEvent = ApplicationEntity.getApplicationEvent(te, "start_event_cluster");
+    assertEquals(null, tEvent);
+
+  }
+}

+ 4 - 31
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java

@@ -28,9 +28,9 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.service.AbstractService;
+import  org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse;
@@ -144,7 +144,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
 
       // if the entity is the application, the destination is the application
       // table
-      boolean isApplication = isApplicationEntity(te);
+      boolean isApplication = ApplicationEntity.isApplicationEntity(te);
       byte[] rowKey;
       if (isApplication) {
         ApplicationRowKey applicationRowKey =
@@ -166,7 +166,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
 
       if (isApplication) {
         TimelineEvent event =
-            getApplicationEvent(te,
+            ApplicationEntity.getApplicationEvent(te,
                 ApplicationMetricsConstants.CREATED_EVENT_TYPE);
         FlowRunRowKey flowRunRowKey =
             new FlowRunRowKey(clusterId, userId, flowName, flowRunId);
@@ -180,7 +180,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
         storeFlowMetricsAppRunning(flowRunRowKey, appId, te);
         // if application has finished, store it's finish time and write final
         // values of all metrics
-        event = getApplicationEvent(te,
+        event = ApplicationEntity.getApplicationEvent(te,
             ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
         if (event != null) {
           onApplicationFinished(flowRunRowKey, flowVersion, appId, te,
@@ -466,33 +466,6 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
     }
   }
 
-  /**
-   * Checks if the input TimelineEntity object is an ApplicationEntity.
-   *
-   * @param te TimelineEntity object.
-   * @return true if input is an ApplicationEntity, false otherwise
-   */
-  static boolean isApplicationEntity(TimelineEntity te) {
-    return te.getType().equals(TimelineEntityType.YARN_APPLICATION.toString());
-  }
-
-  /**
-   * @param te TimelineEntity object.
-   * @param eventId event with this id needs to be fetched
-   * @return TimelineEvent if TimelineEntity contains the desired event.
-   */
-  private static TimelineEvent getApplicationEvent(TimelineEntity te,
-      String eventId) {
-    if (isApplicationEntity(te)) {
-      for (TimelineEvent event : te.getEvents()) {
-        if (event.getId().equals(eventId)) {
-          return event;
-        }
-      }
-    }
-    return null;
-  }
-
   /*
    * (non-Javadoc)
    *