瀏覽代碼

YARN-5559. Analyse 2.8.0/3.0.0 jdiff reports and fix any issues. Contributed by Akira Ajisaka & Wangda Tan

(cherry picked from commit 43ebff2e354142bddcb42755766a965ae8a503a6)
Jian He 8 年之前
父節點
當前提交
802a1fb2fc

+ 41 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodeLabelsResponse.java

@@ -18,7 +18,9 @@
 
 
 package org.apache.hadoop.yarn.api.protocolrecords;
 package org.apache.hadoop.yarn.api.protocolrecords;
 
 
+import java.util.ArrayList;
 import java.util.List;
 import java.util.List;
+import java.util.Set;
 
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
@@ -28,18 +30,48 @@ import org.apache.hadoop.yarn.util.Records;
 @Public
 @Public
 @Evolving
 @Evolving
 public abstract class GetClusterNodeLabelsResponse {
 public abstract class GetClusterNodeLabelsResponse {
+  /**
+   * Creates a new instance.
+   *
+   * @param labels Node labels
+   * @return response
+   * @deprecated Use {@link #newInstance(List)} instead.
+   */
+  @Deprecated
+  public static GetClusterNodeLabelsResponse newInstance(Set<String> labels) {
+    List<NodeLabel> list = new ArrayList<>();
+    for (String label : labels) {
+      list.add(NodeLabel.newInstance(label));
+    }
+    return newInstance(list);
+  }
+
   public static GetClusterNodeLabelsResponse newInstance(List<NodeLabel> labels) {
   public static GetClusterNodeLabelsResponse newInstance(List<NodeLabel> labels) {
-    GetClusterNodeLabelsResponse request =
+    GetClusterNodeLabelsResponse response =
         Records.newRecord(GetClusterNodeLabelsResponse.class);
         Records.newRecord(GetClusterNodeLabelsResponse.class);
-    request.setNodeLabels(labels);
-    return request;
+    response.setNodeLabelList(labels);
+    return response;
   }
   }
 
 
-  @Public
-  @Evolving
-  public abstract void setNodeLabels(List<NodeLabel> labels);
+  public abstract void setNodeLabelList(List<NodeLabel> labels);
+
+  public abstract List<NodeLabel> getNodeLabelList();
+
+  /**
+   * Set node labels to the response.
+   *
+   * @param labels Node labels
+   * @deprecated Use {@link #setNodeLabelList(List)} instead.
+   */
+  @Deprecated
+  public abstract void setNodeLabels(Set<String> labels);
 
 
-  @Public
-  @Evolving
-  public abstract List<NodeLabel> getNodeLabels();
+  /**
+   * Get node labels of the response.
+   *
+   * @return Node labels
+   * @deprecated Use {@link #getNodeLabelList()} instead.
+   */
+  @Deprecated
+  public abstract Set<String> getNodeLabels();
 }
 }

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java

@@ -874,7 +874,7 @@ public class YarnClientImpl extends YarnClient {
   @Override
   @Override
   public List<NodeLabel> getClusterNodeLabels() throws YarnException, IOException {
   public List<NodeLabel> getClusterNodeLabels() throws YarnException, IOException {
     return rmClient.getClusterNodeLabels(
     return rmClient.getClusterNodeLabels(
-        GetClusterNodeLabelsRequest.newInstance()).getNodeLabels();
+        GetClusterNodeLabelsRequest.newInstance()).getNodeLabelList();
   }
   }
 
 
   @Override
   @Override

+ 36 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodeLabelsResponsePBImpl.java

@@ -19,7 +19,9 @@
 package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
 package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
 
 
 import java.util.ArrayList;
 import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.List;
 import java.util.List;
+import java.util.Set;
 
 
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
@@ -46,7 +48,7 @@ public class GetClusterNodeLabelsResponsePBImpl extends
     viaProto = true;
     viaProto = true;
   }
   }
 
 
