You are viewing a plain text version of this content. The canonical link for it is here.
Posted to yarn-commits@hadoop.apache.org by vi...@apache.org on 2013/05/20 23:50:48 UTC

svn commit: r1484612 - in /hadoop/common/trunk/hadoop-yarn-project: ./ hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ hadoop-yarn/hadoop-yarn-api/src/main/proto/ hadoop-yarn/hadoop-yarn-common/src/test/java/org/ap...

Author: vinodkv
Date: Mon May 20 21:50:47 2013
New Revision: 1484612

URL: http://svn.apache.org/r1484612
Log:
YARN-695. Remove masterContainer and status unused fields from ApplicationReportProto and fix bugs in ApplicationReportPBImpl. Contributed by Zhijie Shen.

Added:
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicatonReport.java
Modified:
    hadoop/common/trunk/hadoop-yarn-project/CHANGES.txt
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicationAttemptId.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicationId.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerId.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestNodeId.java

Modified: hadoop/common/trunk/hadoop-yarn-project/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/CHANGES.txt?rev=1484612&r1=1484611&r2=1484612&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-yarn-project/CHANGES.txt Mon May 20 21:50:47 2013
@@ -395,6 +395,10 @@ Release 2.0.5-beta - UNRELEASED
     YARN-628. Fix the way YarnRemoteException is being unrolled to extract out
     the underlying exception. (Siddharth Seth via vinodkv)
 
