Bläddra i källkod

YARN-978. Updated patch for adding ApplicationAttemptReport and Protobuf implementation. Contributed by Mayank Bansal.
svn merge --ignore-ancestry -c 1556729 ../YARN-321


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1562186 13f79535-47bb-0310-9956-ffa450edef68

Vinod Kumar Vavilapalli 11 år sedan
förälder
incheckning
cb52671a9b

+ 165 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptReport.java

@@ -0,0 +1,165 @@
+/**
+ * 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;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * <code>ApplicationAttemptReport</code> is a report of an application attempt.
+ * </p>
+ *
+ * <p>
+ * It includes details such as:
+ * <ul>
+ * <li>{@link ApplicationAttemptId} of the application.</li>
+ * <li>Host on which the <code>ApplicationMaster</code> of this attempt is
+ * running.</li>
+ * <li>RPC port of the <code>ApplicationMaster</code> of this attempt.</li>
+ * <li>Tracking URL.</li>
+ * <li>Diagnostic information in case of errors.</li>
+ * <li>{@link YarnApplicationAttemptState} of the application attempt.</li>
+ * <li>{@link ContainerId} of the master Container.</li>
+ * </ul>
+ * </p>
+ *
+ */
+@Public
+@Unstable
+public abstract class ApplicationAttemptReport {
+
+  @Private
+  @Unstable
+  public static ApplicationAttemptReport newInstance(
+      ApplicationAttemptId applicationAttemptId, String host, int rpcPort,
+      String url, String diagnostics, YarnApplicationAttemptState state,
+      ContainerId amContainerId) {
+    ApplicationAttemptReport report =
+        Records.newRecord(ApplicationAttemptReport.class);
+    report.setApplicationAttemptId(applicationAttemptId);
+    report.setHost(host);
+    report.setRpcPort(rpcPort);
+    report.setTrackingUrl(url);
+    report.setDiagnostics(diagnostics);
+    report.setYarnApplicationAttemptState(state);
+    report.setAMContainerId(amContainerId);
+    return report;
+  }
+
+  /**
+   * Get the <em>YarnApplicationAttemptState</em> of the application attempt.
+   *
+   * @return <em>YarnApplicationAttemptState</em> of the application attempt
+   */
+  @Public
+  @Unstable
+  public abstract YarnApplicationAttemptState getYarnApplicationAttemptState();
+
+  @Private
+  @Unstable
+  public abstract void setYarnApplicationAttemptState(
+      YarnApplicationAttemptState yarnApplicationAttemptState);
+
+  /**
+   * Get the <em>RPC port</em> of this attempt <code>ApplicationMaster</code>.
+   *
+   * @return <em>RPC port</em> of this attempt <code>ApplicationMaster</code>
+   */
+  @Public
+  @Unstable
+  public abstract int getRpcPort();
+
+  @Private
+  @Unstable
+  public abstract void setRpcPort(int rpcPort);
+
+  /**
+   * Get the <em>host</em> on which this attempt of
+   * <code>ApplicationMaster</code> is running.
+   *
+   * @return <em>host</em> on which this attempt of
+   *         <code>ApplicationMaster</code> is running
+   */
+  @Public
+  @Unstable
+  public abstract String getHost();
+
+  @Private
+  @Unstable
+  public abstract void setHost(String host);
+
+  /**
+   * Get the <em>diagnositic information</em> of the application attempt in case
+   * of errors.
+   *
+   * @return <em>diagnositic information</em> of the application attempt in case
+   *         of errors
+   */
+  @Public
+  @Unstable
+  public abstract String getDiagnostics();
+
+  @Private
+  @Unstable
+  public abstract void setDiagnostics(String diagnostics);
+
+  /**
+   * Get the <em>tracking url</em> for the application attempt.
+   *
+   * @return <em>tracking url</em> for the application attempt
+   */
+  @Public
+  @Unstable
+  public abstract String getTrackingUrl();
+
+  @Private
+  @Unstable
+  public abstract void setTrackingUrl(String url);
+
+  /**
+   * Get the <code>ApplicationAttemptId</code> of this attempt of the
+   * application
+   *
+   * @return <code>ApplicationAttemptId</code> of the attempt
+   */
+  @Public
+  @Unstable
+  public abstract ApplicationAttemptId getApplicationAttemptId();
+
+  @Private
+  @Unstable
+  public abstract void setApplicationAttemptId(
+      ApplicationAttemptId applicationAttemptId);
+
+  /**
+   * Get the <code>ContainerId</code> of AMContainer for this attempt
+   *
+   * @return <code>ContainerId</code> of the attempt
+   */
+  @Public
+  @Unstable
+  public abstract ContainerId getAMContainerId();
+
+  @Private
+  @Unstable
+  public abstract void setAMContainerId(ContainerId amContainerId);
+}