-  public GetClusterNodeLabelsResponseProto getProto() {
+  public synchronized GetClusterNodeLabelsResponseProto getProto() {
     mergeLocalToProto();
     mergeLocalToProto();
     proto = viaProto ? proto : builder.build();
     proto = viaProto ? proto : builder.build();
     viaProto = true;
     viaProto = true;
@@ -101,14 +103,43 @@ public class GetClusterNodeLabelsResponsePBImpl extends
   }
   }
 
 
   @Override
   @Override
-  public void setNodeLabels(List<NodeLabel> updatedNodeLabels) {
+  public synchronized void setNodeLabelList(List<NodeLabel> nodeLabels) {
     maybeInitBuilder();
     maybeInitBuilder();
     this.updatedNodeLabels = new ArrayList<>();
     this.updatedNodeLabels = new ArrayList<>();
-    if (updatedNodeLabels == null) {
+    if (nodeLabels == null) {
       builder.clearNodeLabels();
       builder.clearNodeLabels();
       return;
       return;
     }
     }
-    this.updatedNodeLabels.addAll(updatedNodeLabels);
+    this.updatedNodeLabels.addAll(nodeLabels);
+  }
+
+  /**
+   * @deprecated Use {@link #getNodeLabelList()} instead.
+   */
+  @Override
+  @Deprecated
+  public synchronized Set<String> getNodeLabels() {
+    Set<String> set = new HashSet<>();
+    List<NodeLabel> labelList = getNodeLabelList();
+    if (labelList != null) {
+      for (NodeLabel label : labelList) {
+        set.add(label.getName());
+      }
+    }
+    return set;
+  }
+
+  /**
+   * @deprecated Use {@link #setNodeLabelList(List)} instead.
+   */
+  @Override
+  @Deprecated
+  public void setNodeLabels(Set<String> labels) {
+    List<NodeLabel> list = new ArrayList<>();
+    for (String s : labels) {
+      list.add(NodeLabel.newInstance(s));
+    }
+    setNodeLabelList(list);
   }
   }
 
 
   private void initLocalNodeLabels() {
   private void initLocalNodeLabels() {
@@ -121,7 +152,7 @@ public class GetClusterNodeLabelsResponsePBImpl extends
   }
   }
 
 
   @Override
   @Override
-  public List<NodeLabel> getNodeLabels() {
+  public synchronized List<NodeLabel> getNodeLabelList() {
     if (this.updatedNodeLabels != null) {
     if (this.updatedNodeLabels != null) {
       return this.updatedNodeLabels;
       return this.updatedNodeLabels;
     }
     }

+ 25 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java

@@ -71,6 +71,31 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
         CommonNodeLabelsManager.NO_LABEL, ContainerType.TASK);
         CommonNodeLabelsManager.NO_LABEL, ContainerType.TASK);
   }
   }
 
 