+    YARN-695. Remove masterContainer and status unused fields from
+    ApplicationReportProto and fix bugs in ApplicationReportPBImpl. (Zhijie Shen
+    via vinodkv)
+
 Release 2.0.4-alpha - 2013-04-25 
 
   INCOMPATIBLE CHANGES

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java?rev=1484612&r1=1484611&r2=1484612&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java Mon May 20 21:50:47 2013
@@ -223,15 +223,15 @@ implements ApplicationReport {
   public void setApplicationId(ApplicationId applicationId) {
     maybeInitBuilder();
     if (applicationId == null)
-      builder.clearStatus();
+      builder.clearApplicationId();
     this.applicationId = applicationId;
   }
 
   @Override
   public void setCurrentApplicationAttemptId(ApplicationAttemptId applicationAttemptId) {
     maybeInitBuilder();
-    if (applicationId == null)
-      builder.clearStatus();
+    if (applicationAttemptId == null)
+      builder.clearCurrentApplicationAttemptId();
     this.currentApplicationAttemptId = applicationAttemptId;
   }
 

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto?rev=1484612&r1=1484611&r2=1484612&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto Mon May 20 21:50:47 2013
@@ -152,18 +152,16 @@ message ApplicationReportProto {
   optional string host = 5;
   optional int32 rpc_port = 6;
   optional hadoop.common.TokenProto client_token = 7;
-  optional ApplicationStatusProto status = 8;
-  optional YarnApplicationStateProto yarn_application_state = 9;
-  optional ContainerProto masterContainer = 10;
-  optional string trackingUrl = 11;
-  optional string diagnostics = 12 [default = "N/A"];
-  optional int64 startTime = 13;
-  optional int64 finishTime = 14;
-  optional FinalApplicationStatusProto final_application_status = 15;
-  optional ApplicationResourceUsageReportProto app_resource_Usage = 16;
-  optional string originalTrackingUrl = 17;
-  optional ApplicationAttemptIdProto currentApplicationAttemptId = 18;
-  optional float progress = 19;
+  optional YarnApplicationStateProto yarn_application_state = 8;
+  optional string trackingUrl = 9;
+  optional string diagnostics = 10 [default = "N/A"];
+  optional int64 startTime = 11;
+  optional int64 finishTime = 12;
+  optional FinalApplicationStatusProto final_application_status = 13;
+  optional ApplicationResourceUsageReportProto app_resource_Usage = 14;
+  optional string originalTrackingUrl = 15;
+  optional ApplicationAttemptIdProto currentApplicationAttemptId = 16;
+  optional float progress = 17;
 }
 
 enum NodeStateProto {

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicationAttemptId.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicationAttemptId.java?rev=1484612&r1=1484611&r2=1484612&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicationAttemptId.java (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicationAttemptId.java Mon May 20 21:50:47 2013
@@ -23,7 +23,7 @@ import junit.framework.Assert;
 
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.util.Records;
+import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.junit.Test;
 
 public class TestApplicationAttemptId {
@@ -57,18 +57,12 @@ public class TestApplicationAttemptId {
     Assert.assertEquals("appattempt_" + ts + "_543627_33492611", a6.toString());
   }
 
-  private ApplicationAttemptId createAppAttemptId(long clusterTimeStamp,
-      int id, int attemptId) {
-    ApplicationAttemptId appAttemptId =
-        Records.newRecord(ApplicationAttemptId.class);
-    ApplicationId appId = Records.newRecord(ApplicationId.class);
-    appId.setClusterTimestamp(clusterTimeStamp);
-    appId.setId(id);
-    appAttemptId.setApplicationId(appId);
-    appAttemptId.setAttemptId(attemptId);
-    return appAttemptId;
+  private ApplicationAttemptId createAppAttemptId(
+      long clusterTimeStamp, int id, int attemptId) {
+    ApplicationId appId = BuilderUtils.newApplicationId(clusterTimeStamp, id);
+    return BuilderUtils.newApplicationAttemptId(appId, attemptId);
   }
-  
+
   public static void main(String[] args) throws Exception {
     TestApplicationAttemptId t = new TestApplicationAttemptId();
     t.testApplicationAttemptId();

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicationId.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicationId.java?rev=1484612&r1=1484611&r2=1484612&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicationId.java (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicationId.java Mon May 20 21:50:47 2013
@@ -21,17 +21,17 @@ package org.apache.hadoop.yarn.api;
 import junit.framework.Assert;
 
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.util.Records;
+import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.junit.Test;
 
 public class TestApplicationId {
 
   @Test
   public void testApplicationId() {
-    ApplicationId a1 = createAppId(10l, 1);
-    ApplicationId a2 = createAppId(10l, 2);
-    ApplicationId a3 = createAppId(10l, 1);
-    ApplicationId a4 = createAppId(8l, 3);
+    ApplicationId a1 = BuilderUtils.newApplicationId(10l, 1);
+    ApplicationId a2 = BuilderUtils.newApplicationId(10l, 2);
+    ApplicationId a3 = BuilderUtils.newApplicationId(10l, 1);
+    ApplicationId a4 = BuilderUtils.newApplicationId(8l, 3);
 
     Assert.assertFalse(a1.equals(a2));
     Assert.assertFalse(a1.equals(a4));
@@ -46,15 +46,10 @@ public class TestApplicationId {
     Assert.assertFalse(a2.hashCode() == a4.hashCode());
     
     long ts = System.currentTimeMillis();
-    ApplicationId a5 = createAppId(ts, 45436343);
+    ApplicationId a5 =
+        BuilderUtils.newApplicationId(ts, 45436343);
     Assert.assertEquals("application_10_0001", a1.toString());
     Assert.assertEquals("application_" + ts + "_45436343", a5.toString());
   }
 
-  private ApplicationId createAppId(long clusterTimeStamp, int id) {
-    ApplicationId appId = Records.newRecord(ApplicationId.class);
-    appId.setClusterTimestamp(clusterTimeStamp);
-    appId.setId(id);
-    return appId;
-  }
 }
\ No newline at end of file

Added: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicatonReport.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicatonReport.java?rev=1484612&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicatonReport.java (added)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicatonReport.java Mon May 20 21:50:47 2013
@@ -0,0 +1,64 @@
+/**
+ * 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;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.util.BuilderUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestApplicatonReport {
+
+  @Test
+  public void testApplicationReport() {
+    long timestamp = System.currentTimeMillis();
+    ApplicationReport appReport1 =
+        createApplicationReport(1, 1, timestamp);
+    ApplicationReport appReport2 =
+        createApplicationReport(1, 1, timestamp);
+    ApplicationReport appReport3 =
+        createApplicationReport(1, 1, timestamp);
+    Assert.assertEquals(appReport1, appReport2);
+    Assert.assertEquals(appReport2, appReport3);
+    appReport1.setApplicationId(null);
+    Assert.assertNull(appReport1.getApplicationId());
+    Assert.assertNotSame(appReport1, appReport2);
+    appReport2.setCurrentApplicationAttemptId(null);
+    Assert.assertNull(appReport2.getCurrentApplicationAttemptId());
+    Assert.assertNotSame(appReport2, appReport3);
+  }
+
+  protected static ApplicationReport createApplicationReport(
+      int appIdInt, int appAttemptIdInt, long timestamp) {
+    ApplicationId appId = BuilderUtils.newApplicationId(timestamp, appIdInt);
+    ApplicationAttemptId appAttemptId =
+        BuilderUtils.newApplicationAttemptId(appId, appAttemptIdInt);
+    ApplicationReport appReport = BuilderUtils.newApplicationReport(
+        appId, appAttemptId, "user", "queue", "appname", "host", 124, null,
+        YarnApplicationState.FINISHED, "diagnostics", "url", 0, 0,
+        FinalApplicationStatus.SUCCEEDED, null, "N/A", 0.53789f);
+    return appReport;
+  }
+
+}
+

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerId.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerId.java?rev=1484612&r1=1484611&r2=1484612&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerId.java (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerId.java Mon May 20 21:50:47 2013
@@ -21,21 +21,19 @@ package org.apache.hadoop.yarn.api;
 
 import junit.framework.Assert;
 
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.util.Records;
+import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.junit.Test;
 
 public class TestContainerId {
 
   @Test
   public void testContainerId() {
-    ContainerId c1 = createContainerId(10l, 1, 1, 1);
-    ContainerId c2 = createContainerId(10l, 1, 1, 2);
-    ContainerId c3 = createContainerId(10l, 1, 1, 1);
-    ContainerId c4 = createContainerId(10l, 1, 3, 1);
-    ContainerId c5 = createContainerId(8l, 1, 3, 1);
+    ContainerId c1 = BuilderUtils.newContainerId(1, 1, 10l, 1);
+    ContainerId c2 = BuilderUtils.newContainerId(1, 1, 10l, 2);
+    ContainerId c3 = BuilderUtils.newContainerId(1, 1, 10l, 1);
+    ContainerId c4 = BuilderUtils.newContainerId(1, 3, 10l, 1);
+    ContainerId c5 = BuilderUtils.newContainerId(1, 3, 8l, 1);
 
     Assert.assertTrue(c1.equals(c3));
     Assert.assertFalse(c1.equals(c2));
@@ -53,36 +51,10 @@ public class TestContainerId {
     Assert.assertFalse(c1.hashCode() == c5.hashCode());
     
     long ts = System.currentTimeMillis();
-    ContainerId c6 = createContainerId(ts, 36473, 4365472, 25645811);
+    ContainerId c6 = BuilderUtils.newContainerId(36473, 4365472, ts, 25645811);
     Assert.assertEquals("container_10_0001_01_000001", c1.toString());
     Assert.assertEquals("container_" + ts + "_36473_4365472_25645811",
         c6.toString());
   }
 
-  private ContainerId createContainerId(long clusterTimestamp, int appIdInt,
-      int appAttemptIdInt, int containerIdInt) {
-    ApplicationId appId = createAppId(clusterTimestamp, appIdInt);
-    ApplicationAttemptId appAttemptId =
-        createAppAttemptId(appId, appAttemptIdInt);
-    ContainerId containerId = Records.newRecord(ContainerId.class);
-    containerId.setApplicationAttemptId(appAttemptId);
-    containerId.setId(containerIdInt);
-    return containerId;
-  }
-
-  private ApplicationId createAppId(long clusterTimeStamp, int id) {
-    ApplicationId appId = Records.newRecord(ApplicationId.class);
-    appId.setClusterTimestamp(clusterTimeStamp);
-    appId.setId(id);
-    return appId;
-  }
-
-  private ApplicationAttemptId createAppAttemptId(ApplicationId appId,
-      int attemptId) {
-    ApplicationAttemptId appAttemptId =
-        Records.newRecord(ApplicationAttemptId.class);
-    appAttemptId.setApplicationId(appId);
-    appAttemptId.setAttemptId(attemptId);
-    return appAttemptId;
-  }
 }

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestNodeId.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestNodeId.java?rev=1484612&r1=1484611&r2=1484612&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestNodeId.java (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestNodeId.java Mon May 20 21:50:47 2013
@@ -21,16 +21,16 @@ package org.apache.hadoop.yarn.api;
 import junit.framework.Assert;
 
 import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.util.Records;
+import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.junit.Test;
 
 public class TestNodeId {
   @Test
   public void testNodeId() {
-    NodeId nodeId1 = createNodeId("10.18.52.124", 8041);
-    NodeId nodeId2 = createNodeId("10.18.52.125", 8038);
-    NodeId nodeId3 = createNodeId("10.18.52.124", 8041);
-    NodeId nodeId4 = createNodeId("10.18.52.124", 8039);
+    NodeId nodeId1 = BuilderUtils.newNodeId("10.18.52.124", 8041);
+    NodeId nodeId2 = BuilderUtils.newNodeId("10.18.52.125", 8038);
+    NodeId nodeId3 = BuilderUtils.newNodeId("10.18.52.124", 8041);
+    NodeId nodeId4 = BuilderUtils.newNodeId("10.18.52.124", 8039);
 
     Assert.assertTrue(nodeId1.equals(nodeId3));
     Assert.assertFalse(nodeId1.equals(nodeId2));
@@ -47,10 +47,4 @@ public class TestNodeId {
     Assert.assertEquals("10.18.52.124:8041", nodeId1.toString());
   }
 
-  private NodeId createNodeId(String host, int port) {
-    NodeId nodeId = Records.newRecord(NodeId.class);
-    nodeId.setHost(host);
-    nodeId.setPort(port);
-    return nodeId;
-  }
 }