Browse Source

YARN-9624. Use switch case for ProtoUtils#convertFromProtoFormat containerState. Contributed by Bilwa S T

Ayush Saxena 5 years ago
parent
commit
3f0a7cd17a

+ 59 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java

@@ -113,27 +113,74 @@ public class ProtoUtils {
   /*
    * ContainerState
    */
-  private final static String CONTAINER_STATE_PREFIX = "C_";
-  public static ContainerStateProto convertToProtoFormat(ContainerState e) {
-    return ContainerStateProto.valueOf(CONTAINER_STATE_PREFIX + e.name());
+  public static ContainerStateProto convertToProtoFormat(ContainerState state) {
+    switch (state) {
+    case NEW:
+      return ContainerStateProto.C_NEW;
+    case RUNNING:
+      return ContainerStateProto.C_RUNNING;
+    case COMPLETE:
+      return ContainerStateProto.C_COMPLETE;
+    default:
+      throw new IllegalArgumentException(
+          "ContainerState conversion unsupported");
+    }
   }
-  public static ContainerState convertFromProtoFormat(ContainerStateProto e) {
-    return ContainerState.valueOf(e.name().replace(CONTAINER_STATE_PREFIX, ""));
+
+  public static ContainerState convertFromProtoFormat(
+      ContainerStateProto proto) {
+    switch (proto) {
+    case C_NEW:
+      return ContainerState.NEW;
+    case C_RUNNING:
+      return ContainerState.RUNNING;
+    case C_COMPLETE:
+      return ContainerState.COMPLETE;
+    default:
+      throw new IllegalArgumentException(
+          "ContainerStateProto conversion unsupported");
+    }
   }
 
   /*
    * Container SubState
    */
-  private final static String CONTAINER_SUB_STATE_PREFIX = "CSS_";
   public static ContainerSubStateProto convertToProtoFormat(
-      ContainerSubState e) {
-    return ContainerSubStateProto.valueOf(
-        CONTAINER_SUB_STATE_PREFIX + e.name());
+      ContainerSubState state) {
+    switch (state) {
+    case SCHEDULED:
+      return ContainerSubStateProto.CSS_SCHEDULED;
+    case RUNNING:
+      return ContainerSubStateProto.CSS_RUNNING;
+    case PAUSED:
+      return ContainerSubStateProto.CSS_PAUSED;
+    case COMPLETING:
+      return ContainerSubStateProto.CSS_COMPLETING;
+    case DONE:
+      return ContainerSubStateProto.CSS_DONE;
+    default:
+      throw new IllegalArgumentException(
+          "ContainerSubState conversion unsupported");
+    }
   }
+
   public static ContainerSubState convertFromProtoFormat(
-      ContainerSubStateProto e) {
-    return ContainerSubState.valueOf(
-        e.name().substring(CONTAINER_SUB_STATE_PREFIX.length()));
+      ContainerSubStateProto proto) {
+    switch (proto) {
+    case CSS_SCHEDULED:
+      return ContainerSubState.SCHEDULED;
+    case CSS_RUNNING:
+      return ContainerSubState.RUNNING;
+    case CSS_PAUSED:
+      return ContainerSubState.PAUSED;
+    case CSS_COMPLETING:
+      return ContainerSubState.COMPLETING;
+    case CSS_DONE:
+      return ContainerSubState.DONE;
+    default:
+      throw new IllegalArgumentException(
+          "ContainerSubStateProto conversion unsupported");
+    }
   }
   /*
    * NodeState

+ 48 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/impl/pb/TestProtoUtils.java

@@ -0,0 +1,48 @@
+/**
+ * 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 static org.junit.Assert.*;
+
+import java.util.stream.Stream;
+
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerSubState;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStateProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerSubStateProto;
+import org.junit.Test;
+
+public class TestProtoUtils {
+
+  @Test
+  public void testConvertFromOrToProtoFormat() {
+    // Check if utility has all enum values
+    try {
+      Stream.of(ContainerState.values())
+          .forEach(a -> ProtoUtils.convertToProtoFormat(a));
+      Stream.of(ContainerSubState.values())
+          .forEach(a -> ProtoUtils.convertToProtoFormat(a));
+      Stream.of(ContainerSubStateProto.values())
+          .forEach(a -> ProtoUtils.convertFromProtoFormat(a));
+      Stream.of(ContainerStateProto.values())
+          .forEach(a -> ProtoUtils.convertFromProtoFormat(a));
+    } catch (IllegalArgumentException ex) {
+      fail(ex.getMessage());
+    }
+  }
+}