+  /**
+   * Creates a instance.
+   *
+   * @param appSubmitter appSubmitter
+   * @param containerID container ID
+   * @param creationTime creation time
+   * @param expiryTimeStamp expiry timestamp
+   * @param hostName hostname
+   * @param logAggregationContext log aggregation context
+   * @param masterKeyId master key ID
+   * @param priority priority
+   * @param r resource needed by the container
+   * @param rmIdentifier ResourceManager identifier
+   * @deprecated Use one of the other constructors instead.
+   */
+  @Deprecated
+  public ContainerTokenIdentifier(ContainerId containerID, String hostName,
+      String appSubmitter, Resource r, long expiryTimeStamp, int masterKeyId,
+      long rmIdentifier, Priority priority, long creationTime,
+      LogAggregationContext logAggregationContext) {
+    this(containerID, hostName, appSubmitter, r, expiryTimeStamp, masterKeyId,
+        rmIdentifier, priority, creationTime, logAggregationContext,
+        CommonNodeLabelsManager.NO_LABEL);
+  }
+
   public ContainerTokenIdentifier(ContainerId containerID, String hostName,
   public ContainerTokenIdentifier(ContainerId containerID, String hostName,
       String appSubmitter, Resource r, long expiryTimeStamp, int masterKeyId,
       String appSubmitter, Resource r, long expiryTimeStamp, int masterKeyId,
       long rmIdentifier, Priority priority, long creationTime,
       long rmIdentifier, Priority priority, long creationTime,

+ 5 - 17
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitionException.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.yarn.state;
 
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
-import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 
 
 /**
 /**
  * The exception that happens when you call invalid state transition.
  * The exception that happens when you call invalid state transition.
@@ -28,24 +27,13 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
  */
  */
 @Public
 @Public
 @Evolving
 @Evolving
-public class InvalidStateTransitionException extends YarnRuntimeException {
+@SuppressWarnings("deprecation")
+public class InvalidStateTransitionException extends
+    InvalidStateTransitonException {
 
 
-  private static final long serialVersionUID = -6188669113571351684L;
-  private Enum<?> currentState;
-  private Enum<?> event;
+  private static final long serialVersionUID = 8610511635996283691L;
 
 
   public InvalidStateTransitionException(Enum<?> currentState, Enum<?> event) {
   public InvalidStateTransitionException(Enum<?> currentState, Enum<?> event) {
-    super("Invalid event: " + event + " at " + currentState);
-    this.currentState = currentState;
-    this.event = event;
+    super(currentState, event);
   }
   }
-
-  public Enum<?> getCurrentState() {
-    return currentState;
-  }
-
-  public Enum<?> getEvent() {
-    return event;
-  }
-
 }
 }

+ 15 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitonException.java

@@ -20,20 +20,31 @@ package org.apache.hadoop.yarn.state;
 
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 
 
 /** @deprecated Use {@link InvalidStateTransitionException} instead. */
 /** @deprecated Use {@link InvalidStateTransitionException} instead. */
 
 
 @Public
 @Public
 @Evolving
 @Evolving
 @Deprecated
 @Deprecated
-public class InvalidStateTransitonException extends
-    InvalidStateTransitionException {
+public class InvalidStateTransitonException extends YarnRuntimeException {
 
 
-  private static final long serialVersionUID = 8610511635996283691L;
+  private static final long serialVersionUID = -6188669113571351684L;
+  private Enum<?> currentState;
+  private Enum<?> event;
 
 
   public InvalidStateTransitonException(Enum<?> currentState, Enum<?> event) {
   public InvalidStateTransitonException(Enum<?> currentState, Enum<?> event) {
-    super(currentState, event);
+    super("Invalid event: " + event + " at " + currentState);
+    this.currentState = currentState;
+    this.event = event;
   }
   }
 
 
+  public Enum<?> getCurrentState() {
+    return currentState;
+  }
+
+  public Enum<?> getEvent() {
+    return event;
+  }
 
 
 }
 }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java

@@ -1504,7 +1504,7 @@ public class TestClientRMService {
     // Get node labels collection
     // Get node labels collection
     GetClusterNodeLabelsResponse response = client
     GetClusterNodeLabelsResponse response = client
         .getClusterNodeLabels(GetClusterNodeLabelsRequest.newInstance());
         .getClusterNodeLabels(GetClusterNodeLabelsRequest.newInstance());
-    Assert.assertTrue(response.getNodeLabels().containsAll(
+    Assert.assertTrue(response.getNodeLabelList().containsAll(
         Arrays.asList(labelX, labelY)));
         Arrays.asList(labelX, labelY)));
 
 
     // Get node labels mapping
     // Get node labels mapping
@@ -1575,7 +1575,7 @@ public class TestClientRMService {
     // Get node labels collection
     // Get node labels collection
     GetClusterNodeLabelsResponse response = client
     GetClusterNodeLabelsResponse response = client
         .getClusterNodeLabels(GetClusterNodeLabelsRequest.newInstance());
         .getClusterNodeLabels(GetClusterNodeLabelsRequest.newInstance());
-    Assert.assertTrue(response.getNodeLabels().containsAll(
+    Assert.assertTrue(response.getNodeLabelList().containsAll(
         Arrays.asList(labelX, labelY, labelZ)));
         Arrays.asList(labelX, labelY, labelZ)));
 
 
     // Get labels to nodes mapping
     // Get labels to nodes mapping