You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by oz...@apache.org on 2015/05/26 12:08:30 UTC

hadoop git commit: YARN-2336. Fair scheduler's REST API returns a missing '[' bracket JSON for deep queue tree. Contributed by Kenji Kikushima and Akira Ajisaka.

Repository: hadoop
Updated Branches:
  refs/heads/trunk 56996a685 -> 9a3d617b6


YARN-2336. Fair scheduler's REST API returns a missing '[' bracket JSON for deep queue tree. Contributed by Kenji Kikushima and Akira Ajisaka.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/9a3d617b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9a3d617b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9a3d617b

Branch: refs/heads/trunk
Commit: 9a3d617b6325d8918f2833c3e9ce329ecada9242
Parents: 56996a6
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Tue May 26 19:07:40 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Tue May 26 19:07:40 2015 +0900

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../webapp/JAXBContextResolver.java             |   3 +-
 .../webapp/dao/FairSchedulerQueueInfo.java      |  31 ++-
 .../webapp/dao/FairSchedulerQueueInfoList.java  |  49 ++++
 .../webapp/TestRMWebServicesCapacitySched.java  |  10 +
 .../webapp/TestRMWebServicesFairScheduler.java  |  35 +++
 .../src/site/markdown/ResourceManagerRest.md    | 248 ++++++++++---------
 7 files changed, 249 insertions(+), 130 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a3d617b/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index ae4a9d7..adb685d 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -52,6 +52,9 @@ Release 2.8.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
 
+    YARN-2336. Fair scheduler's REST API returns a missing '[' bracket JSON for
+    deep queue tree. (Kenji Kikushima and Akira Ajisaka via ozawa)
+
   NEW FEATURES
 
     YARN-3345. Add non-exclusive node label API. (Wangda Tan via jianhe)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a3d617b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java
index 5f5e1ef..7d7a347 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java
@@ -53,7 +53,8 @@ public class JAXBContextResolver implements ContextResolver<JAXBContext> {
             NodesInfo.class, RemoteExceptionData.class,
             CapacitySchedulerQueueInfoList.class, ResourceInfo.class,
             UsersInfo.class, UserInfo.class, ApplicationStatisticsInfo.class,
-            StatisticsItemInfo.class, CapacitySchedulerHealthInfo.class };
+            StatisticsItemInfo.class, CapacitySchedulerHealthInfo.class,
+            FairSchedulerQueueInfoList.class};
     // these dao classes need root unwrapping
     final Class[] rootUnwrappedTypes =
         { NewApplication.class, ApplicationSubmissionContextInfo.class,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a3d617b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerQueueInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerQueueInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerQueueInfo.java
index 5fbfe51..9b297a2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerQueueInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerQueueInfo.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
 
 
-import java.util.ArrayList;
 import java.util.Collection;
 
 import javax.xml.bind.annotation.XmlAccessType;
@@ -60,9 +59,9 @@ public class FairSchedulerQueueInfo {
   
   private String queueName;
   private String schedulingPolicy;
-  
-  private Collection<FairSchedulerQueueInfo> childQueues;
-  
+
+  private FairSchedulerQueueInfoList childQueues;
+
   public FairSchedulerQueueInfo() {
   }
   
@@ -95,20 +94,34 @@ public class FairSchedulerQueueInfo {
     
     maxApps = allocConf.getQueueMaxApps(queueName);
 
-    childQueues = new ArrayList<FairSchedulerQueueInfo>();
     if (allocConf.isReservable(queueName) &&
         !allocConf.getShowReservationAsQueues(queueName)) {
       return;
     }
 
+    childQueues = getChildQueues(queue, scheduler);
+  }
+
+  protected FairSchedulerQueueInfoList getChildQueues(FSQueue queue,
+                                                      FairScheduler scheduler) {
+    // Return null to omit 'childQueues' field from the return value of
+    // REST API if it is empty. We omit the field to keep the consistency
+    // with CapacitySchedulerQueueInfo, which omits 'queues' field if empty.
     Collection<FSQueue> children = queue.getChildQueues();
+    if (children.isEmpty()) {
+      return null;
+    }
+    FairSchedulerQueueInfoList list = new FairSchedulerQueueInfoList();
     for (FSQueue child : children) {
       if (child instanceof FSLeafQueue) {
-        childQueues.add(new FairSchedulerLeafQueueInfo((FSLeafQueue)child, scheduler));
+        list.addToQueueInfoList(
+            new FairSchedulerLeafQueueInfo((FSLeafQueue) child, scheduler));
       } else {
-        childQueues.add(new FairSchedulerQueueInfo(child, scheduler));
+        list.addToQueueInfoList(
+            new FairSchedulerQueueInfo(child, scheduler));
       }
     }
+    return list;
   }
   
   /**
@@ -189,8 +202,8 @@ public class FairSchedulerQueueInfo {
   public String getSchedulingPolicy() {
     return schedulingPolicy;
   }
-  
+
   public Collection<FairSchedulerQueueInfo> getChildQueues() {
-    return childQueues;
+    return childQueues.getQueueInfoList();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a3d617b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerQueueInfoList.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerQueueInfoList.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerQueueInfoList.java
new file mode 100644
index 0000000..9c088c4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerQueueInfoList.java
@@ -0,0 +1,49 @@
+/**
+ * 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.server.resourcemanager.webapp.dao;
+
+import java.util.ArrayList;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+/**
+ * FairScheduler QueueInfo list used for mapping to XML or JSON.
+ */
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+public class FairSchedulerQueueInfoList {
+  private ArrayList<FairSchedulerQueueInfo> queue;
+
+  public FairSchedulerQueueInfoList() {
+    queue = new ArrayList<>();
+  }
+
+  public ArrayList<FairSchedulerQueueInfo> getQueueInfoList() {
+    return this.queue;
+  }
+
+  public boolean addToQueueInfoList(FairSchedulerQueueInfo e) {
+    return this.queue.add(e);
+  }
+
+  public FairSchedulerQueueInfo getQueueInfo(int i) {
+    return this.queue.get(i);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a3d617b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
index e028d31..456595d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.StringReader;
 
@@ -575,6 +576,15 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase {
         user.getInt("numPendingApplications");
         checkResourcesUsed(user);
       }
+
+      // Verify 'queues' field is omitted from CapacitySchedulerLeafQueueInfo.
+      try {
+        b1.getJSONObject("queues");
+        fail("CapacitySchedulerQueueInfo should omit field 'queues'" +
+             "if child queue is empty.");
+      } catch (JSONException je) {
+        assertEquals("JSONObject[\"queues\"] not found.", je.getMessage());
+      }
     } finally {
       rm.stop();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a3d617b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesFairScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesFairScheduler.java
index 21ca6a7..1aed3a6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesFairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesFairScheduler.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 import javax.ws.rs.core.MediaType;
 
@@ -27,8 +28,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.QueueManager;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
+import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.junit.Test;
@@ -99,6 +102,38 @@ public class TestRMWebServicesFairScheduler extends JerseyTestBase {
     verifyClusterScheduler(json);
   }
   
+  @Test
+  public void testClusterSchedulerWithSubQueues() throws JSONException,
+      Exception {
+    FairScheduler scheduler = (FairScheduler)rm.getResourceScheduler();
+    QueueManager queueManager = scheduler.getQueueManager();
+    // create LeafQueue
+    queueManager.getLeafQueue("root.q.subqueue1", true);
+    queueManager.getLeafQueue("root.q.subqueue2", true);
+
+    WebResource r = resource();
+    ClientResponse response = r.path("ws").path("v1").path("cluster")
+        .path("scheduler").accept(MediaType.APPLICATION_JSON)
+        .get(ClientResponse.class);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    JSONObject json = response.getEntity(JSONObject.class);
+    JSONArray subQueueInfo = json.getJSONObject("scheduler")
+        .getJSONObject("schedulerInfo").getJSONObject("rootQueue")
+        .getJSONObject("childQueues").getJSONArray("queue")
+        .getJSONObject(1).getJSONObject("childQueues").getJSONArray("queue");
+    // subQueueInfo is consist of subqueue1 and subqueue2 info
+    assertEquals(2, subQueueInfo.length());
+
+    // Verify 'childQueues' field is omitted from FairSchedulerLeafQueueInfo.
+    try {
+      subQueueInfo.getJSONObject(1).getJSONObject("childQueues");
+      fail("FairSchedulerQueueInfo should omit field 'childQueues'" +
+           "if child queue is empty.");
+    } catch (JSONException je) {
+      assertEquals("JSONObject[\"childQueues\"] not found.", je.getMessage());
+    }
+  }
+
   private void verifyClusterScheduler(JSONObject json) throws JSONException,
       Exception {
     assertEquals("incorrect number of elements", 1, json.length());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a3d617b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
index c3fd9b8..c8fe855 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
@@ -321,10 +321,10 @@ The capacity scheduler supports hierarchical queues. This one request will print
 | usedResources | string | A string describing the current resources used by the queue |
 | queueName | string | The name of the queue |
 | state | string of QueueState | The state of the queue |
-| queues | array of queues(JSON)/zero or more queue objects(XML) | A collection of sub-queue information |
+| queues | array of queues(JSON)/zero or more queue objects(XML) | A collection of sub-queue information. Omitted if the queue has no sub-queues. |
 | resourcesUsed | A single resource object | The total amount of resources used by this queue |
 
-### Elements of the queues object for a Leaf queue - contains all elements in parent plus the following:
+### Elements of the queues object for a Leaf queue - contains all the elements in parent except 'queues' plus the following:
 
 | Item | Data Type | Description |
 |:---- |:---- |:---- |
@@ -1005,9 +1005,9 @@ Response Body:
 | clusterResources | A single resource object | The capacity of the cluster |
 | queueName | string | The name of the queue |
 | schedulingPolicy | string | The name of the scheduling policy used by the queue |
-| childQueues | array of queues(JSON)/queue objects(XML) | A collection of sub-queue information |
+| childQueues | array of queues(JSON)/queue objects(XML) | A collection of sub-queue information. Omitted if the queue has no childQueues. |
 
-### Elements of the queues object for a Leaf queue - contains all elements in parent plus the following
+### Elements of the queues object for a Leaf queue - contains all the elements in parent except 'childQueues' plus the following
 
 | Item | Data Type | Description |
 |:---- |:---- |:---- |
@@ -1044,43 +1044,15 @@ Response Body:
     "scheduler": {
         "schedulerInfo": {
             "rootQueue": {
-                "childQueues": [
-                    {
-                        "clusterResources": {
-                            "memory": 8192,
-                            "vCores": 8
-                        },
-                        "fairResources": {
-                            "memory": 0,
-                            "vCores": 0
-                        },
-                        "maxApps": 2147483647,
-                        "maxResources": {
-                            "memory": 8192,
-                            "vCores": 8
-                        },
-                        "minResources": {
-                            "memory": 0,
-                            "vCores": 0
-                        },
-                        "numActiveApps": 0,
-                        "numPendingApps": 0,
-                        "queueName": "root.default",
-                        "schedulingPolicy": "fair",
-                        "type": "fairSchedulerLeafQueueInfo",
-                        "usedResources": {
-                            "memory": 0,
-                            "vCores": 0
-                        }
-                    },
-                    {
-                        "childQueues": {
+                "childQueues": {
+                    "queue": [
+                        {
                             "clusterResources": {
                                 "memory": 8192,
                                 "vCores": 8
                             },
                             "fairResources": {
-                                "memory": 10000,
+                                "memory": 0,
                                 "vCores": 0
                             },
                             "maxApps": 2147483647,
@@ -1089,46 +1061,78 @@ Response Body:
                                 "vCores": 8
                             },
                             "minResources": {
-                                "memory": 5000,
+                                "memory": 0,
                                 "vCores": 0
                             },
                             "numActiveApps": 0,
                             "numPendingApps": 0,
-                            "queueName": "root.sample_queue.sample_sub_queue",
+                            "queueName": "root.default",
                             "schedulingPolicy": "fair",
-                            "type": [
-                                "fairSchedulerLeafQueueInfo"
-                            ],
+                            "type": "fairSchedulerLeafQueueInfo",
                             "usedResources": {
                                 "memory": 0,
                                 "vCores": 0
                             }
                         },
-                        "clusterResources": {
-                            "memory": 8192,
-                            "vCores": 8
-                        },
-                        "fairResources": {
-                            "memory": 10000,
-                            "vCores": 0
-                        },
-                        "maxApps": 50,
-                        "maxResources": {
-                            "memory": 8192,
-                            "vCores": 0
-                        },
-                        "minResources": {
-                            "memory": 10000,
-                            "vCores": 0
-                        },
-                        "queueName": "root.sample_queue",
-                        "schedulingPolicy": "fair",
-                        "usedResources": {
-                            "memory": 0,
-                            "vCores": 0
+                        {
+                            "childQueues": {
+                                "queue": [
+                                    {
+                                        "clusterResources": {
+                                            "memory": 8192,
+                                           "vCores": 8
+                                        },
+                                        "fairResources": {
+                                            "memory": 10000,
+                                            "vCores": 0
+                                        },
+                                        "maxApps": 2147483647,
+                                        "maxResources": {
+                                            "memory": 8192,
+                                            "vCores": 8
+                                        },
+                                        "minResources": {
+                                            "memory": 5000,
+                                            "vCores": 0
+                                        },
+                                        "numActiveApps": 0,
+                                        "numPendingApps": 0,
+                                        "queueName": "root.sample_queue.sample_sub_queue",
+                                        "schedulingPolicy": "fair",
+                                        "type": "fairSchedulerLeafQueueInfo",
+                                        "usedResources": {
+                                            "memory": 0,
+                                            "vCores": 0
+                                        }
+                                    }
+                                ]
+                            },
+                            "clusterResources": {
+                                "memory": 8192,
+                                "vCores": 8
+                            },
+                            "fairResources": {
+                                "memory": 10000,
+                                "vCores": 0
+                            },
+                            "maxApps": 50,
+                            "maxResources": {
+                                "memory": 8192,
+                                "vCores": 0
+                            },
+                            "minResources": {
+                                "memory": 10000,
+                                "vCores": 0
+                            },
+                            "queueName": "root.sample_queue",
+                            "schedulingPolicy": "fair",
+                            "usedResources": {
+                                "memory": 0,
+                                "vCores": 0
+                            }
                         }
-                    }
-                ],
+                    ],
+                },
                 "clusterResources": {
                     "memory": 8192,
                     "vCores": 8
@@ -1203,61 +1207,11 @@ Response Body:
       </clusterResources>
       <queueName>root</queueName>
       <schedulingPolicy>fair</schedulingPolicy>
-      <childQueues xsi:type="fairSchedulerLeafQueueInfo">
-        <maxApps>2147483647</maxApps>
-        <minResources>
-          <memory>0</memory>
-          <vCores>0</vCores>
-        </minResources>
-        <maxResources>
-          <memory>8192</memory>
-          <vCores>8</vCores>
-        </maxResources>
-        <usedResources>
-          <memory>0</memory>
-          <vCores>0</vCores>
-        </usedResources>
-        <fairResources>
-          <memory>0</memory>
-          <vCores>0</vCores>
-        </fairResources>
-        <clusterResources>
-          <memory>8192</memory>
-          <vCores>8</vCores>
-        </clusterResources>
-        <queueName>root.default</queueName>
-        <schedulingPolicy>fair</schedulingPolicy>
-        <numPendingApps>0</numPendingApps>
-        <numActiveApps>0</numActiveApps>
-      </childQueues>
       <childQueues>
-        <maxApps>50</maxApps>
-        <minResources>
-          <memory>10000</memory>
-          <vCores>0</vCores>
-        </minResources>
-        <maxResources>
-          <memory>8192</memory>
-          <vCores>0</vCores>
-        </maxResources>
-        <usedResources>
-          <memory>0</memory>
-          <vCores>0</vCores>
-        </usedResources>
-        <fairResources>
-          <memory>10000</memory>
-          <vCores>0</vCores>
-        </fairResources>
-        <clusterResources>
-          <memory>8192</memory>
-          <vCores>8</vCores>
-        </clusterResources>
-        <queueName>root.sample_queue</queueName>
-        <schedulingPolicy>fair</schedulingPolicy>
-        <childQueues xsi:type="fairSchedulerLeafQueueInfo">
+        <queue xsi:type="fairSchedulerLeafQueueInfo">
           <maxApps>2147483647</maxApps>
           <minResources>
-            <memory>5000</memory>
+            <memory>0</memory>
             <vCores>0</vCores>
           </minResources>
           <maxResources>
@@ -1269,18 +1223,72 @@ Response Body:
             <vCores>0</vCores>
           </usedResources>
           <fairResources>
-            <memory>10000</memory>
+            <memory>0</memory>
             <vCores>0</vCores>
           </fairResources>
           <clusterResources>
             <memory>8192</memory>
             <vCores>8</vCores>
           </clusterResources>
-          <queueName>root.sample_queue.sample_sub_queue</queueName>
+          <queueName>root.default</queueName>
           <schedulingPolicy>fair</schedulingPolicy>
           <numPendingApps>0</numPendingApps>
           <numActiveApps>0</numActiveApps>
-        </childQueues>
+        </queue>
+        <queue>
+          <maxApps>50</maxApps>
+          <minResources>
+            <memory>10000</memory>
+            <vCores>0</vCores>
+          </minResources>
+          <maxResources>
+            <memory>8192</memory>
+            <vCores>0</vCores>
+          </maxResources>
+          <usedResources>
+            <memory>0</memory>
+            <vCores>0</vCores>
+          </usedResources>
+          <fairResources>
+            <memory>10000</memory>
+            <vCores>0</vCores>
+          </fairResources>
+          <clusterResources>
+            <memory>8192</memory>
+            <vCores>8</vCores>
+          </clusterResources>
+          <queueName>root.sample_queue</queueName>
+          <schedulingPolicy>fair</schedulingPolicy>
+          <childQueues>
+            <queue xsi:type="fairSchedulerLeafQueueInfo">
+              <maxApps>2147483647</maxApps>
+              <minResources>
+                <memory>5000</memory>
+                <vCores>0</vCores>
+              </minResources>
+              <maxResources>
+                <memory>8192</memory>
+                <vCores>8</vCores>
+              </maxResources>
+              <usedResources>
+                <memory>0</memory>
+                <vCores>0</vCores>
+              </usedResources>
+              <fairResources>
+                <memory>10000</memory>
+                <vCores>0</vCores>
+              </fairResources>
+              <clusterResources>
+                <memory>8192</memory>
+                <vCores>8</vCores>
+              </clusterResources>
+              <queueName>root.sample_queue.sample_sub_queue</queueName>
+              <schedulingPolicy>fair</schedulingPolicy>
+              <numPendingApps>0</numPendingApps>
+              <numActiveApps>0</numActiveApps>
+            </queue>
+          </childQueues>
+        </queue>
       </childQueues>
     </rootQueue>
   </schedulerInfo>