+ 10 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto

@@ -192,6 +192,16 @@ message ApplicationReportProto {
   optional hadoop.common.TokenProto am_rm_token = 19;
 }
 
+message ApplicationAttemptReportProto {
+  optional ApplicationAttemptIdProto application_attempt_id = 1;
+  optional string host = 2;
+  optional int32 rpc_port = 3;
+  optional string tracking_url = 4;
+  optional string diagnostics = 5 [default = "N/A"];
+  optional YarnApplicationAttemptStateProto yarn_application_attempt_state = 6;
+  optional ContainerIdProto am_container_id = 7;
+}
+
 enum NodeStateProto {
   NS_NEW = 1; 
   NS_RUNNING = 2; 

+ 270 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationAttemptReportPBImpl.java

@@ -0,0 +1,270 @@
+/**
+ * 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.impl.pb;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
+import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptReportProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptReportProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationAttemptStateProto;
+
+import com.google.protobuf.TextFormat;
+
+public class ApplicationAttemptReportPBImpl extends ApplicationAttemptReport {
+  ApplicationAttemptReportProto proto =
+      ApplicationAttemptReportProto.getDefaultInstance();
+  ApplicationAttemptReportProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private ApplicationAttemptId ApplicationAttemptId;
+  private ContainerId amContainerId;
+
+  public ApplicationAttemptReportPBImpl() {
+    builder = ApplicationAttemptReportProto.newBuilder();
+  }
+
+  public ApplicationAttemptReportPBImpl(ApplicationAttemptReportProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  @Override
+  public ApplicationAttemptId getApplicationAttemptId() {
+    if (this.ApplicationAttemptId != null) {
+      return this.ApplicationAttemptId;
+    }
+
+    ApplicationAttemptReportProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasApplicationAttemptId()) {
+      return null;
+    }
+    this.ApplicationAttemptId =
+        convertFromProtoFormat(p.getApplicationAttemptId());
+    return this.ApplicationAttemptId;
+  }
+
+  @Override
+  public String getHost() {
+    ApplicationAttemptReportProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasHost()) {
+      return null;
+    }
+    return p.getHost();
+  }
+
+  @Override
+  public int getRpcPort() {
+    ApplicationAttemptReportProtoOrBuilder p = viaProto ? proto : builder;
+    return p.getRpcPort();
+  }
+
+  @Override
+  public String getTrackingUrl() {
+    ApplicationAttemptReportProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasTrackingUrl()) {
+      return null;
+    }
+    return p.getTrackingUrl();
+  }
+
+  @Override
+  public String getDiagnostics() {
+    ApplicationAttemptReportProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasDiagnostics()) {
+      return null;
+    }
+    return p.getDiagnostics();
+  }
+
+  @Override
+  public YarnApplicationAttemptState getYarnApplicationAttemptState() {
+    ApplicationAttemptReportProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasYarnApplicationAttemptState()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getYarnApplicationAttemptState());
+  }
+
+  @Override
+  public void setYarnApplicationAttemptState(YarnApplicationAttemptState state) {
+    maybeInitBuilder();
+    if (state == null) {
+      builder.clearYarnApplicationAttemptState();
+      return;
+    }
+    builder.setYarnApplicationAttemptState(convertToProtoFormat(state));
+  }
+
+  private YarnApplicationAttemptStateProto convertToProtoFormat(
+      YarnApplicationAttemptState state) {
+    return ProtoUtils.convertToProtoFormat(state);
+  }
+
+  private YarnApplicationAttemptState convertFromProtoFormat(
+      YarnApplicationAttemptStateProto yarnApplicationAttemptState) {
+    return ProtoUtils.convertFromProtoFormat(yarnApplicationAttemptState);
+  }
+
+  @Override
+  public void setApplicationAttemptId(ApplicationAttemptId applicationAttemptId) {
+    maybeInitBuilder();
+    if (applicationAttemptId == null)
+      builder.clearApplicationAttemptId();
+    this.ApplicationAttemptId = applicationAttemptId;
+  }
+
+  @Override
+  public void setHost(String host) {
+    maybeInitBuilder();
+    if (host == null) {
+      builder.clearHost();
+      return;
+    }
+    builder.setHost(host);
+  }
+
+  @Override
+  public void setRpcPort(int rpcPort) {
+    maybeInitBuilder();
+    builder.setRpcPort(rpcPort);
+  }
+
+  @Override
+  public  void setTrackingUrl(String url) {
+    maybeInitBuilder();
+    if (url == null) {
+      builder.clearTrackingUrl();
+      return;
+    }
+    builder.setTrackingUrl(url);
+  }
+
+  @Override
+  public void setDiagnostics(String diagnostics) {
+    maybeInitBuilder();
+    if (diagnostics == null) {
+      builder.clearDiagnostics();
+      return;
+    }
+    builder.setDiagnostics(diagnostics);
+  }
+
+  public ApplicationAttemptReportProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ApplicationAttemptReportProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto)
+      maybeInitBuilder();
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.ApplicationAttemptId != null &&
+        !((ApplicationAttemptIdPBImpl) this.ApplicationAttemptId)
+        .getProto().equals(builder.getApplicationAttemptId())) {
+      builder.setApplicationAttemptId(
+          convertToProtoFormat(this.ApplicationAttemptId));
+    }
+
+    if (this.amContainerId != null
+        && !((ContainerIdPBImpl) this.amContainerId).getProto().equals(
+            builder.getAmContainerId())) {
+      builder.setAmContainerId(convertToProtoFormat(this.amContainerId));
+    }
+  }
+
+  private ContainerIdProto convertToProtoFormat(ContainerId amContainerId) {
+    return ((ContainerIdPBImpl)amContainerId).getProto();
+  }
+
+  private ContainerIdPBImpl convertFromProtoFormat(
+      ContainerIdProto amContainerId) {
+    return new ContainerIdPBImpl(amContainerId);
+  }
+
+  private ApplicationAttemptIdProto
+      convertToProtoFormat(ApplicationAttemptId t) {
+    return ((ApplicationAttemptIdPBImpl) t).getProto();
+  }
+
+  private ApplicationAttemptIdPBImpl convertFromProtoFormat(
+      ApplicationAttemptIdProto applicationAttemptId) {
+    return new ApplicationAttemptIdPBImpl(applicationAttemptId);
+  }
+
+  @Override
+  public ContainerId getAMContainerId() {
+    if (this.amContainerId != null) {
+      return this.amContainerId;
+    }
+
+    ApplicationAttemptReportProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasAmContainerId()) {
+      return null;
+    }
+    this.amContainerId =
+        convertFromProtoFormat(p.getAmContainerId());
+    return this.amContainerId;
+  }
+
+  @Override
+  public void setAMContainerId(ContainerId amContainerId) {
+    maybeInitBuilder();
+    if (amContainerId == null)
+      builder.clearAmContainerId();
+    this.amContainerId = amContainerId;
+  }
+}