You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by da...@apache.org on 2015/10/30 22:17:52 UTC

[01/12] storm git commit: [STORM-1144] Display resource usage in UI

Repository: storm
Updated Branches:
  refs/heads/master a0f3412a8 -> ffb5d0338


[STORM-1144] Display resource usage in UI


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/70f455f1
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/70f455f1
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/70f455f1

Branch: refs/heads/master
Commit: 70f455f133db1c015999d1748fa778dacefc7684
Parents: b6615d5
Author: zhuol <zh...@yahoo-inc.com>
Authored: Thu Oct 29 13:55:55 2015 -0500
Committer: zhuol <zh...@yahoo-inc.com>
Committed: Thu Oct 29 13:55:55 2015 -0500

----------------------------------------------------------------------
 .../src/clj/backtype/storm/daemon/nimbus.clj    | 26 ++++++++-
 storm-core/src/clj/backtype/storm/ui/core.clj   | 16 +++++-
 .../jvm/backtype/storm/scheduler/Cluster.java   | 16 +++++-
 .../storm/scheduler/TopologyDetails.java        | 51 +++++++++++++++++
 .../resource/ResourceAwareScheduler.java        | 38 +++++++++----
 storm-core/src/storm.thrift                     | 18 ++++++
 .../public/templates/index-page-template.html   | 17 ++++--
 .../templates/topology-page-template.html       | 60 ++++++++++++++++++++
 storm-core/src/ui/public/topology.html          |  8 +++
 .../scheduler/resource_aware_scheduler_test.clj | 28 ++++-----
 10 files changed, 246 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/70f455f1/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index dd533b1..30a8677 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -122,6 +122,7 @@
      :leader-elector (zk-leader-elector conf)
      :code-distributor (mk-code-distributor conf)
      :id->sched-status (atom {})
+     :id->resources (atom {})
      :cred-renewers (AuthUtils/GetCredentialRenewers conf)
      :nimbus-autocred-plugins (AuthUtils/getNimbusAutoCredPlugins conf)
      }))
@@ -699,7 +700,8 @@
         ;; call scheduler.schedule to schedule all the topologies
         ;; the new assignments for all the topologies are in the cluster object.
         _ (.schedule (:scheduler nimbus) topologies cluster)
-        _ (reset! (:id->sched-status nimbus) (.getStatusMap cluster))]
+        _ (reset! (:id->sched-status nimbus) (.getStatusMap cluster))
+        _ (reset! (:id->resources nimbus) (merge @(:id->resources nimbus) (.getResourcesMap cluster)))]
     (.getAssignments cluster)))
 
 (defn changed-executors [executor->node+port new-executor->node+port]
@@ -1499,6 +1501,13 @@
                                                             (extract-status-str base))]
                                                (when-let [owner (:owner base)] (.set_owner topo-summ owner))
                                                (when-let [sched-status (.get @(:id->sched-status nimbus) id)] (.set_sched_status topo-summ sched-status))
+                                               (when-let [resources (.get @(:id->resources nimbus) id)]
+                                                 (.set_requested_memonheap topo-summ (get resources 0))
+                                                 (.set_requested_memoffheap topo-summ (get resources 1))
+                                                 (.set_requested_cpu topo-summ (get resources 2))
+                                                 (.set_assigned_memonheap topo-summ (get resources 3))
+                                                 (.set_assigned_memoffheap topo-summ (get resources 4))
+                                                 (.set_assigned_cpu topo-summ (get resources 5)))
                                                (.set_replication_count topo-summ (if (:code-distributor nimbus)
                                                                                    (.getReplicationCount (:code-distributor nimbus) id)
                                                                                    1))
@@ -1558,9 +1567,17 @@
                            )]
             (when-let [owner (:owner base)] (.set_owner topo-info owner))
             (when-let [sched-status (.get @(:id->sched-status nimbus) storm-id)] (.set_sched_status topo-info sched-status))
+            (when-let [resources (.get @(:id->resources nimbus) storm-id)]
+              (.set_requested_memonheap topo-info (get resources 0))
+              (.set_requested_memoffheap topo-info (get resources 1))
+              (.set_requested_cpu topo-info (get resources 2))
+              (.set_assigned_memonheap topo-info (get resources 3))
+              (.set_assigned_memoffheap topo-info (get resources 4))
+              (.set_assigned_cpu topo-info (get resources 5)))
             (when-let [component->debug (:component->debug base)]
               (.set_component_debug topo-info (map-val converter/thriftify-debugoptions component->debug)))
             (.set_replication_count topo-info (if (:code-distributor nimbus) (.getReplicationCount (:code-distributor nimbus) storm-id) 1))
+
             topo-info
           ))
 
@@ -1589,6 +1606,13 @@
             (.set_owner topo-page-info owner))
           (when-let [sched-status (.get @(:id->sched-status nimbus) topo-id)]
             (.set_sched_status topo-page-info sched-status))
+          (when-let [resources (.get @(:id->resources nimbus) storm-id)]
+            (.set_requested_memonheap topo-page-info (get resources 0))
+            (.set_requested_memoffheap topo-page-info (get resources 1))
+            (.set_requested_cpu topo-page-info (get resources 2))
+            (.set_assigned_memonheap topo-page-info (get resources 3))
+            (.set_assigned_memoffheap topo-page-info (get resources 4))
+            (.set_assigned_cpu topo-page-info (get resources 5)))
           (doto topo-page-info
             (.set_name (:storm-name info))
             (.set_status (extract-status-str (:base info)))

http://git-wip-us.apache.org/repos/asf/storm/blob/70f455f1/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/ui/core.clj b/storm-core/src/clj/backtype/storm/ui/core.clj
index cb5dbbe..aaaeef0 100644
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@ -407,7 +407,15 @@
        "workersTotal" (.get_num_workers t)
        "executorsTotal" (.get_num_executors t)
        "replicationCount" (.get_replication_count t)
-       "schedulerInfo" (.get_sched_status t)})}))
+       "schedulerInfo" (.get_sched_status t)
+       "requestedMemOnHeap" (.get_requested_memonheap t)
+       "requestedMemOffHeap" (.get_requested_memoffheap t)
+       "requestedMem" (+ (.get_requested_memonheap t) (.get_requested_memoffheap t))
+       "requestedCpu" (.get_requested_cpu t)
+       "assignedMemOnHeap" (.get_assigned_memonheap t)
+       "assignedMemOffHeap" (.get_assigned_memoffheap t)
+       "assignedTotalMem" (+ (.get_assigned_memonheap t) (.get_assigned_memoffheap t))
+       "assignedTotalCpu" (.get_assigned_cpu t)})}))
 
 (defn topology-stats [window stats]
   (let [times (stats-times (:emitted stats))
@@ -525,6 +533,12 @@
      "workersTotal" (.get_num_workers topo-info)
      "executorsTotal" (.get_num_executors topo-info)
      "schedulerInfo" (.get_sched_status topo-info)
+     "requestedMemOnHeap" (.get_requested_memonheap topo-info)
+     "requestedMemOffHeap" (.get_requested_memoffheap topo-info)
+     "requestedCpu" (.get_requested_cpu topo-info)
+     "assignedMemOnHeap" (.get_assigned_memonheap topo-info)
+     "assignedMemOffHeap" (.get_assigned_memoffheap topo-info)
+     "assignedCpu" (.get_assigned_cpu topo-info)
      "topologyStats" topo-stats
      "spouts" (map (partial comp-agg-stats-json id secure?)
                    (.get_id_to_spout_agg_stats topo-info))

http://git-wip-us.apache.org/repos/asf/storm/blob/70f455f1/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java b/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
index dd15999..d676feb 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
@@ -51,9 +51,14 @@ public class Cluster {
     private Map<String, String> status;
 
     /**
+     * key topologyId, Value: requested and assigned resources (e.g., on-heap/off-heap mem, cpu) for each topology.
+     */
+    private Map<String, Double[]> resources;
+
+    /**
      * a map from hostname to supervisor id.
      */
-    private Map<String, List<String>>        hostToId;
+    private Map<String, List<String>> hostToId;
 
     private Map conf = null;
 
@@ -67,6 +72,7 @@ public class Cluster {
         this.assignments = new HashMap<String, SchedulerAssignmentImpl>(assignments.size());
         this.assignments.putAll(assignments);
         this.status = new HashMap<String, String>();
+        this.resources = new HashMap<String, Double[]>();
         this.hostToId = new HashMap<String, List<String>>();
         for (String nodeId : supervisors.keySet()) {
             SupervisorDetails supervisor = supervisors.get(nodeId);
@@ -487,4 +493,12 @@ public class Cluster {
     public Map<String, String> getStatusMap() {
         return this.status;
     }
+
+    public void setResources(String topologyId, Double[] resources) {
+        this.resources.put(topologyId, resources);
+    }
+
+    public Map<String, Double[]> getResourcesMap() {
+        return this.resources;
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/70f455f1/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java b/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
index 872a834..01fa0df 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
@@ -319,6 +319,57 @@ public class TopologyDetails {
     }
 
     /**
+     * Note: The public API relevant to resource aware scheduling is unstable as of May 2015.
+     *       We reserve the right to change them.
+     *
+     * @return the total on-heap memory requested for this topology
+     */
+    public Double getTotalRequestedMemOnHeap() {
+        Double total_memonheap = 0.0;
+        for (ExecutorDetails exec : this.getExecutors()) {
+            Double exec_mem = getOnHeapMemoryRequirement(exec);
+            if (exec_mem != null) {
+                total_memonheap += exec_mem;
+            }
+        }
+        return total_memonheap;
+    }
+
+    /**
+     * Note: The public API relevant to resource aware scheduling is unstable as of May 2015.
+     *       We reserve the right to change them.
+     *
+     * @return the total off-heap memory requested for this topology
+     */
+    public Double getTotalRequestedMemOffHeap() {
+        Double total_memoffheap = 0.0;
+        for (ExecutorDetails exec : this.getExecutors()) {
+            Double exec_mem = getOffHeapMemoryRequirement(exec);
+            if (exec_mem != null) {
+                total_memoffheap += exec_mem;
+            }
+        }
+        return total_memoffheap;
+    }
+
+    /**
+     * Note: The public API relevant to resource aware scheduling is unstable as of May 2015.
+     *       We reserve the right to change them.
+     *
+     * @return the total cpu requested for this topology
+     */
+    public Double getTotalRequestedCpu() {
+        Double total_cpu = 0.0;
+        for (ExecutorDetails exec : this.getExecutors()) {
+            Double exec_cpu = getTotalCpuReqTask(exec);
+            if (exec_cpu != null) {
+                total_cpu += exec_cpu;
+            }
+        }
+        return total_cpu;
+    }
+
+    /**
      * get the resources requirements for a executor
      * @param exec
      * @return a map containing the resource requirements for this exec

http://git-wip-us.apache.org/repos/asf/storm/blob/70f455f1/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
index b3ff97b..6e455f2 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
@@ -63,12 +63,20 @@ public class ResourceAwareScheduler implements IScheduler {
 
                 schedulerAssignmentMap = RAStrategy.schedule(td);
 
+                double requestedMemOnHeap = td.getTotalRequestedMemOnHeap();
+                double requestedMemOffHeap = td.getTotalRequestedMemOffHeap();
+                double requestedCpu = td.getTotalRequestedCpu();
+                double assignedMemOnHeap = 0.0;
+                double assignedMemOffHeap = 0.0;
+                double assignedCpu = 0.0;
+
                 if (schedulerAssignmentMap != null) {
                     try {
                         Set<String> nodesUsed = new HashSet<String>();
-                        for (Map.Entry<WorkerSlot, Collection<ExecutorDetails>> taskToWorkerEntry : schedulerAssignmentMap.entrySet()) {
-                            WorkerSlot targetSlot = taskToWorkerEntry.getKey();
-                            Collection<ExecutorDetails> execsNeedScheduling = taskToWorkerEntry.getValue();
+                        int assignedWorkers = schedulerAssignmentMap.keySet().size();
+                        for (Map.Entry<WorkerSlot, Collection<ExecutorDetails>> workerToTasksEntry : schedulerAssignmentMap.entrySet()) {
+                            WorkerSlot targetSlot = workerToTasksEntry.getKey();
+                            Collection<ExecutorDetails> execsNeedScheduling = workerToTasksEntry.getValue();
                             RAS_Node targetNode = RAStrategy.idToNode(targetSlot.getNodeId());
                             targetNode.assign(targetSlot, td, execsNeedScheduling, cluster);
                             LOG.debug("ASSIGNMENT    TOPOLOGY: {}  TASKS: {} To Node: {} on Slot: {}",
@@ -76,20 +84,30 @@ public class ResourceAwareScheduler implements IScheduler {
                             if (!nodesUsed.contains(targetNode.getId())) {
                                 nodesUsed.add(targetNode.getId());
                             }
+                            assignedMemOnHeap += targetSlot.getAllocatedMemOnHeap();
+                            assignedMemOffHeap += targetSlot.getAllocatedMemOffHeap();
+                            assignedCpu += targetSlot.getAllocatedCpu();
                         }
-                        LOG.debug("Topology: {} assigned to {} nodes on {} workers", td.getId(), nodesUsed.size(), schedulerAssignmentMap.keySet().size());
-                        cluster.setStatus(td.getId(), td.getId() + " Fully Scheduled");
+                        LOG.debug("Topology: {} assigned to {} nodes on {} workers", td.getId(), nodesUsed.size(), assignedWorkers);
+                        cluster.setStatus(td.getId(), "Fully Scheduled");
                     } catch (IllegalStateException ex) {
                         LOG.error(ex.toString());
-                        LOG.error("Unsuccessfull in scheduling {}", td.getId());
-                        cluster.setStatus(td.getId(), "Unsuccessfull in scheduling " + td.getId());
+                        LOG.error("Unsuccessful in scheduling", td.getId());
+                        cluster.setStatus(td.getId(), "Unsuccessful in scheduling");
                     }
                 } else {
-                    LOG.error("Unsuccessfull in scheduling topology {}", td.getId());
-                    cluster.setStatus(td.getId(), "Unsuccessfull in scheduling " + td.getId());
+                    LOG.error("Unsuccessful in scheduling", td.getId());
+                    cluster.setStatus(td.getId(), "Unsuccessful in scheduling");
                 }
+                Double[] resources = {requestedMemOnHeap, requestedMemOffHeap, requestedCpu,
+                        assignedMemOnHeap, assignedMemOffHeap, assignedCpu};
+                LOG.debug("setResources for {}: requested on-heap mem, off-heap mem, cpu: {} {} {} " +
+                                "assigned on-heap mem, off-heap mem, cpu: {} {} {}",
+                        td.getId(), requestedMemOnHeap, requestedMemOffHeap, requestedCpu,
+                        assignedMemOnHeap, assignedMemOffHeap, assignedCpu);
+                cluster.setResources(td.getId(), resources);
             } else {
-                cluster.setStatus(td.getId(), td.getId() + " Fully Scheduled");
+                cluster.setStatus(td.getId(), "Fully Scheduled");
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/70f455f1/storm-core/src/storm.thrift
----------------------------------------------------------------------
diff --git a/storm-core/src/storm.thrift b/storm-core/src/storm.thrift
index aae851e..3c07cac 100644
--- a/storm-core/src/storm.thrift
+++ b/storm-core/src/storm.thrift
@@ -146,6 +146,12 @@ struct TopologySummary {
 513: optional string sched_status;
 514: optional string owner;
 515: optional i32 replication_count;
+521: optional double requested_memonheap;
+522: optional double requested_memoffheap;
+523: optional double requested_cpu;
+524: optional double assigned_memonheap;
+525: optional double assigned_memoffheap;
+526: optional double assigned_cpu;
 }
 
 struct SupervisorSummary {
@@ -232,6 +238,12 @@ struct TopologyInfo {
 513: optional string sched_status;
 514: optional string owner;
 515: optional i32 replication_count;
+521: optional double requested_memonheap;
+522: optional double requested_memoffheap;
+523: optional double requested_cpu;
+524: optional double assigned_memonheap;
+525: optional double assigned_memoffheap;
+526: optional double assigned_cpu;
 }
 
 struct DebugOptions {
@@ -300,6 +312,12 @@ struct TopologyPageInfo {
 13: optional string owner;
 14: optional DebugOptions debug_options;
 15: optional i32 replication_count;
+521: optional double requested_memonheap;
+522: optional double requested_memoffheap;
+523: optional double requested_cpu;
+524: optional double assigned_memonheap;
+525: optional double assigned_memoffheap;
+526: optional double assigned_cpu;
 }
 
 struct ExecutorAggregateStats {

http://git-wip-us.apache.org/repos/asf/storm/blob/70f455f1/storm-core/src/ui/public/templates/index-page-template.html
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/templates/index-page-template.html b/storm-core/src/ui/public/templates/index-page-template.html
index 9c93e46..47bcdfd 100644
--- a/storm-core/src/ui/public/templates/index-page-template.html
+++ b/storm-core/src/ui/public/templates/index-page-template.html
@@ -115,11 +115,6 @@
           </span>
         </th>
         <th>
-          <span data-toggle="tooltip" data-placement="right" title="The unique ID given to a Topology each time it is launched.">
-            Id
-          </span>
-        </th>
-        <th>
           <span data-toggle="tooltip" data-placement="above" title="The user that submitted the Topology, if authentication is enabled.">
             Owner
           </span>
@@ -159,6 +154,16 @@
             Scheduler Info
           </span>
         </th>
+        <th>
+          <span data-toggle="tooltip" data-placement="above" title="Assigned Total Memory by Scheduler. Every 100 means 1 core.">
+            Assigned CPU (%)
+          </span>
+        </th>
+        <th>
+          <span data-toggle="tooltip" data-placement="left" title="This shows information from the scheduler about the latest attempt to schedule the Topology on the cluster.">
+            Scheduler Info
+          </span>
+        </th>
       </tr>
     </thead>
     <tbody>
@@ -174,6 +179,8 @@
         <td>{{tasksTotal}}</td>
         <td>{{replicationCount}}</td>
         <td>{{schedulerInfo}}</td>
+        <td>{{assignedTotalMem}}</td>
+        <td>{{assignedTotalCpu}}</td>
       </tr>
       {{/topologies}}
     </tbody>

http://git-wip-us.apache.org/repos/asf/storm/blob/70f455f1/storm-core/src/ui/public/templates/topology-page-template.html
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/templates/topology-page-template.html b/storm-core/src/ui/public/templates/topology-page-template.html
index 817a7ab..537f105 100644
--- a/storm-core/src/ui/public/templates/topology-page-template.html
+++ b/storm-core/src/ui/public/templates/topology-page-template.html
@@ -86,6 +86,66 @@
     </tbody>
   </table>
 </script>
+<script id="topology-resources-template" type="text/html">
+  <table id="topology-resources-table" class="table compact">
+    <thead>
+    <tr>
+      <th>
+          <span data-toggle="tooltip" data-placement="right" title="The name given to the topology by when it was submitted.">
+            Name
+          </span>
+      </th>
+      <th>
+          <span data-toggle="tooltip" data-placement="right" title="The unique ID given to a Topology each time it is launched.">
+            Id
+          </span>
+      </th>
+      <th>
+          <span data-toggle="tooltip" data-placement="above" title="Requested Total On-Heap Memory by Scheduler.">
+            Requested On-Heap Memory (MB)
+          </span>
+      </th>
+      <th>
+          <span data-toggle="tooltip" data-placement="above" title="Assigned Total On-Heap Memory by Scheduler.">
+            Assigned On-Heap Memory (MB)
+          </span>
+      </th>
+      <th>
+          <span data-toggle="tooltip" data-placement="above" title="Requested Total Off-Heap Memory by Scheduler.">
+            Requested Off-Heap Memory (MB)
+          </span>
+      </th>
+      <th>
+          <span data-toggle="tooltip" data-placement="above" title="Assigned Total Off-Heap Memory by Scheduler.">
+            Assigned Off-Heap Memory (MB)
+          </span>
+      </th>
+      <th>
+          <span data-toggle="tooltip" data-placement="above" title="Requested Total CPU by Scheduler. Every 100 means 1 core.">
+            Requested CPU (%)
+          </span>
+      </th>
+      <th>
+          <span data-toggle="tooltip" data-placement="left" title="Assigned Total CPU by Scheduler. Every 100 means 1 core.">
+            Assigned CPU (%)
+          </span>
+      </th>
+    </tr>
+    </thead>
+    <tbody>
+    <tr>
+      <td>{{name}}</td>
+      <td>{{id}}</td>
+      <td>{{requestedMemOnHeap}}</td>
+      <td>{{assignedMemOnHeap}}</td>
+      <td>{{requestedMemOffHeap}}</td>
+      <td>{{assignedMemOffHeap}}</td>
+      <td>{{requestedCpu}}</td>
+      <td>{{assignedCpu}}</td>
+    </tr>
+    </tbody>
+  </table>
+</script>
 <script id="topology-stats-template" type="text/html">
   <h2>Topology stats</h2>
   <table class="table table-striped compact" id="topology-stats-table">

http://git-wip-us.apache.org/repos/asf/storm/blob/70f455f1/storm-core/src/ui/public/topology.html
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/topology.html b/storm-core/src/ui/public/topology.html
index e873bb6..a0e359f 100644
--- a/storm-core/src/ui/public/topology.html
+++ b/storm-core/src/ui/public/topology.html
@@ -54,6 +54,12 @@
     </div>
   </div>
   <div class="row">
+      <div class="col-md-12">
+          <h2>Topology resources</h2>
+          <div id="topology-resources"></div>
+      </div>
+  </div>
+  <div class="row">
     <div id="topology-actions" class="col-md-12"></div>
   </div>
   <div class="row">
@@ -265,6 +271,7 @@ $(document).ready(function() {
         });
 
         var topologySummary = $("#topology-summary");
+        var topologyResources = $("#topology-resources");
         var topologyStats = $("#topology-stats");
         var spoutStats = $("#spout-stats");
         var boltStats = $("#bolt-stats");
@@ -279,6 +286,7 @@ $(document).ready(function() {
               renderLogLevelForm (template, data);
             }
             topologySummary.append(Mustache.render($(template).filter("#topology-summary-template").html(),response));
+            topologyResources.append(Mustache.render($(template).filter("#topology-resources-template").html(),response));
             topologyActions.append(Mustache.render($(template).filter("#topology-actions-template").html(),buttonJsonData));
             topologyStats.append(Mustache.render($(template).filter("#topology-stats-template").html(),response));
             //window, emitted, transferred, complete latency, acked, failed

http://git-wip-us.apache.org/repos/asf/storm/blob/70f455f1/storm-core/test/clj/backtype/storm/scheduler/resource_aware_scheduler_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/scheduler/resource_aware_scheduler_test.clj b/storm-core/test/clj/backtype/storm/scheduler/resource_aware_scheduler_test.clj
index 7f9b247..bd097a5 100644
--- a/storm-core/test/clj/backtype/storm/scheduler/resource_aware_scheduler_test.clj
+++ b/storm-core/test/clj/backtype/storm/scheduler/resource_aware_scheduler_test.clj
@@ -386,7 +386,7 @@
         ;; for each executor that was scheduled on healthy workers, their slots should remain unchanged after a new scheduling
         (doseq [ed healthy-eds]
           (is (.equals (.get copy-old-mapping ed) (.get new-ed->slot ed))))
-        (is (= "topology2 Fully Scheduled" (.get (.getStatusMap cluster) "topology2")))))
+        (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology2")))))
     
     (testing "When a supervisor fails, RAS does not alter existing assignments"
       (let [existing-assignments {"topology1" (SchedulerAssignmentImpl. "topology1"
@@ -411,7 +411,7 @@
             new-ed->slot (.getExecutorToSlot new-assignment)]
         (doseq [ed existing-eds]
           (is (.equals (.get copy-old-mapping ed) (.get new-ed->slot ed))))
-        (is (= "topology1 Fully Scheduled" (.get (.getStatusMap new-cluster) "topology1")))))
+        (is (= "Fully Scheduled" (.get (.getStatusMap new-cluster) "topology1")))))
 
     (testing "When a supervisor and a worker on it fails, RAS does not alter existing assignments"
       (let [existing-assignments {"topology1" (SchedulerAssignmentImpl. "topology1"
@@ -437,7 +437,7 @@
             new-ed->slot (.getExecutorToSlot new-assignment)]
         (doseq [ed existing-eds]
           (is (.equals (.get copy-old-mapping ed) (.get new-ed->slot ed))))
-        (is (= "topology1 Fully Scheduled" (.get (.getStatusMap new-cluster) "topology1")))))
+        (is (= "Fully Scheduled" (.get (.getStatusMap new-cluster) "topology1")))))
 
     (testing "Scheduling a new topology does not disturb other assignments unnecessarily"
       (let [cluster (Cluster. (nimbus/standalone-nimbus) supers {}
@@ -454,8 +454,8 @@
             new-ed->slot (.getExecutorToSlot new-assignment)]
         (doseq [ed (.keySet copy-old-mapping)]
           (is (.equals (.get copy-old-mapping ed) (.get new-ed->slot ed))))  ;; the assignment for topo1 should not change
-        (is (= "topology1 Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))
-        (is (= "topology2 Fully Scheduled" (.get (.getStatusMap cluster) "topology2")))))))
+        (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))
+        (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology2")))))))
 
 ;; Automated tests for heterogeneous cluster
 (deftest test-heterogeneous-cluster
@@ -555,9 +555,9 @@
             _ (.schedule scheduler topologies cluster)
             super->mem-usage (get-super->mem-usage cluster topologies)
             super->cpu-usage (get-super->cpu-usage cluster topologies)]
-        (is (= "topology1 Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))
-        (is (= "topology2 Fully Scheduled" (.get (.getStatusMap cluster) "topology2")))
-        (is (= "topology3 Fully Scheduled" (.get (.getStatusMap cluster) "topology3")))
+        (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))
+        (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology2")))
+        (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology3")))
         (doseq [super (.values supers)] 
           (let [mem-avail (.getTotalMemory super)
                 mem-used (.get super->mem-usage super)
@@ -572,9 +572,9 @@
                                "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
             topologies (Topologies. (to-top-map [topology1 topology2 topology3]))
             _ (.schedule scheduler topologies cluster)
-                scheduled-topos (if (= "topology1 Fully Scheduled" (.get (.getStatusMap cluster) "topology1")) 1 0)
-                scheduled-topos (+ scheduled-topos (if (= "topology2 Fully Scheduled" (.get (.getStatusMap cluster) "topology2")) 1 0))
-                scheduled-topos (+ scheduled-topos (if (= "topology4 Fully Scheduled" (.get (.getStatusMap cluster) "topology4")) 1 0))]
+                scheduled-topos (if (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1")) 1 0)
+                scheduled-topos (+ scheduled-topos (if (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology2")) 1 0))
+                scheduled-topos (+ scheduled-topos (if (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology4")) 1 0))]
             (is (= scheduled-topos 2)))) ;; only 2 topos will get (fully) scheduled
 
     (testing "Launch topo5 only, both mem and cpu should be exactly used up"
@@ -585,7 +585,7 @@
             _ (.schedule scheduler topologies cluster)
             super->mem-usage (get-super->mem-usage cluster topologies)
             super->cpu-usage (get-super->cpu-usage cluster topologies)]
-        (is (= "topology5 Fully Scheduled" (.get (.getStatusMap cluster) "topology5")))
+        (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology5")))
         (doseq [super (.values supers)] 
           (let [mem-avail (.getTotalMemory super)
                 mem-used (.get super->mem-usage super)
@@ -616,7 +616,7 @@
                       (mk-ed-map [["spout1" 0 4]]))
           topologies (Topologies. (to-top-map [topology1]))]
       (.schedule scheduler topologies cluster)
-      (is (= (.get (.getStatusMap cluster) "topology1") "topology1 Fully Scheduled"))
+      (is (= (.get (.getStatusMap cluster) "topology1") "Fully Scheduled"))
       (is (= (.getAssignedNumWorkers cluster topology1) 4)))
     (testing "test when no more workers are available due to topology worker max heap size limit but there is memory is still available")
     (let [cluster (Cluster. (nimbus/standalone-nimbus) supers {}
@@ -642,7 +642,7 @@
       ;;The cluster contains 4 free WorkerSlots. For this topolology each worker is limited to a max heap size of 128.0
       ;;Thus, one executor not going to be able to get scheduled thus failing the scheduling of this topology and no executors of this topology will be scheduleded
       (is (= (.size (.getUnassignedExecutors cluster topology1)) 5))
-      (is (= (.get (.getStatusMap cluster) "topology1")  "Unsuccessfull in scheduling topology1")))
+      (is (= (.get (.getStatusMap cluster) "topology1")  "Unsuccessful in scheduling")))
     
     (let [cluster (Cluster. (nimbus/standalone-nimbus) supers {}
                              {STORM-NETWORK-TOPOGRAPHY-PLUGIN


[10/12] storm git commit: Merge branch '1144' of https://github.com/zhuoliu/storm into STORM-1144

Posted by da...@apache.org.
Merge branch '1144' of https://github.com/zhuoliu/storm into STORM-1144


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/090991dd
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/090991dd
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/090991dd

Branch: refs/heads/master
Commit: 090991ddde328a23554421e4ea89b77f80fb057f
Parents: a0f3412 9b9efc4
Author: Derek Dagit <de...@yahoo-inc.com>
Authored: Fri Oct 30 16:07:50 2015 -0500
Committer: Derek Dagit <de...@yahoo-inc.com>
Committed: Fri Oct 30 16:07:50 2015 -0500

----------------------------------------------------------------------
 STORM-UI-REST-API.md                            |   19 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |   26 +-
 storm-core/src/clj/backtype/storm/ui/core.clj   |   16 +-
 .../backtype/storm/generated/TopologyInfo.java  |  604 ++++++++-
 .../storm/generated/TopologyPageInfo.java       |  606 ++++++++-
 .../storm/generated/TopologySummary.java        |  606 ++++++++-
 .../jvm/backtype/storm/scheduler/Cluster.java   |   16 +-
 .../storm/scheduler/TopologyDetails.java        |   51 +
 .../resource/ResourceAwareScheduler.java        |   38 +-
 storm-core/src/py/storm/ttypes.py               | 1145 +++++++++++++++---
 storm-core/src/storm.thrift                     |   18 +
 storm-core/src/ui/public/index.html             |    6 +-
 .../public/templates/index-page-template.html   |   18 +-
 .../templates/topology-page-template.html       |   60 +
 storm-core/src/ui/public/topology.html          |    8 +
 .../scheduler/resource_aware_scheduler_test.clj |   38 +-
 16 files changed, 3018 insertions(+), 257 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/090991dd/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------


[12/12] storm git commit: Merge branch 'STORM-1144'

Posted by da...@apache.org.
Merge branch 'STORM-1144'


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

Branch: refs/heads/master
Commit: ffb5d033864362378200dcd6bfa3dd086a225a47
Parents: a0f3412 3502907
Author: Derek Dagit <de...@yahoo-inc.com>
Authored: Fri Oct 30 16:17:03 2015 -0500
Committer: Derek Dagit <de...@yahoo-inc.com>
Committed: Fri Oct 30 16:17:03 2015 -0500

----------------------------------------------------------------------
 CHANGELOG.md                                    |    1 +
 STORM-UI-REST-API.md                            |   19 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |   26 +-
 storm-core/src/clj/backtype/storm/ui/core.clj   |   16 +-
 .../backtype/storm/generated/TopologyInfo.java  |  604 ++++++++-
 .../storm/generated/TopologyPageInfo.java       |  606 ++++++++-
 .../storm/generated/TopologySummary.java        |  606 ++++++++-
 .../jvm/backtype/storm/scheduler/Cluster.java   |   16 +-
 .../storm/scheduler/TopologyDetails.java        |   51 +
 .../resource/ResourceAwareScheduler.java        |   38 +-
 storm-core/src/py/storm/ttypes.py               | 1145 +++++++++++++++---
 storm-core/src/storm.thrift                     |   18 +
 storm-core/src/ui/public/index.html             |    6 +-
 .../public/templates/index-page-template.html   |   18 +-
 .../templates/topology-page-template.html       |   60 +
 storm-core/src/ui/public/topology.html          |    8 +
 .../scheduler/resource_aware_scheduler_test.clj |   38 +-
 17 files changed, 3019 insertions(+), 257 deletions(-)
----------------------------------------------------------------------



[07/12] storm git commit: Minor

Posted by da...@apache.org.
Minor


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

Branch: refs/heads/master
Commit: f6149c87763d431dcfca4a3a0abdccc534c8d180
Parents: 2ef1414
Author: zhuol <zh...@yahoo-inc.com>
Authored: Fri Oct 30 15:25:52 2015 -0500
Committer: zhuol <zh...@yahoo-inc.com>
Committed: Fri Oct 30 15:25:52 2015 -0500

----------------------------------------------------------------------
 storm-core/src/ui/public/templates/index-page-template.html | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/f6149c87/storm-core/src/ui/public/templates/index-page-template.html
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/templates/index-page-template.html b/storm-core/src/ui/public/templates/index-page-template.html
index 5005f8d..15d72a7 100644
--- a/storm-core/src/ui/public/templates/index-page-template.html
+++ b/storm-core/src/ui/public/templates/index-page-template.html
@@ -155,7 +155,7 @@
           </span>
         </th>
         <th>
-          <span data-toggle="tooltip" data-placement="above" title="Assigned Total Memory by Scheduler. Every 100 means 1 core.">
+          <span data-toggle="tooltip" data-placement="above" title="Assigned Total CPU by Scheduler. Every 100 means 1 core.">
             Assigned CPU (%)
           </span>
         </th>


[05/12] storm git commit: Fix issues

Posted by da...@apache.org.
Fix issues


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

Branch: refs/heads/master
Commit: bed92d6f7624244a834006c5188cbfd5dd7b6424
Parents: 416686e
Author: zhuol <zh...@yahoo-inc.com>
Authored: Thu Oct 29 14:57:55 2015 -0500
Committer: zhuol <zh...@yahoo-inc.com>
Committed: Thu Oct 29 15:36:07 2015 -0500

----------------------------------------------------------------------
 storm-core/src/ui/public/index.html                         | 6 +++---
 storm-core/src/ui/public/templates/index-page-template.html | 7 +++----
 2 files changed, 6 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/bed92d6f/storm-core/src/ui/public/index.html
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/index.html b/storm-core/src/ui/public/index.html
index 1cd6445..9eedfb8 100644
--- a/storm-core/src/ui/public/index.html
+++ b/storm-core/src/ui/public/index.html
@@ -135,11 +135,11 @@ $(document).ready(function() {
     $.getJSON("/api/v1/topology/summary",function(response,status,jqXHR) {
       $.get("/templates/index-page-template.html", function(template) {
           topologySummary.append(Mustache.render($(template).filter("#topology-summary-template").html(),response));
-          //name, id, owner, status, uptime, num workers, num executors, num tasks, scheduler info
+          //name, owner, status, uptime, num workers, num executors, num tasks, replication count, assigned total mem, assigned total cpu, scheduler info
           dtAutoPage("#topology-summary-table", {
             columnDefs: [
-              {type: "num", targets: [5, 6, 7]},
-              {type: "time-str", targets: [4]}
+              {type: "num", targets: [4, 5, 6, 7, 8, 9]},
+              {type: "time-str", targets: [3]}
             ]
           });
           $('#topology-summary [data-toggle="tooltip"]').tooltip();

http://git-wip-us.apache.org/repos/asf/storm/blob/bed92d6f/storm-core/src/ui/public/templates/index-page-template.html
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/templates/index-page-template.html b/storm-core/src/ui/public/templates/index-page-template.html
index 47bcdfd..a5a885b 100644
--- a/storm-core/src/ui/public/templates/index-page-template.html
+++ b/storm-core/src/ui/public/templates/index-page-template.html
@@ -150,8 +150,8 @@
           </span>
         </th>
         <th>
-          <span data-toggle="tooltip" data-placement="left" title="This shows information from the scheduler about the latest attempt to schedule the Topology on the cluster.">
-            Scheduler Info
+          <span data-toggle="tooltip" data-placement="above" title="Assigned Total Memory by Scheduler.">
+            Assigned Mem (MB)
           </span>
         </th>
         <th>
@@ -170,7 +170,6 @@
       {{#topologies}}
       <tr>
         <td><a href="/topology.html?id={{encodedId}}">{{name}}</a></td>
-        <td>{{id}}</td>
         <td>{{owner}}</td>
         <td>{{status}}</td>
         <td>{{uptime}}</td>
@@ -178,9 +177,9 @@
         <td>{{executorsTotal}}</td>
         <td>{{tasksTotal}}</td>
         <td>{{replicationCount}}</td>
-        <td>{{schedulerInfo}}</td>
         <td>{{assignedTotalMem}}</td>
         <td>{{assignedTotalCpu}}</td>
+        <td>{{schedulerInfo}}</td>
       </tr>
       {{/topologies}}
     </tbody>


[08/12] storm git commit: Fix the nits in document

Posted by da...@apache.org.
Fix the nits in document


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/1d488d9c
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/1d488d9c
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/1d488d9c

Branch: refs/heads/master
Commit: 1d488d9c1638e8ac27b8363f725fc79d7e56342d
Parents: f6149c8
Author: zhuol <zh...@yahoo-inc.com>
Authored: Fri Oct 30 15:53:19 2015 -0500
Committer: zhuol <zh...@yahoo-inc.com>
Committed: Fri Oct 30 15:53:19 2015 -0500

----------------------------------------------------------------------
 STORM-UI-REST-API.md | 16 ++++++++--------
 1 file changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/1d488d9c/STORM-UI-REST-API.md
----------------------------------------------------------------------
diff --git a/STORM-UI-REST-API.md b/STORM-UI-REST-API.md
index f081337..ba31179 100644
--- a/STORM-UI-REST-API.md
+++ b/STORM-UI-REST-API.md
@@ -212,14 +212,14 @@ Sample response:
             "tasksTotal": 28,
             "workersTotal": 3,
             "executorsTotal": 28,
-            "replicationCount": 1
-            "requestedMemOnHeap": 640
-            "requestedMemOffHeap": 128
-            "requestedTotalMem": 768
-            "requestedCpu": 80
-            "assignedMemOnHeap": 640
-            "assignedMemOffHeap": 128
-            "assignedTotalMem": 768
+            "replicationCount": 1,
+            "requestedMemOnHeap": 640,
+            "requestedMemOffHeap": 128,
+            "requestedTotalMem": 768,
+            "requestedCpu": 80,
+            "assignedMemOnHeap": 640,
+            "assignedMemOffHeap": 128,
+            "assignedTotalMem": 768,
             "assignedCpu": 80
         }
     ]


[11/12] storm git commit: STORM-1144

Posted by da...@apache.org.
STORM-1144


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/3502907f
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/3502907f
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/3502907f

Branch: refs/heads/master
Commit: 3502907fac5b454a878e29fbdefe73ff9c2327b7
Parents: 090991d
Author: Derek Dagit <de...@yahoo-inc.com>
Authored: Fri Oct 30 16:16:33 2015 -0500
Committer: Derek Dagit <de...@yahoo-inc.com>
Committed: Fri Oct 30 16:16:33 2015 -0500

----------------------------------------------------------------------
 CHANGELOG.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/3502907f/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index b3dd875..4d5c40f 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -102,6 +102,7 @@
  * STORM-949: On the topology summary UI page, last shown error should have the time and date
  * STORM-1142: Some config validators for positive ints need to allow 0
  * STORM-901: Worker Artifacts Directory
+ * STORM-1144: Display requested and assigned cpu/mem resources for schedulers in UI
 
 ## 0.10.0-beta2
  * STORM-1108: Fix NPE in simulated time


[04/12] storm git commit: Update unit test

Posted by da...@apache.org.
Update unit test


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/3688fa80
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/3688fa80
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/3688fa80

Branch: refs/heads/master
Commit: 3688fa80416fd8b8422a52c4754fdca9000c603d
Parents: bed92d6
Author: zhuol <zh...@yahoo-inc.com>
Authored: Thu Oct 29 15:09:18 2015 -0500
Committer: zhuol <zh...@yahoo-inc.com>
Committed: Thu Oct 29 15:36:07 2015 -0500

----------------------------------------------------------------------
 .../storm/scheduler/resource_aware_scheduler_test.clj     | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/3688fa80/storm-core/test/clj/backtype/storm/scheduler/resource_aware_scheduler_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/scheduler/resource_aware_scheduler_test.clj b/storm-core/test/clj/backtype/storm/scheduler/resource_aware_scheduler_test.clj
index bd097a5..aec297d 100644
--- a/storm-core/test/clj/backtype/storm/scheduler/resource_aware_scheduler_test.clj
+++ b/storm-core/test/clj/backtype/storm/scheduler/resource_aware_scheduler_test.clj
@@ -171,7 +171,7 @@
       (is (= 1 (.size assigned-slots)))
       (is (= 1 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot))))))
       (is (= 2 (.size executors))))
-    (is (= "topology1 Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))))
+    (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))))
 
 (deftest test-topology-with-multiple-spouts
   (let [builder1 (TopologyBuilder.)  ;; a topology with multiple spouts
@@ -230,14 +230,14 @@
       (is (= 1 (.size assigned-slots)))
       (is (= 1 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot))))))
       (is (= 7 (.size executors))))
-    (is (= "topology1 Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))
+    (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))
     (let [assignment (.getAssignmentById cluster "topology2")
           assigned-slots (.getSlots assignment)
           executors (.getExecutors assignment)]
       (is (= 1 (.size assigned-slots)))
       (is (= 1 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot))))))
       (is (= 2 (.size executors))))
-    (is (= "topology2 Fully Scheduled" (.get (.getStatusMap cluster) "topology2")))))
+    (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology2")))))
 
 (deftest test-topology-set-memory-and-cpu-load
   (let [builder (TopologyBuilder.)
@@ -273,7 +273,7 @@
       (is (= 1 (.size assigned-slots)))
       (is (= 1 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot))))))
       (is (= 2 (.size executors))))
-    (is (= "topology2 Fully Scheduled" (.get (.getStatusMap cluster) "topology2")))))
+    (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology2")))))
 
 (deftest test-resource-limitation
   (let [builder (TopologyBuilder.)
@@ -335,7 +335,7 @@
       (is (>= avail used)))
     (doseq [[avail used] cpu-avail->used] ;; for each node, assigned cpu smaller than total
       (is (>= avail used))))
-  (is (= "topology1 Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))))
+  (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))))
 
 (deftest test-scheduling-resilience
   (let [supers (gen-supervisors 2 2)


[03/12] storm git commit: Update the thrift java files

Posted by da...@apache.org.
Update the thrift java files

Minor fix


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/416686ec
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/416686ec
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/416686ec

Branch: refs/heads/master
Commit: 416686ecceef66829c98bf90d112dea5c1870e7b
Parents: 70f455f
Author: zhuol <zh...@yahoo-inc.com>
Authored: Thu Oct 29 14:03:17 2015 -0500
Committer: zhuol <zh...@yahoo-inc.com>
Committed: Thu Oct 29 15:36:05 2015 -0500

----------------------------------------------------------------------
 .../src/clj/backtype/storm/daemon/nimbus.clj    |    2 +-
 .../backtype/storm/generated/DebugOptions.java  |    2 +-
 .../backtype/storm/generated/NimbusSummary.java |    2 +-
 .../backtype/storm/generated/TopologyInfo.java  |  604 ++++++++-
 .../storm/generated/TopologyPageInfo.java       |  606 ++++++++-
 .../storm/generated/TopologySummary.java        |  606 ++++++++-
 storm-core/src/py/storm/ttypes.py               | 1145 +++++++++++++++---
 7 files changed, 2749 insertions(+), 218 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/416686ec/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index 30a8677..faf7cde 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -1606,7 +1606,7 @@
             (.set_owner topo-page-info owner))
           (when-let [sched-status (.get @(:id->sched-status nimbus) topo-id)]
             (.set_sched_status topo-page-info sched-status))
-          (when-let [resources (.get @(:id->resources nimbus) storm-id)]
+          (when-let [resources (.get @(:id->resources nimbus) topo-id)]
             (.set_requested_memonheap topo-page-info (get resources 0))
             (.set_requested_memoffheap topo-page-info (get resources 1))
             (.set_requested_cpu topo-page-info (get resources 2))

http://git-wip-us.apache.org/repos/asf/storm/blob/416686ec/storm-core/src/jvm/backtype/storm/generated/DebugOptions.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/DebugOptions.java b/storm-core/src/jvm/backtype/storm/generated/DebugOptions.java
index da7a45a..315fbf1 100644
--- a/storm-core/src/jvm/backtype/storm/generated/DebugOptions.java
+++ b/storm-core/src/jvm/backtype/storm/generated/DebugOptions.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-29")
 public class DebugOptions implements org.apache.thrift.TBase<DebugOptions, DebugOptions._Fields>, java.io.Serializable, Cloneable, Comparable<DebugOptions> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DebugOptions");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/416686ec/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java b/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java
index 7d1e2fd..a8beace 100644
--- a/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java
+++ b/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-29")
 public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, NimbusSummary._Fields>, java.io.Serializable, Cloneable, Comparable<NimbusSummary> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NimbusSummary");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/416686ec/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java b/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
index 4f78417..9a4111f 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-29")
 public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, TopologyInfo._Fields>, java.io.Serializable, Cloneable, Comparable<TopologyInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologyInfo");
 
@@ -65,6 +65,12 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
   private static final org.apache.thrift.protocol.TField SCHED_STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("sched_status", org.apache.thrift.protocol.TType.STRING, (short)513);
   private static final org.apache.thrift.protocol.TField OWNER_FIELD_DESC = new org.apache.thrift.protocol.TField("owner", org.apache.thrift.protocol.TType.STRING, (short)514);
   private static final org.apache.thrift.protocol.TField REPLICATION_COUNT_FIELD_DESC = new org.apache.thrift.protocol.TField("replication_count", org.apache.thrift.protocol.TType.I32, (short)515);
+  private static final org.apache.thrift.protocol.TField REQUESTED_MEMONHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_memonheap", org.apache.thrift.protocol.TType.DOUBLE, (short)521);
+  private static final org.apache.thrift.protocol.TField REQUESTED_MEMOFFHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_memoffheap", org.apache.thrift.protocol.TType.DOUBLE, (short)522);
+  private static final org.apache.thrift.protocol.TField REQUESTED_CPU_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_cpu", org.apache.thrift.protocol.TType.DOUBLE, (short)523);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_MEMONHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_memonheap", org.apache.thrift.protocol.TType.DOUBLE, (short)524);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_MEMOFFHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_memoffheap", org.apache.thrift.protocol.TType.DOUBLE, (short)525);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_CPU_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_cpu", org.apache.thrift.protocol.TType.DOUBLE, (short)526);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -82,6 +88,12 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
   private String sched_status; // optional
   private String owner; // optional
   private int replication_count; // optional
+  private double requested_memonheap; // optional
+  private double requested_memoffheap; // optional
+  private double requested_cpu; // optional
+  private double assigned_memonheap; // optional
+  private double assigned_memoffheap; // optional
+  private double assigned_cpu; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -94,7 +106,13 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     COMPONENT_DEBUG((short)7, "component_debug"),
     SCHED_STATUS((short)513, "sched_status"),
     OWNER((short)514, "owner"),
-    REPLICATION_COUNT((short)515, "replication_count");
+    REPLICATION_COUNT((short)515, "replication_count"),
+    REQUESTED_MEMONHEAP((short)521, "requested_memonheap"),
+    REQUESTED_MEMOFFHEAP((short)522, "requested_memoffheap"),
+    REQUESTED_CPU((short)523, "requested_cpu"),
+    ASSIGNED_MEMONHEAP((short)524, "assigned_memonheap"),
+    ASSIGNED_MEMOFFHEAP((short)525, "assigned_memoffheap"),
+    ASSIGNED_CPU((short)526, "assigned_cpu");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -129,6 +147,18 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
           return OWNER;
         case 515: // REPLICATION_COUNT
           return REPLICATION_COUNT;
+        case 521: // REQUESTED_MEMONHEAP
+          return REQUESTED_MEMONHEAP;
+        case 522: // REQUESTED_MEMOFFHEAP
+          return REQUESTED_MEMOFFHEAP;
+        case 523: // REQUESTED_CPU
+          return REQUESTED_CPU;
+        case 524: // ASSIGNED_MEMONHEAP
+          return ASSIGNED_MEMONHEAP;
+        case 525: // ASSIGNED_MEMOFFHEAP
+          return ASSIGNED_MEMOFFHEAP;
+        case 526: // ASSIGNED_CPU
+          return ASSIGNED_CPU;
         default:
           return null;
       }
@@ -171,8 +201,14 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
   // isset id assignments
   private static final int __UPTIME_SECS_ISSET_ID = 0;
   private static final int __REPLICATION_COUNT_ISSET_ID = 1;
+  private static final int __REQUESTED_MEMONHEAP_ISSET_ID = 2;
+  private static final int __REQUESTED_MEMOFFHEAP_ISSET_ID = 3;
+  private static final int __REQUESTED_CPU_ISSET_ID = 4;
+  private static final int __ASSIGNED_MEMONHEAP_ISSET_ID = 5;
+  private static final int __ASSIGNED_MEMOFFHEAP_ISSET_ID = 6;
+  private static final int __ASSIGNED_CPU_ISSET_ID = 7;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.COMPONENT_DEBUG,_Fields.SCHED_STATUS,_Fields.OWNER,_Fields.REPLICATION_COUNT};
+  private static final _Fields optionals[] = {_Fields.COMPONENT_DEBUG,_Fields.SCHED_STATUS,_Fields.OWNER,_Fields.REPLICATION_COUNT,_Fields.REQUESTED_MEMONHEAP,_Fields.REQUESTED_MEMOFFHEAP,_Fields.REQUESTED_CPU,_Fields.ASSIGNED_MEMONHEAP,_Fields.ASSIGNED_MEMOFFHEAP,_Fields.ASSIGNED_CPU};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -202,6 +238,18 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.REPLICATION_COUNT, new org.apache.thrift.meta_data.FieldMetaData("replication_count", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.REQUESTED_MEMONHEAP, new org.apache.thrift.meta_data.FieldMetaData("requested_memonheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.REQUESTED_MEMOFFHEAP, new org.apache.thrift.meta_data.FieldMetaData("requested_memoffheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.REQUESTED_CPU, new org.apache.thrift.meta_data.FieldMetaData("requested_cpu", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_MEMONHEAP, new org.apache.thrift.meta_data.FieldMetaData("assigned_memonheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_MEMOFFHEAP, new org.apache.thrift.meta_data.FieldMetaData("assigned_memoffheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_CPU, new org.apache.thrift.meta_data.FieldMetaData("assigned_cpu", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologyInfo.class, metaDataMap);
   }
@@ -289,6 +337,12 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       this.owner = other.owner;
     }
     this.replication_count = other.replication_count;
+    this.requested_memonheap = other.requested_memonheap;
+    this.requested_memoffheap = other.requested_memoffheap;
+    this.requested_cpu = other.requested_cpu;
+    this.assigned_memonheap = other.assigned_memonheap;
+    this.assigned_memoffheap = other.assigned_memoffheap;
+    this.assigned_cpu = other.assigned_cpu;
   }
 
   public TopologyInfo deepCopy() {
@@ -309,6 +363,18 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     this.owner = null;
     set_replication_count_isSet(false);
     this.replication_count = 0;
+    set_requested_memonheap_isSet(false);
+    this.requested_memonheap = 0.0;
+    set_requested_memoffheap_isSet(false);
+    this.requested_memoffheap = 0.0;
+    set_requested_cpu_isSet(false);
+    this.requested_cpu = 0.0;
+    set_assigned_memonheap_isSet(false);
+    this.assigned_memonheap = 0.0;
+    set_assigned_memoffheap_isSet(false);
+    this.assigned_memoffheap = 0.0;
+    set_assigned_cpu_isSet(false);
+    this.assigned_cpu = 0.0;
   }
 
   public String get_id() {
@@ -576,6 +642,138 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID, value);
   }
 
+  public double get_requested_memonheap() {
+    return this.requested_memonheap;
+  }
+
+  public void set_requested_memonheap(double requested_memonheap) {
+    this.requested_memonheap = requested_memonheap;
+    set_requested_memonheap_isSet(true);
+  }
+
+  public void unset_requested_memonheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field requested_memonheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_memonheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID);
+  }
+
+  public void set_requested_memonheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID, value);
+  }
+
+  public double get_requested_memoffheap() {
+    return this.requested_memoffheap;
+  }
+
+  public void set_requested_memoffheap(double requested_memoffheap) {
+    this.requested_memoffheap = requested_memoffheap;
+    set_requested_memoffheap_isSet(true);
+  }
+
+  public void unset_requested_memoffheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field requested_memoffheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_memoffheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  public void set_requested_memoffheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID, value);
+  }
+
+  public double get_requested_cpu() {
+    return this.requested_cpu;
+  }
+
+  public void set_requested_cpu(double requested_cpu) {
+    this.requested_cpu = requested_cpu;
+    set_requested_cpu_isSet(true);
+  }
+
+  public void unset_requested_cpu() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID);
+  }
+
+  /** Returns true if field requested_cpu is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_cpu() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID);
+  }
+
+  public void set_requested_cpu_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID, value);
+  }
+
+  public double get_assigned_memonheap() {
+    return this.assigned_memonheap;
+  }
+
+  public void set_assigned_memonheap(double assigned_memonheap) {
+    this.assigned_memonheap = assigned_memonheap;
+    set_assigned_memonheap_isSet(true);
+  }
+
+  public void unset_assigned_memonheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_memonheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_memonheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID);
+  }
+
+  public void set_assigned_memonheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID, value);
+  }
+
+  public double get_assigned_memoffheap() {
+    return this.assigned_memoffheap;
+  }
+
+  public void set_assigned_memoffheap(double assigned_memoffheap) {
+    this.assigned_memoffheap = assigned_memoffheap;
+    set_assigned_memoffheap_isSet(true);
+  }
+
+  public void unset_assigned_memoffheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_memoffheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_memoffheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  public void set_assigned_memoffheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID, value);
+  }
+
+  public double get_assigned_cpu() {
+    return this.assigned_cpu;
+  }
+
+  public void set_assigned_cpu(double assigned_cpu) {
+    this.assigned_cpu = assigned_cpu;
+    set_assigned_cpu_isSet(true);
+  }
+
+  public void unset_assigned_cpu() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_cpu is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_cpu() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID);
+  }
+
+  public void set_assigned_cpu_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case ID:
@@ -658,6 +856,54 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       }
       break;
 
+    case REQUESTED_MEMONHEAP:
+      if (value == null) {
+        unset_requested_memonheap();
+      } else {
+        set_requested_memonheap((Double)value);
+      }
+      break;
+
+    case REQUESTED_MEMOFFHEAP:
+      if (value == null) {
+        unset_requested_memoffheap();
+      } else {
+        set_requested_memoffheap((Double)value);
+      }
+      break;
+
+    case REQUESTED_CPU:
+      if (value == null) {
+        unset_requested_cpu();
+      } else {
+        set_requested_cpu((Double)value);
+      }
+      break;
+
+    case ASSIGNED_MEMONHEAP:
+      if (value == null) {
+        unset_assigned_memonheap();
+      } else {
+        set_assigned_memonheap((Double)value);
+      }
+      break;
+
+    case ASSIGNED_MEMOFFHEAP:
+      if (value == null) {
+        unset_assigned_memoffheap();
+      } else {
+        set_assigned_memoffheap((Double)value);
+      }
+      break;
+
+    case ASSIGNED_CPU:
+      if (value == null) {
+        unset_assigned_cpu();
+      } else {
+        set_assigned_cpu((Double)value);
+      }
+      break;
+
     }
   }
 
@@ -693,6 +939,24 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     case REPLICATION_COUNT:
       return Integer.valueOf(get_replication_count());
 
+    case REQUESTED_MEMONHEAP:
+      return Double.valueOf(get_requested_memonheap());
+
+    case REQUESTED_MEMOFFHEAP:
+      return Double.valueOf(get_requested_memoffheap());
+
+    case REQUESTED_CPU:
+      return Double.valueOf(get_requested_cpu());
+
+    case ASSIGNED_MEMONHEAP:
+      return Double.valueOf(get_assigned_memonheap());
+
+    case ASSIGNED_MEMOFFHEAP:
+      return Double.valueOf(get_assigned_memoffheap());
+
+    case ASSIGNED_CPU:
+      return Double.valueOf(get_assigned_cpu());
+
     }
     throw new IllegalStateException();
   }
@@ -724,6 +988,18 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       return is_set_owner();
     case REPLICATION_COUNT:
       return is_set_replication_count();
+    case REQUESTED_MEMONHEAP:
+      return is_set_requested_memonheap();
+    case REQUESTED_MEMOFFHEAP:
+      return is_set_requested_memoffheap();
+    case REQUESTED_CPU:
+      return is_set_requested_cpu();
+    case ASSIGNED_MEMONHEAP:
+      return is_set_assigned_memonheap();
+    case ASSIGNED_MEMOFFHEAP:
+      return is_set_assigned_memoffheap();
+    case ASSIGNED_CPU:
+      return is_set_assigned_cpu();
     }
     throw new IllegalStateException();
   }
@@ -831,6 +1107,60 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         return false;
     }
 
+    boolean this_present_requested_memonheap = true && this.is_set_requested_memonheap();
+    boolean that_present_requested_memonheap = true && that.is_set_requested_memonheap();
+    if (this_present_requested_memonheap || that_present_requested_memonheap) {
+      if (!(this_present_requested_memonheap && that_present_requested_memonheap))
+        return false;
+      if (this.requested_memonheap != that.requested_memonheap)
+        return false;
+    }
+
+    boolean this_present_requested_memoffheap = true && this.is_set_requested_memoffheap();
+    boolean that_present_requested_memoffheap = true && that.is_set_requested_memoffheap();
+    if (this_present_requested_memoffheap || that_present_requested_memoffheap) {
+      if (!(this_present_requested_memoffheap && that_present_requested_memoffheap))
+        return false;
+      if (this.requested_memoffheap != that.requested_memoffheap)
+        return false;
+    }
+
+    boolean this_present_requested_cpu = true && this.is_set_requested_cpu();
+    boolean that_present_requested_cpu = true && that.is_set_requested_cpu();
+    if (this_present_requested_cpu || that_present_requested_cpu) {
+      if (!(this_present_requested_cpu && that_present_requested_cpu))
+        return false;
+      if (this.requested_cpu != that.requested_cpu)
+        return false;
+    }
+
+    boolean this_present_assigned_memonheap = true && this.is_set_assigned_memonheap();
+    boolean that_present_assigned_memonheap = true && that.is_set_assigned_memonheap();
+    if (this_present_assigned_memonheap || that_present_assigned_memonheap) {
+      if (!(this_present_assigned_memonheap && that_present_assigned_memonheap))
+        return false;
+      if (this.assigned_memonheap != that.assigned_memonheap)
+        return false;
+    }
+
+    boolean this_present_assigned_memoffheap = true && this.is_set_assigned_memoffheap();
+    boolean that_present_assigned_memoffheap = true && that.is_set_assigned_memoffheap();
+    if (this_present_assigned_memoffheap || that_present_assigned_memoffheap) {
+      if (!(this_present_assigned_memoffheap && that_present_assigned_memoffheap))
+        return false;
+      if (this.assigned_memoffheap != that.assigned_memoffheap)
+        return false;
+    }
+
+    boolean this_present_assigned_cpu = true && this.is_set_assigned_cpu();
+    boolean that_present_assigned_cpu = true && that.is_set_assigned_cpu();
+    if (this_present_assigned_cpu || that_present_assigned_cpu) {
+      if (!(this_present_assigned_cpu && that_present_assigned_cpu))
+        return false;
+      if (this.assigned_cpu != that.assigned_cpu)
+        return false;
+    }
+
     return true;
   }
 
@@ -888,6 +1218,36 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     if (present_replication_count)
       list.add(replication_count);
 
+    boolean present_requested_memonheap = true && (is_set_requested_memonheap());
+    list.add(present_requested_memonheap);
+    if (present_requested_memonheap)
+      list.add(requested_memonheap);
+
+    boolean present_requested_memoffheap = true && (is_set_requested_memoffheap());
+    list.add(present_requested_memoffheap);
+    if (present_requested_memoffheap)
+      list.add(requested_memoffheap);
+
+    boolean present_requested_cpu = true && (is_set_requested_cpu());
+    list.add(present_requested_cpu);
+    if (present_requested_cpu)
+      list.add(requested_cpu);
+
+    boolean present_assigned_memonheap = true && (is_set_assigned_memonheap());
+    list.add(present_assigned_memonheap);
+    if (present_assigned_memonheap)
+      list.add(assigned_memonheap);
+
+    boolean present_assigned_memoffheap = true && (is_set_assigned_memoffheap());
+    list.add(present_assigned_memoffheap);
+    if (present_assigned_memoffheap)
+      list.add(assigned_memoffheap);
+
+    boolean present_assigned_cpu = true && (is_set_assigned_cpu());
+    list.add(present_assigned_cpu);
+    if (present_assigned_cpu)
+      list.add(assigned_cpu);
+
     return list.hashCode();
   }
 
@@ -999,6 +1359,66 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(is_set_requested_memonheap()).compareTo(other.is_set_requested_memonheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_memonheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_memonheap, other.requested_memonheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_requested_memoffheap()).compareTo(other.is_set_requested_memoffheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_memoffheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_memoffheap, other.requested_memoffheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_requested_cpu()).compareTo(other.is_set_requested_cpu());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_cpu()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_cpu, other.requested_cpu);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_memonheap()).compareTo(other.is_set_assigned_memonheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_memonheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_memonheap, other.assigned_memonheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_memoffheap()).compareTo(other.is_set_assigned_memoffheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_memoffheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_memoffheap, other.assigned_memoffheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_cpu()).compareTo(other.is_set_assigned_cpu());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_cpu()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_cpu, other.assigned_cpu);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -1098,6 +1518,42 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       sb.append(this.replication_count);
       first = false;
     }
+    if (is_set_requested_memonheap()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_memonheap:");
+      sb.append(this.requested_memonheap);
+      first = false;
+    }
+    if (is_set_requested_memoffheap()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_memoffheap:");
+      sb.append(this.requested_memoffheap);
+      first = false;
+    }
+    if (is_set_requested_cpu()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_cpu:");
+      sb.append(this.requested_cpu);
+      first = false;
+    }
+    if (is_set_assigned_memonheap()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_memonheap:");
+      sb.append(this.assigned_memonheap);
+      first = false;
+    }
+    if (is_set_assigned_memoffheap()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_memoffheap:");
+      sb.append(this.assigned_memoffheap);
+      first = false;
+    }
+    if (is_set_assigned_cpu()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_cpu:");
+      sb.append(this.assigned_cpu);
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -1294,6 +1750,54 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 521: // REQUESTED_MEMONHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_memonheap = iprot.readDouble();
+              struct.set_requested_memonheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 522: // REQUESTED_MEMOFFHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_memoffheap = iprot.readDouble();
+              struct.set_requested_memoffheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 523: // REQUESTED_CPU
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_cpu = iprot.readDouble();
+              struct.set_requested_cpu_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 524: // ASSIGNED_MEMONHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_memonheap = iprot.readDouble();
+              struct.set_assigned_memonheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 525: // ASSIGNED_MEMOFFHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_memoffheap = iprot.readDouble();
+              struct.set_assigned_memoffheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 526: // ASSIGNED_CPU
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_cpu = iprot.readDouble();
+              struct.set_assigned_cpu_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -1391,6 +1895,36 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         oprot.writeI32(struct.replication_count);
         oprot.writeFieldEnd();
       }
+      if (struct.is_set_requested_memonheap()) {
+        oprot.writeFieldBegin(REQUESTED_MEMONHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.requested_memonheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        oprot.writeFieldBegin(REQUESTED_MEMOFFHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.requested_memoffheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_requested_cpu()) {
+        oprot.writeFieldBegin(REQUESTED_CPU_FIELD_DESC);
+        oprot.writeDouble(struct.requested_cpu);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        oprot.writeFieldBegin(ASSIGNED_MEMONHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_memonheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        oprot.writeFieldBegin(ASSIGNED_MEMOFFHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_memoffheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_cpu()) {
+        oprot.writeFieldBegin(ASSIGNED_CPU_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_cpu);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1446,7 +1980,25 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       if (struct.is_set_replication_count()) {
         optionals.set(3);
       }
-      oprot.writeBitSet(optionals, 4);
+      if (struct.is_set_requested_memonheap()) {
+        optionals.set(4);
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        optionals.set(5);
+      }
+      if (struct.is_set_requested_cpu()) {
+        optionals.set(6);
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        optionals.set(7);
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        optionals.set(8);
+      }
+      if (struct.is_set_assigned_cpu()) {
+        optionals.set(9);
+      }
+      oprot.writeBitSet(optionals, 10);
       if (struct.is_set_component_debug()) {
         {
           oprot.writeI32(struct.component_debug.size());
@@ -1466,6 +2018,24 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       if (struct.is_set_replication_count()) {
         oprot.writeI32(struct.replication_count);
       }
+      if (struct.is_set_requested_memonheap()) {
+        oprot.writeDouble(struct.requested_memonheap);
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        oprot.writeDouble(struct.requested_memoffheap);
+      }
+      if (struct.is_set_requested_cpu()) {
+        oprot.writeDouble(struct.requested_cpu);
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        oprot.writeDouble(struct.assigned_memonheap);
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        oprot.writeDouble(struct.assigned_memoffheap);
+      }
+      if (struct.is_set_assigned_cpu()) {
+        oprot.writeDouble(struct.assigned_cpu);
+      }
     }
 
     @Override
@@ -1514,7 +2084,7 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         }
       }
       struct.set_errors_isSet(true);
-      BitSet incoming = iprot.readBitSet(4);
+      BitSet incoming = iprot.readBitSet(10);
       if (incoming.get(0)) {
         {
           org.apache.thrift.protocol.TMap _map340 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
@@ -1543,6 +2113,30 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         struct.replication_count = iprot.readI32();
         struct.set_replication_count_isSet(true);
       }
+      if (incoming.get(4)) {
+        struct.requested_memonheap = iprot.readDouble();
+        struct.set_requested_memonheap_isSet(true);
+      }
+      if (incoming.get(5)) {
+        struct.requested_memoffheap = iprot.readDouble();
+        struct.set_requested_memoffheap_isSet(true);
+      }
+      if (incoming.get(6)) {
+        struct.requested_cpu = iprot.readDouble();
+        struct.set_requested_cpu_isSet(true);
+      }
+      if (incoming.get(7)) {
+        struct.assigned_memonheap = iprot.readDouble();
+        struct.set_assigned_memonheap_isSet(true);
+      }
+      if (incoming.get(8)) {
+        struct.assigned_memoffheap = iprot.readDouble();
+        struct.set_assigned_memoffheap_isSet(true);
+      }
+      if (incoming.get(9)) {
+        struct.assigned_cpu = iprot.readDouble();
+        struct.set_assigned_cpu_isSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/416686ec/storm-core/src/jvm/backtype/storm/generated/TopologyPageInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyPageInfo.java b/storm-core/src/jvm/backtype/storm/generated/TopologyPageInfo.java
index 180b608..257ec9d 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologyPageInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyPageInfo.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-29")
 public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInfo, TopologyPageInfo._Fields>, java.io.Serializable, Cloneable, Comparable<TopologyPageInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologyPageInfo");
 
@@ -70,6 +70,12 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
   private static final org.apache.thrift.protocol.TField OWNER_FIELD_DESC = new org.apache.thrift.protocol.TField("owner", org.apache.thrift.protocol.TType.STRING, (short)13);
   private static final org.apache.thrift.protocol.TField DEBUG_OPTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("debug_options", org.apache.thrift.protocol.TType.STRUCT, (short)14);
   private static final org.apache.thrift.protocol.TField REPLICATION_COUNT_FIELD_DESC = new org.apache.thrift.protocol.TField("replication_count", org.apache.thrift.protocol.TType.I32, (short)15);
+  private static final org.apache.thrift.protocol.TField REQUESTED_MEMONHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_memonheap", org.apache.thrift.protocol.TType.DOUBLE, (short)521);
+  private static final org.apache.thrift.protocol.TField REQUESTED_MEMOFFHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_memoffheap", org.apache.thrift.protocol.TType.DOUBLE, (short)522);
+  private static final org.apache.thrift.protocol.TField REQUESTED_CPU_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_cpu", org.apache.thrift.protocol.TType.DOUBLE, (short)523);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_MEMONHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_memonheap", org.apache.thrift.protocol.TType.DOUBLE, (short)524);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_MEMOFFHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_memoffheap", org.apache.thrift.protocol.TType.DOUBLE, (short)525);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_CPU_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_cpu", org.apache.thrift.protocol.TType.DOUBLE, (short)526);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -92,6 +98,12 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
   private String owner; // optional
   private DebugOptions debug_options; // optional
   private int replication_count; // optional
+  private double requested_memonheap; // optional
+  private double requested_memoffheap; // optional
+  private double requested_cpu; // optional
+  private double assigned_memonheap; // optional
+  private double assigned_memoffheap; // optional
+  private double assigned_cpu; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -109,7 +121,13 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
     TOPOLOGY_STATS((short)12, "topology_stats"),
     OWNER((short)13, "owner"),
     DEBUG_OPTIONS((short)14, "debug_options"),
-    REPLICATION_COUNT((short)15, "replication_count");
+    REPLICATION_COUNT((short)15, "replication_count"),
+    REQUESTED_MEMONHEAP((short)521, "requested_memonheap"),
+    REQUESTED_MEMOFFHEAP((short)522, "requested_memoffheap"),
+    REQUESTED_CPU((short)523, "requested_cpu"),
+    ASSIGNED_MEMONHEAP((short)524, "assigned_memonheap"),
+    ASSIGNED_MEMOFFHEAP((short)525, "assigned_memoffheap"),
+    ASSIGNED_CPU((short)526, "assigned_cpu");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -154,6 +172,18 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
           return DEBUG_OPTIONS;
         case 15: // REPLICATION_COUNT
           return REPLICATION_COUNT;
+        case 521: // REQUESTED_MEMONHEAP
+          return REQUESTED_MEMONHEAP;
+        case 522: // REQUESTED_MEMOFFHEAP
+          return REQUESTED_MEMOFFHEAP;
+        case 523: // REQUESTED_CPU
+          return REQUESTED_CPU;
+        case 524: // ASSIGNED_MEMONHEAP
+          return ASSIGNED_MEMONHEAP;
+        case 525: // ASSIGNED_MEMOFFHEAP
+          return ASSIGNED_MEMOFFHEAP;
+        case 526: // ASSIGNED_CPU
+          return ASSIGNED_CPU;
         default:
           return null;
       }
@@ -199,8 +229,14 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
   private static final int __NUM_WORKERS_ISSET_ID = 2;
   private static final int __NUM_EXECUTORS_ISSET_ID = 3;
   private static final int __REPLICATION_COUNT_ISSET_ID = 4;
-  private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.NAME,_Fields.UPTIME_SECS,_Fields.STATUS,_Fields.NUM_TASKS,_Fields.NUM_WORKERS,_Fields.NUM_EXECUTORS,_Fields.TOPOLOGY_CONF,_Fields.ID_TO_SPOUT_AGG_STATS,_Fields.ID_TO_BOLT_AGG_STATS,_Fields.SCHED_STATUS,_Fields.TOPOLOGY_STATS,_Fields.OWNER,_Fields.DEBUG_OPTIONS,_Fields.REPLICATION_COUNT};
+  private static final int __REQUESTED_MEMONHEAP_ISSET_ID = 5;
+  private static final int __REQUESTED_MEMOFFHEAP_ISSET_ID = 6;
+  private static final int __REQUESTED_CPU_ISSET_ID = 7;
+  private static final int __ASSIGNED_MEMONHEAP_ISSET_ID = 8;
+  private static final int __ASSIGNED_MEMOFFHEAP_ISSET_ID = 9;
+  private static final int __ASSIGNED_CPU_ISSET_ID = 10;
+  private short __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.NAME,_Fields.UPTIME_SECS,_Fields.STATUS,_Fields.NUM_TASKS,_Fields.NUM_WORKERS,_Fields.NUM_EXECUTORS,_Fields.TOPOLOGY_CONF,_Fields.ID_TO_SPOUT_AGG_STATS,_Fields.ID_TO_BOLT_AGG_STATS,_Fields.SCHED_STATUS,_Fields.TOPOLOGY_STATS,_Fields.OWNER,_Fields.DEBUG_OPTIONS,_Fields.REPLICATION_COUNT,_Fields.REQUESTED_MEMONHEAP,_Fields.REQUESTED_MEMOFFHEAP,_Fields.REQUESTED_CPU,_Fields.ASSIGNED_MEMONHEAP,_Fields.ASSIGNED_MEMOFFHEAP,_Fields.ASSIGNED_CPU};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -238,6 +274,18 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, DebugOptions.class)));
     tmpMap.put(_Fields.REPLICATION_COUNT, new org.apache.thrift.meta_data.FieldMetaData("replication_count", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.REQUESTED_MEMONHEAP, new org.apache.thrift.meta_data.FieldMetaData("requested_memonheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.REQUESTED_MEMOFFHEAP, new org.apache.thrift.meta_data.FieldMetaData("requested_memoffheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.REQUESTED_CPU, new org.apache.thrift.meta_data.FieldMetaData("requested_cpu", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_MEMONHEAP, new org.apache.thrift.meta_data.FieldMetaData("assigned_memonheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_MEMOFFHEAP, new org.apache.thrift.meta_data.FieldMetaData("assigned_memoffheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_CPU, new org.apache.thrift.meta_data.FieldMetaData("assigned_cpu", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologyPageInfo.class, metaDataMap);
   }
@@ -316,6 +364,12 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
       this.debug_options = new DebugOptions(other.debug_options);
     }
     this.replication_count = other.replication_count;
+    this.requested_memonheap = other.requested_memonheap;
+    this.requested_memoffheap = other.requested_memoffheap;
+    this.requested_cpu = other.requested_cpu;
+    this.assigned_memonheap = other.assigned_memonheap;
+    this.assigned_memoffheap = other.assigned_memoffheap;
+    this.assigned_cpu = other.assigned_cpu;
   }
 
   public TopologyPageInfo deepCopy() {
@@ -344,6 +398,18 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
     this.debug_options = null;
     set_replication_count_isSet(false);
     this.replication_count = 0;
+    set_requested_memonheap_isSet(false);
+    this.requested_memonheap = 0.0;
+    set_requested_memoffheap_isSet(false);
+    this.requested_memoffheap = 0.0;
+    set_requested_cpu_isSet(false);
+    this.requested_cpu = 0.0;
+    set_assigned_memonheap_isSet(false);
+    this.assigned_memonheap = 0.0;
+    set_assigned_memoffheap_isSet(false);
+    this.assigned_memoffheap = 0.0;
+    set_assigned_cpu_isSet(false);
+    this.assigned_cpu = 0.0;
   }
 
   public String get_id() {
@@ -708,6 +774,138 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID, value);
   }
 
+  public double get_requested_memonheap() {
+    return this.requested_memonheap;
+  }
+
+  public void set_requested_memonheap(double requested_memonheap) {
+    this.requested_memonheap = requested_memonheap;
+    set_requested_memonheap_isSet(true);
+  }
+
+  public void unset_requested_memonheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field requested_memonheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_memonheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID);
+  }
+
+  public void set_requested_memonheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID, value);
+  }
+
+  public double get_requested_memoffheap() {
+    return this.requested_memoffheap;
+  }
+
+  public void set_requested_memoffheap(double requested_memoffheap) {
+    this.requested_memoffheap = requested_memoffheap;
+    set_requested_memoffheap_isSet(true);
+  }
+
+  public void unset_requested_memoffheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field requested_memoffheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_memoffheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  public void set_requested_memoffheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID, value);
+  }
+
+  public double get_requested_cpu() {
+    return this.requested_cpu;
+  }
+
+  public void set_requested_cpu(double requested_cpu) {
+    this.requested_cpu = requested_cpu;
+    set_requested_cpu_isSet(true);
+  }
+
+  public void unset_requested_cpu() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID);
+  }
+
+  /** Returns true if field requested_cpu is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_cpu() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID);
+  }
+
+  public void set_requested_cpu_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID, value);
+  }
+
+  public double get_assigned_memonheap() {
+    return this.assigned_memonheap;
+  }
+
+  public void set_assigned_memonheap(double assigned_memonheap) {
+    this.assigned_memonheap = assigned_memonheap;
+    set_assigned_memonheap_isSet(true);
+  }
+
+  public void unset_assigned_memonheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_memonheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_memonheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID);
+  }
+
+  public void set_assigned_memonheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID, value);
+  }
+
+  public double get_assigned_memoffheap() {
+    return this.assigned_memoffheap;
+  }
+
+  public void set_assigned_memoffheap(double assigned_memoffheap) {
+    this.assigned_memoffheap = assigned_memoffheap;
+    set_assigned_memoffheap_isSet(true);
+  }
+
+  public void unset_assigned_memoffheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_memoffheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_memoffheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  public void set_assigned_memoffheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID, value);
+  }
+
+  public double get_assigned_cpu() {
+    return this.assigned_cpu;
+  }
+
+  public void set_assigned_cpu(double assigned_cpu) {
+    this.assigned_cpu = assigned_cpu;
+    set_assigned_cpu_isSet(true);
+  }
+
+  public void unset_assigned_cpu() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_cpu is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_cpu() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID);
+  }
+
+  public void set_assigned_cpu_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case ID:
@@ -830,6 +1028,54 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
       }
       break;
 
+    case REQUESTED_MEMONHEAP:
+      if (value == null) {
+        unset_requested_memonheap();
+      } else {
+        set_requested_memonheap((Double)value);
+      }
+      break;
+
+    case REQUESTED_MEMOFFHEAP:
+      if (value == null) {
+        unset_requested_memoffheap();
+      } else {
+        set_requested_memoffheap((Double)value);
+      }
+      break;
+
+    case REQUESTED_CPU:
+      if (value == null) {
+        unset_requested_cpu();
+      } else {
+        set_requested_cpu((Double)value);
+      }
+      break;
+
+    case ASSIGNED_MEMONHEAP:
+      if (value == null) {
+        unset_assigned_memonheap();
+      } else {
+        set_assigned_memonheap((Double)value);
+      }
+      break;
+
+    case ASSIGNED_MEMOFFHEAP:
+      if (value == null) {
+        unset_assigned_memoffheap();
+      } else {
+        set_assigned_memoffheap((Double)value);
+      }
+      break;
+
+    case ASSIGNED_CPU:
+      if (value == null) {
+        unset_assigned_cpu();
+      } else {
+        set_assigned_cpu((Double)value);
+      }
+      break;
+
     }
   }
 
@@ -880,6 +1126,24 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
     case REPLICATION_COUNT:
       return Integer.valueOf(get_replication_count());
 
+    case REQUESTED_MEMONHEAP:
+      return Double.valueOf(get_requested_memonheap());
+
+    case REQUESTED_MEMOFFHEAP:
+      return Double.valueOf(get_requested_memoffheap());
+
+    case REQUESTED_CPU:
+      return Double.valueOf(get_requested_cpu());
+
+    case ASSIGNED_MEMONHEAP:
+      return Double.valueOf(get_assigned_memonheap());
+
+    case ASSIGNED_MEMOFFHEAP:
+      return Double.valueOf(get_assigned_memoffheap());
+
+    case ASSIGNED_CPU:
+      return Double.valueOf(get_assigned_cpu());
+
     }
     throw new IllegalStateException();
   }
@@ -921,6 +1185,18 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
       return is_set_debug_options();
     case REPLICATION_COUNT:
       return is_set_replication_count();
+    case REQUESTED_MEMONHEAP:
+      return is_set_requested_memonheap();
+    case REQUESTED_MEMOFFHEAP:
+      return is_set_requested_memoffheap();
+    case REQUESTED_CPU:
+      return is_set_requested_cpu();
+    case ASSIGNED_MEMONHEAP:
+      return is_set_assigned_memonheap();
+    case ASSIGNED_MEMOFFHEAP:
+      return is_set_assigned_memoffheap();
+    case ASSIGNED_CPU:
+      return is_set_assigned_cpu();
     }
     throw new IllegalStateException();
   }
@@ -1073,6 +1349,60 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
         return false;
     }
 
+    boolean this_present_requested_memonheap = true && this.is_set_requested_memonheap();
+    boolean that_present_requested_memonheap = true && that.is_set_requested_memonheap();
+    if (this_present_requested_memonheap || that_present_requested_memonheap) {
+      if (!(this_present_requested_memonheap && that_present_requested_memonheap))
+        return false;
+      if (this.requested_memonheap != that.requested_memonheap)
+        return false;
+    }
+
+    boolean this_present_requested_memoffheap = true && this.is_set_requested_memoffheap();
+    boolean that_present_requested_memoffheap = true && that.is_set_requested_memoffheap();
+    if (this_present_requested_memoffheap || that_present_requested_memoffheap) {
+      if (!(this_present_requested_memoffheap && that_present_requested_memoffheap))
+        return false;
+      if (this.requested_memoffheap != that.requested_memoffheap)
+        return false;
+    }
+
+    boolean this_present_requested_cpu = true && this.is_set_requested_cpu();
+    boolean that_present_requested_cpu = true && that.is_set_requested_cpu();
+    if (this_present_requested_cpu || that_present_requested_cpu) {
+      if (!(this_present_requested_cpu && that_present_requested_cpu))
+        return false;
+      if (this.requested_cpu != that.requested_cpu)
+        return false;
+    }
+
+    boolean this_present_assigned_memonheap = true && this.is_set_assigned_memonheap();
+    boolean that_present_assigned_memonheap = true && that.is_set_assigned_memonheap();
+    if (this_present_assigned_memonheap || that_present_assigned_memonheap) {
+      if (!(this_present_assigned_memonheap && that_present_assigned_memonheap))
+        return false;
+      if (this.assigned_memonheap != that.assigned_memonheap)
+        return false;
+    }
+
+    boolean this_present_assigned_memoffheap = true && this.is_set_assigned_memoffheap();
+    boolean that_present_assigned_memoffheap = true && that.is_set_assigned_memoffheap();
+    if (this_present_assigned_memoffheap || that_present_assigned_memoffheap) {
+      if (!(this_present_assigned_memoffheap && that_present_assigned_memoffheap))
+        return false;
+      if (this.assigned_memoffheap != that.assigned_memoffheap)
+        return false;
+    }
+
+    boolean this_present_assigned_cpu = true && this.is_set_assigned_cpu();
+    boolean that_present_assigned_cpu = true && that.is_set_assigned_cpu();
+    if (this_present_assigned_cpu || that_present_assigned_cpu) {
+      if (!(this_present_assigned_cpu && that_present_assigned_cpu))
+        return false;
+      if (this.assigned_cpu != that.assigned_cpu)
+        return false;
+    }
+
     return true;
   }
 
@@ -1155,6 +1485,36 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
     if (present_replication_count)
       list.add(replication_count);
 
+    boolean present_requested_memonheap = true && (is_set_requested_memonheap());
+    list.add(present_requested_memonheap);
+    if (present_requested_memonheap)
+      list.add(requested_memonheap);
+
+    boolean present_requested_memoffheap = true && (is_set_requested_memoffheap());
+    list.add(present_requested_memoffheap);
+    if (present_requested_memoffheap)
+      list.add(requested_memoffheap);
+
+    boolean present_requested_cpu = true && (is_set_requested_cpu());
+    list.add(present_requested_cpu);
+    if (present_requested_cpu)
+      list.add(requested_cpu);
+
+    boolean present_assigned_memonheap = true && (is_set_assigned_memonheap());
+    list.add(present_assigned_memonheap);
+    if (present_assigned_memonheap)
+      list.add(assigned_memonheap);
+
+    boolean present_assigned_memoffheap = true && (is_set_assigned_memoffheap());
+    list.add(present_assigned_memoffheap);
+    if (present_assigned_memoffheap)
+      list.add(assigned_memoffheap);
+
+    boolean present_assigned_cpu = true && (is_set_assigned_cpu());
+    list.add(present_assigned_cpu);
+    if (present_assigned_cpu)
+      list.add(assigned_cpu);
+
     return list.hashCode();
   }
 
@@ -1316,6 +1676,66 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(is_set_requested_memonheap()).compareTo(other.is_set_requested_memonheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_memonheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_memonheap, other.requested_memonheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_requested_memoffheap()).compareTo(other.is_set_requested_memoffheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_memoffheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_memoffheap, other.requested_memoffheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_requested_cpu()).compareTo(other.is_set_requested_cpu());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_cpu()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_cpu, other.requested_cpu);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_memonheap()).compareTo(other.is_set_assigned_memonheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_memonheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_memonheap, other.assigned_memonheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_memoffheap()).compareTo(other.is_set_assigned_memoffheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_memoffheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_memoffheap, other.assigned_memoffheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_cpu()).compareTo(other.is_set_assigned_cpu());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_cpu()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_cpu, other.assigned_cpu);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -1463,6 +1883,42 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
       sb.append(this.replication_count);
       first = false;
     }
+    if (is_set_requested_memonheap()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_memonheap:");
+      sb.append(this.requested_memonheap);
+      first = false;
+    }
+    if (is_set_requested_memoffheap()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_memoffheap:");
+      sb.append(this.requested_memoffheap);
+      first = false;
+    }
+    if (is_set_requested_cpu()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_cpu:");
+      sb.append(this.requested_cpu);
+      first = false;
+    }
+    if (is_set_assigned_memonheap()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_memonheap:");
+      sb.append(this.assigned_memonheap);
+      first = false;
+    }
+    if (is_set_assigned_memoffheap()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_memoffheap:");
+      sb.append(this.assigned_memoffheap);
+      first = false;
+    }
+    if (is_set_assigned_cpu()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_cpu:");
+      sb.append(this.assigned_cpu);
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -1666,6 +2122,54 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 521: // REQUESTED_MEMONHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_memonheap = iprot.readDouble();
+              struct.set_requested_memonheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 522: // REQUESTED_MEMOFFHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_memoffheap = iprot.readDouble();
+              struct.set_requested_memoffheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 523: // REQUESTED_CPU
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_cpu = iprot.readDouble();
+              struct.set_requested_cpu_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 524: // ASSIGNED_MEMONHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_memonheap = iprot.readDouble();
+              struct.set_assigned_memonheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 525: // ASSIGNED_MEMOFFHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_memoffheap = iprot.readDouble();
+              struct.set_assigned_memoffheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 526: // ASSIGNED_CPU
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_cpu = iprot.readDouble();
+              struct.set_assigned_cpu_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -1788,6 +2292,36 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
         oprot.writeI32(struct.replication_count);
         oprot.writeFieldEnd();
       }
+      if (struct.is_set_requested_memonheap()) {
+        oprot.writeFieldBegin(REQUESTED_MEMONHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.requested_memonheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        oprot.writeFieldBegin(REQUESTED_MEMOFFHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.requested_memoffheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_requested_cpu()) {
+        oprot.writeFieldBegin(REQUESTED_CPU_FIELD_DESC);
+        oprot.writeDouble(struct.requested_cpu);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        oprot.writeFieldBegin(ASSIGNED_MEMONHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_memonheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        oprot.writeFieldBegin(ASSIGNED_MEMOFFHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_memoffheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_cpu()) {
+        oprot.writeFieldBegin(ASSIGNED_CPU_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_cpu);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1849,7 +2383,25 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
       if (struct.is_set_replication_count()) {
         optionals.set(13);
       }
-      oprot.writeBitSet(optionals, 14);
+      if (struct.is_set_requested_memonheap()) {
+        optionals.set(14);
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        optionals.set(15);
+      }
+      if (struct.is_set_requested_cpu()) {
+        optionals.set(16);
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        optionals.set(17);
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        optionals.set(18);
+      }
+      if (struct.is_set_assigned_cpu()) {
+        optionals.set(19);
+      }
+      oprot.writeBitSet(optionals, 20);
       if (struct.is_set_name()) {
         oprot.writeString(struct.name);
       }
@@ -1906,6 +2458,24 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
       if (struct.is_set_replication_count()) {
         oprot.writeI32(struct.replication_count);
       }
+      if (struct.is_set_requested_memonheap()) {
+        oprot.writeDouble(struct.requested_memonheap);
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        oprot.writeDouble(struct.requested_memoffheap);
+      }
+      if (struct.is_set_requested_cpu()) {
+        oprot.writeDouble(struct.requested_cpu);
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        oprot.writeDouble(struct.assigned_memonheap);
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        oprot.writeDouble(struct.assigned_memoffheap);
+      }
+      if (struct.is_set_assigned_cpu()) {
+        oprot.writeDouble(struct.assigned_cpu);
+      }
     }
 
     @Override
@@ -1913,7 +2483,7 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
       TTupleProtocol iprot = (TTupleProtocol) prot;
       struct.id = iprot.readString();
       struct.set_id_isSet(true);
-      BitSet incoming = iprot.readBitSet(14);
+      BitSet incoming = iprot.readBitSet(20);
       if (incoming.get(0)) {
         struct.name = iprot.readString();
         struct.set_name_isSet(true);
@@ -1996,6 +2566,30 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
         struct.replication_count = iprot.readI32();
         struct.set_replication_count_isSet(true);
       }
+      if (incoming.get(14)) {
+        struct.requested_memonheap = iprot.readDouble();
+        struct.set_requested_memonheap_isSet(true);
+      }
+      if (incoming.get(15)) {
+        struct.requested_memoffheap = iprot.readDouble();
+        struct.set_requested_memoffheap_isSet(true);
+      }
+      if (incoming.get(16)) {
+        struct.requested_cpu = iprot.readDouble();
+        struct.set_requested_cpu_isSet(true);
+      }
+      if (incoming.get(17)) {
+        struct.assigned_memonheap = iprot.readDouble();
+        struct.set_assigned_memonheap_isSet(true);
+      }
+      if (incoming.get(18)) {
+        struct.assigned_memoffheap = iprot.readDouble();
+        struct.set_assigned_memoffheap_isSet(true);
+      }
+      if (incoming.get(19)) {
+        struct.assigned_cpu = iprot.readDouble();
+        struct.set_assigned_cpu_isSet(true);
+      }
     }
   }
 


[06/12] storm git commit: Update the RESTful API doc for topology summary

Posted by da...@apache.org.
Update the RESTful API doc for topology summary


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/2ef1414c
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/2ef1414c
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/2ef1414c

Branch: refs/heads/master
Commit: 2ef1414cc0088dfea3f8e7396fc621de8d19d783
Parents: 3688fa8
Author: zhuol <zh...@yahoo-inc.com>
Authored: Fri Oct 30 14:55:23 2015 -0500
Committer: zhuol <zh...@yahoo-inc.com>
Committed: Fri Oct 30 14:55:23 2015 -0500

----------------------------------------------------------------------
 STORM-UI-REST-API.md                               | 17 +++++++++++++++++
 storm-core/src/clj/backtype/storm/ui/core.clj      |  2 +-
 .../ui/public/templates/index-page-template.html   |  2 +-
 .../public/templates/topology-page-template.html   |  6 +++---
 4 files changed, 22 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/2ef1414c/STORM-UI-REST-API.md
----------------------------------------------------------------------
diff --git a/STORM-UI-REST-API.md b/STORM-UI-REST-API.md
index 62167fb..f081337 100644
--- a/STORM-UI-REST-API.md
+++ b/STORM-UI-REST-API.md
@@ -189,6 +189,15 @@ Response fields:
 |workersTotal| Integer |Number of workers used for this topology|
 |executorsTotal| Integer |Number of executors used for this topology|
 |replicationCount| Integer |Number of nimbus hosts on which this topology code is replicated|
+|requestedMemOnHeap| Double|Requested On-Heap Memory by User (MB)
+|requestedMemOffHeap| Double|Requested Off-Heap Memory by User (MB)|
+|requestedTotalMem| Double|Requested Total Memory by User (MB)|
+|requestedCpu| Double|Requested CPU by User (%)|
+|assignedMemOnHeap| Double|Assigned On-Heap Memory by Scheduler (MB)|
+|assignedMemOffHeap| Double|Assigned Off-Heap Memory by Scheduler (MB)|
+|assignedTotalMem| Double|Assigned Total Memory by Scheduler (MB)|
+|assignedCpu| Double|Assigned CPU by Scheduler (%)|
+
 Sample response:
 
 ```json
@@ -204,6 +213,14 @@ Sample response:
             "workersTotal": 3,
             "executorsTotal": 28,
             "replicationCount": 1
+            "requestedMemOnHeap": 640
+            "requestedMemOffHeap": 128
+            "requestedTotalMem": 768
+            "requestedCpu": 80
+            "assignedMemOnHeap": 640
+            "assignedMemOffHeap": 128
+            "assignedTotalMem": 768
+            "assignedCpu": 80
         }
     ]
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/2ef1414c/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/ui/core.clj b/storm-core/src/clj/backtype/storm/ui/core.clj
index aaaeef0..9bfe76d 100644
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@ -415,7 +415,7 @@
        "assignedMemOnHeap" (.get_assigned_memonheap t)
        "assignedMemOffHeap" (.get_assigned_memoffheap t)
        "assignedTotalMem" (+ (.get_assigned_memonheap t) (.get_assigned_memoffheap t))
-       "assignedTotalCpu" (.get_assigned_cpu t)})}))
+       "assignedCpu" (.get_assigned_cpu t)})}))
 
 (defn topology-stats [window stats]
   (let [times (stats-times (:emitted stats))

http://git-wip-us.apache.org/repos/asf/storm/blob/2ef1414c/storm-core/src/ui/public/templates/index-page-template.html
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/templates/index-page-template.html b/storm-core/src/ui/public/templates/index-page-template.html
index a5a885b..5005f8d 100644
--- a/storm-core/src/ui/public/templates/index-page-template.html
+++ b/storm-core/src/ui/public/templates/index-page-template.html
@@ -178,7 +178,7 @@
         <td>{{tasksTotal}}</td>
         <td>{{replicationCount}}</td>
         <td>{{assignedTotalMem}}</td>
-        <td>{{assignedTotalCpu}}</td>
+        <td>{{assignedCpu}}</td>
         <td>{{schedulerInfo}}</td>
       </tr>
       {{/topologies}}

http://git-wip-us.apache.org/repos/asf/storm/blob/2ef1414c/storm-core/src/ui/public/templates/topology-page-template.html
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/templates/topology-page-template.html b/storm-core/src/ui/public/templates/topology-page-template.html
index 537f105..00e4fa2 100644
--- a/storm-core/src/ui/public/templates/topology-page-template.html
+++ b/storm-core/src/ui/public/templates/topology-page-template.html
@@ -101,7 +101,7 @@
           </span>
       </th>
       <th>
-          <span data-toggle="tooltip" data-placement="above" title="Requested Total On-Heap Memory by Scheduler.">
+          <span data-toggle="tooltip" data-placement="above" title="Requested Total On-Heap Memory by User.">
             Requested On-Heap Memory (MB)
           </span>
       </th>
@@ -111,7 +111,7 @@
           </span>
       </th>
       <th>
-          <span data-toggle="tooltip" data-placement="above" title="Requested Total Off-Heap Memory by Scheduler.">
+          <span data-toggle="tooltip" data-placement="above" title="Requested Total Off-Heap Memory by User.">
             Requested Off-Heap Memory (MB)
           </span>
       </th>
@@ -121,7 +121,7 @@
           </span>
       </th>
       <th>
-          <span data-toggle="tooltip" data-placement="above" title="Requested Total CPU by Scheduler. Every 100 means 1 core.">
+          <span data-toggle="tooltip" data-placement="above" title="Requested Total CPU by User. Every 100 means 1 core.">
             Requested CPU (%)
           </span>
       </th>


[09/12] storm git commit: Remove two date-only-change files

Posted by da...@apache.org.
Remove two date-only-change files


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

Branch: refs/heads/master
Commit: 9b9efc46d5e4fc538f59df88e5614d811f0d1e63
Parents: 1d488d9
Author: zhuol <zh...@yahoo-inc.com>
Authored: Fri Oct 30 16:02:56 2015 -0500
Committer: zhuol <zh...@yahoo-inc.com>
Committed: Fri Oct 30 16:02:56 2015 -0500

----------------------------------------------------------------------
 storm-core/src/jvm/backtype/storm/generated/DebugOptions.java  | 2 +-
 storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/9b9efc46/storm-core/src/jvm/backtype/storm/generated/DebugOptions.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/DebugOptions.java b/storm-core/src/jvm/backtype/storm/generated/DebugOptions.java
index 315fbf1..da7a45a 100644
--- a/storm-core/src/jvm/backtype/storm/generated/DebugOptions.java
+++ b/storm-core/src/jvm/backtype/storm/generated/DebugOptions.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-29")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 public class DebugOptions implements org.apache.thrift.TBase<DebugOptions, DebugOptions._Fields>, java.io.Serializable, Cloneable, Comparable<DebugOptions> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DebugOptions");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/9b9efc46/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java b/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java
index a8beace..7d1e2fd 100644
--- a/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java
+++ b/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-29")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, NimbusSummary._Fields>, java.io.Serializable, Cloneable, Comparable<NimbusSummary> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NimbusSummary");
 


[02/12] storm git commit: Update the thrift java files

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/416686ec/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java b/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java
index 055a01a..7bb2e44 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-29")
 public class TopologySummary implements org.apache.thrift.TBase<TopologySummary, TopologySummary._Fields>, java.io.Serializable, Cloneable, Comparable<TopologySummary> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologySummary");
 
@@ -65,6 +65,12 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
   private static final org.apache.thrift.protocol.TField SCHED_STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("sched_status", org.apache.thrift.protocol.TType.STRING, (short)513);
   private static final org.apache.thrift.protocol.TField OWNER_FIELD_DESC = new org.apache.thrift.protocol.TField("owner", org.apache.thrift.protocol.TType.STRING, (short)514);
   private static final org.apache.thrift.protocol.TField REPLICATION_COUNT_FIELD_DESC = new org.apache.thrift.protocol.TField("replication_count", org.apache.thrift.protocol.TType.I32, (short)515);
+  private static final org.apache.thrift.protocol.TField REQUESTED_MEMONHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_memonheap", org.apache.thrift.protocol.TType.DOUBLE, (short)521);
+  private static final org.apache.thrift.protocol.TField REQUESTED_MEMOFFHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_memoffheap", org.apache.thrift.protocol.TType.DOUBLE, (short)522);
+  private static final org.apache.thrift.protocol.TField REQUESTED_CPU_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_cpu", org.apache.thrift.protocol.TType.DOUBLE, (short)523);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_MEMONHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_memonheap", org.apache.thrift.protocol.TType.DOUBLE, (short)524);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_MEMOFFHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_memoffheap", org.apache.thrift.protocol.TType.DOUBLE, (short)525);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_CPU_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_cpu", org.apache.thrift.protocol.TType.DOUBLE, (short)526);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -82,6 +88,12 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
   private String sched_status; // optional
   private String owner; // optional
   private int replication_count; // optional
+  private double requested_memonheap; // optional
+  private double requested_memoffheap; // optional
+  private double requested_cpu; // optional
+  private double assigned_memonheap; // optional
+  private double assigned_memoffheap; // optional
+  private double assigned_cpu; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -94,7 +106,13 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
     STATUS((short)7, "status"),
     SCHED_STATUS((short)513, "sched_status"),
     OWNER((short)514, "owner"),
-    REPLICATION_COUNT((short)515, "replication_count");
+    REPLICATION_COUNT((short)515, "replication_count"),
+    REQUESTED_MEMONHEAP((short)521, "requested_memonheap"),
+    REQUESTED_MEMOFFHEAP((short)522, "requested_memoffheap"),
+    REQUESTED_CPU((short)523, "requested_cpu"),
+    ASSIGNED_MEMONHEAP((short)524, "assigned_memonheap"),
+    ASSIGNED_MEMOFFHEAP((short)525, "assigned_memoffheap"),
+    ASSIGNED_CPU((short)526, "assigned_cpu");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -129,6 +147,18 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
           return OWNER;
         case 515: // REPLICATION_COUNT
           return REPLICATION_COUNT;
+        case 521: // REQUESTED_MEMONHEAP
+          return REQUESTED_MEMONHEAP;
+        case 522: // REQUESTED_MEMOFFHEAP
+          return REQUESTED_MEMOFFHEAP;
+        case 523: // REQUESTED_CPU
+          return REQUESTED_CPU;
+        case 524: // ASSIGNED_MEMONHEAP
+          return ASSIGNED_MEMONHEAP;
+        case 525: // ASSIGNED_MEMOFFHEAP
+          return ASSIGNED_MEMOFFHEAP;
+        case 526: // ASSIGNED_CPU
+          return ASSIGNED_CPU;
         default:
           return null;
       }
@@ -174,8 +204,14 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
   private static final int __NUM_WORKERS_ISSET_ID = 2;
   private static final int __UPTIME_SECS_ISSET_ID = 3;
   private static final int __REPLICATION_COUNT_ISSET_ID = 4;
-  private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.SCHED_STATUS,_Fields.OWNER,_Fields.REPLICATION_COUNT};
+  private static final int __REQUESTED_MEMONHEAP_ISSET_ID = 5;
+  private static final int __REQUESTED_MEMOFFHEAP_ISSET_ID = 6;
+  private static final int __REQUESTED_CPU_ISSET_ID = 7;
+  private static final int __ASSIGNED_MEMONHEAP_ISSET_ID = 8;
+  private static final int __ASSIGNED_MEMOFFHEAP_ISSET_ID = 9;
+  private static final int __ASSIGNED_CPU_ISSET_ID = 10;
+  private short __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.SCHED_STATUS,_Fields.OWNER,_Fields.REPLICATION_COUNT,_Fields.REQUESTED_MEMONHEAP,_Fields.REQUESTED_MEMOFFHEAP,_Fields.REQUESTED_CPU,_Fields.ASSIGNED_MEMONHEAP,_Fields.ASSIGNED_MEMOFFHEAP,_Fields.ASSIGNED_CPU};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -199,6 +235,18 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.REPLICATION_COUNT, new org.apache.thrift.meta_data.FieldMetaData("replication_count", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.REQUESTED_MEMONHEAP, new org.apache.thrift.meta_data.FieldMetaData("requested_memonheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.REQUESTED_MEMOFFHEAP, new org.apache.thrift.meta_data.FieldMetaData("requested_memoffheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.REQUESTED_CPU, new org.apache.thrift.meta_data.FieldMetaData("requested_cpu", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_MEMONHEAP, new org.apache.thrift.meta_data.FieldMetaData("assigned_memonheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_MEMOFFHEAP, new org.apache.thrift.meta_data.FieldMetaData("assigned_memoffheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_CPU, new org.apache.thrift.meta_data.FieldMetaData("assigned_cpu", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologySummary.class, metaDataMap);
   }
@@ -254,6 +302,12 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
       this.owner = other.owner;
     }
     this.replication_count = other.replication_count;
+    this.requested_memonheap = other.requested_memonheap;
+    this.requested_memoffheap = other.requested_memoffheap;
+    this.requested_cpu = other.requested_cpu;
+    this.assigned_memonheap = other.assigned_memonheap;
+    this.assigned_memoffheap = other.assigned_memoffheap;
+    this.assigned_cpu = other.assigned_cpu;
   }
 
   public TopologySummary deepCopy() {
@@ -277,6 +331,18 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
     this.owner = null;
     set_replication_count_isSet(false);
     this.replication_count = 0;
+    set_requested_memonheap_isSet(false);
+    this.requested_memonheap = 0.0;
+    set_requested_memoffheap_isSet(false);
+    this.requested_memoffheap = 0.0;
+    set_requested_cpu_isSet(false);
+    this.requested_cpu = 0.0;
+    set_assigned_memonheap_isSet(false);
+    this.assigned_memonheap = 0.0;
+    set_assigned_memoffheap_isSet(false);
+    this.assigned_memoffheap = 0.0;
+    set_assigned_cpu_isSet(false);
+    this.assigned_cpu = 0.0;
   }
 
   public String get_id() {
@@ -504,6 +570,138 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID, value);
   }
 
+  public double get_requested_memonheap() {
+    return this.requested_memonheap;
+  }
+
+  public void set_requested_memonheap(double requested_memonheap) {
+    this.requested_memonheap = requested_memonheap;
+    set_requested_memonheap_isSet(true);
+  }
+
+  public void unset_requested_memonheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field requested_memonheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_memonheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID);
+  }
+
+  public void set_requested_memonheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID, value);
+  }
+
+  public double get_requested_memoffheap() {
+    return this.requested_memoffheap;
+  }
+
+  public void set_requested_memoffheap(double requested_memoffheap) {
+    this.requested_memoffheap = requested_memoffheap;
+    set_requested_memoffheap_isSet(true);
+  }
+
+  public void unset_requested_memoffheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field requested_memoffheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_memoffheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  public void set_requested_memoffheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID, value);
+  }
+
+  public double get_requested_cpu() {
+    return this.requested_cpu;
+  }
+
+  public void set_requested_cpu(double requested_cpu) {
+    this.requested_cpu = requested_cpu;
+    set_requested_cpu_isSet(true);
+  }
+
+  public void unset_requested_cpu() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID);
+  }
+
+  /** Returns true if field requested_cpu is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_cpu() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID);
+  }
+
+  public void set_requested_cpu_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID, value);
+  }
+
+  public double get_assigned_memonheap() {
+    return this.assigned_memonheap;
+  }
+
+  public void set_assigned_memonheap(double assigned_memonheap) {
+    this.assigned_memonheap = assigned_memonheap;
+    set_assigned_memonheap_isSet(true);
+  }
+
+  public void unset_assigned_memonheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_memonheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_memonheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID);
+  }
+
+  public void set_assigned_memonheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID, value);
+  }
+
+  public double get_assigned_memoffheap() {
+    return this.assigned_memoffheap;
+  }
+
+  public void set_assigned_memoffheap(double assigned_memoffheap) {
+    this.assigned_memoffheap = assigned_memoffheap;
+    set_assigned_memoffheap_isSet(true);
+  }
+
+  public void unset_assigned_memoffheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_memoffheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_memoffheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  public void set_assigned_memoffheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID, value);
+  }
+
+  public double get_assigned_cpu() {
+    return this.assigned_cpu;
+  }
+
+  public void set_assigned_cpu(double assigned_cpu) {
+    this.assigned_cpu = assigned_cpu;
+    set_assigned_cpu_isSet(true);
+  }
+
+  public void unset_assigned_cpu() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_cpu is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_cpu() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID);
+  }
+
+  public void set_assigned_cpu_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case ID:
@@ -586,6 +784,54 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
       }
       break;
 
+    case REQUESTED_MEMONHEAP:
+      if (value == null) {
+        unset_requested_memonheap();
+      } else {
+        set_requested_memonheap((Double)value);
+      }
+      break;
+
+    case REQUESTED_MEMOFFHEAP:
+      if (value == null) {
+        unset_requested_memoffheap();
+      } else {
+        set_requested_memoffheap((Double)value);
+      }
+      break;
+
+    case REQUESTED_CPU:
+      if (value == null) {
+        unset_requested_cpu();
+      } else {
+        set_requested_cpu((Double)value);
+      }
+      break;
+
+    case ASSIGNED_MEMONHEAP:
+      if (value == null) {
+        unset_assigned_memonheap();
+      } else {
+        set_assigned_memonheap((Double)value);
+      }
+      break;
+
+    case ASSIGNED_MEMOFFHEAP:
+      if (value == null) {
+        unset_assigned_memoffheap();
+      } else {
+        set_assigned_memoffheap((Double)value);
+      }
+      break;
+
+    case ASSIGNED_CPU:
+      if (value == null) {
+        unset_assigned_cpu();
+      } else {
+        set_assigned_cpu((Double)value);
+      }
+      break;
+
     }
   }
 
@@ -621,6 +867,24 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
     case REPLICATION_COUNT:
       return Integer.valueOf(get_replication_count());
 
+    case REQUESTED_MEMONHEAP:
+      return Double.valueOf(get_requested_memonheap());
+
+    case REQUESTED_MEMOFFHEAP:
+      return Double.valueOf(get_requested_memoffheap());
+
+    case REQUESTED_CPU:
+      return Double.valueOf(get_requested_cpu());
+
+    case ASSIGNED_MEMONHEAP:
+      return Double.valueOf(get_assigned_memonheap());
+
+    case ASSIGNED_MEMOFFHEAP:
+      return Double.valueOf(get_assigned_memoffheap());
+
+    case ASSIGNED_CPU:
+      return Double.valueOf(get_assigned_cpu());
+
     }
     throw new IllegalStateException();
   }
@@ -652,6 +916,18 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
       return is_set_owner();
     case REPLICATION_COUNT:
       return is_set_replication_count();
+    case REQUESTED_MEMONHEAP:
+      return is_set_requested_memonheap();
+    case REQUESTED_MEMOFFHEAP:
+      return is_set_requested_memoffheap();
+    case REQUESTED_CPU:
+      return is_set_requested_cpu();
+    case ASSIGNED_MEMONHEAP:
+      return is_set_assigned_memonheap();
+    case ASSIGNED_MEMOFFHEAP:
+      return is_set_assigned_memoffheap();
+    case ASSIGNED_CPU:
+      return is_set_assigned_cpu();
     }
     throw new IllegalStateException();
   }
@@ -759,6 +1035,60 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
         return false;
     }
 
+    boolean this_present_requested_memonheap = true && this.is_set_requested_memonheap();
+    boolean that_present_requested_memonheap = true && that.is_set_requested_memonheap();
+    if (this_present_requested_memonheap || that_present_requested_memonheap) {
+      if (!(this_present_requested_memonheap && that_present_requested_memonheap))
+        return false;
+      if (this.requested_memonheap != that.requested_memonheap)
+        return false;
+    }
+
+    boolean this_present_requested_memoffheap = true && this.is_set_requested_memoffheap();
+    boolean that_present_requested_memoffheap = true && that.is_set_requested_memoffheap();
+    if (this_present_requested_memoffheap || that_present_requested_memoffheap) {
+      if (!(this_present_requested_memoffheap && that_present_requested_memoffheap))
+        return false;
+      if (this.requested_memoffheap != that.requested_memoffheap)
+        return false;
+    }
+
+    boolean this_present_requested_cpu = true && this.is_set_requested_cpu();
+    boolean that_present_requested_cpu = true && that.is_set_requested_cpu();
+    if (this_present_requested_cpu || that_present_requested_cpu) {
+      if (!(this_present_requested_cpu && that_present_requested_cpu))
+        return false;
+      if (this.requested_cpu != that.requested_cpu)
+        return false;
+    }
+
+    boolean this_present_assigned_memonheap = true && this.is_set_assigned_memonheap();
+    boolean that_present_assigned_memonheap = true && that.is_set_assigned_memonheap();
+    if (this_present_assigned_memonheap || that_present_assigned_memonheap) {
+      if (!(this_present_assigned_memonheap && that_present_assigned_memonheap))
+        return false;
+      if (this.assigned_memonheap != that.assigned_memonheap)
+        return false;
+    }
+
+    boolean this_present_assigned_memoffheap = true && this.is_set_assigned_memoffheap();
+    boolean that_present_assigned_memoffheap = true && that.is_set_assigned_memoffheap();
+    if (this_present_assigned_memoffheap || that_present_assigned_memoffheap) {
+      if (!(this_present_assigned_memoffheap && that_present_assigned_memoffheap))
+        return false;
+      if (this.assigned_memoffheap != that.assigned_memoffheap)
+        return false;
+    }
+
+    boolean this_present_assigned_cpu = true && this.is_set_assigned_cpu();
+    boolean that_present_assigned_cpu = true && that.is_set_assigned_cpu();
+    if (this_present_assigned_cpu || that_present_assigned_cpu) {
+      if (!(this_present_assigned_cpu && that_present_assigned_cpu))
+        return false;
+      if (this.assigned_cpu != that.assigned_cpu)
+        return false;
+    }
+
     return true;
   }
 
@@ -816,6 +1146,36 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
     if (present_replication_count)
       list.add(replication_count);
 
+    boolean present_requested_memonheap = true && (is_set_requested_memonheap());
+    list.add(present_requested_memonheap);
+    if (present_requested_memonheap)
+      list.add(requested_memonheap);
+
+    boolean present_requested_memoffheap = true && (is_set_requested_memoffheap());
+    list.add(present_requested_memoffheap);
+    if (present_requested_memoffheap)
+      list.add(requested_memoffheap);
+
+    boolean present_requested_cpu = true && (is_set_requested_cpu());
+    list.add(present_requested_cpu);
+    if (present_requested_cpu)
+      list.add(requested_cpu);
+
+    boolean present_assigned_memonheap = true && (is_set_assigned_memonheap());
+    list.add(present_assigned_memonheap);
+    if (present_assigned_memonheap)
+      list.add(assigned_memonheap);
+
+    boolean present_assigned_memoffheap = true && (is_set_assigned_memoffheap());
+    list.add(present_assigned_memoffheap);
+    if (present_assigned_memoffheap)
+      list.add(assigned_memoffheap);
+
+    boolean present_assigned_cpu = true && (is_set_assigned_cpu());
+    list.add(present_assigned_cpu);
+    if (present_assigned_cpu)
+      list.add(assigned_cpu);
+
     return list.hashCode();
   }
 
@@ -927,6 +1287,66 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(is_set_requested_memonheap()).compareTo(other.is_set_requested_memonheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_memonheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_memonheap, other.requested_memonheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_requested_memoffheap()).compareTo(other.is_set_requested_memoffheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_memoffheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_memoffheap, other.requested_memoffheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_requested_cpu()).compareTo(other.is_set_requested_cpu());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_cpu()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_cpu, other.requested_cpu);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_memonheap()).compareTo(other.is_set_assigned_memonheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_memonheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_memonheap, other.assigned_memonheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_memoffheap()).compareTo(other.is_set_assigned_memoffheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_memoffheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_memoffheap, other.assigned_memoffheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_cpu()).compareTo(other.is_set_assigned_cpu());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_cpu()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_cpu, other.assigned_cpu);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -1012,6 +1432,42 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
       sb.append(this.replication_count);
       first = false;
     }
+    if (is_set_requested_memonheap()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_memonheap:");
+      sb.append(this.requested_memonheap);
+      first = false;
+    }
+    if (is_set_requested_memoffheap()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_memoffheap:");
+      sb.append(this.requested_memoffheap);
+      first = false;
+    }
+    if (is_set_requested_cpu()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_cpu:");
+      sb.append(this.requested_cpu);
+      first = false;
+    }
+    if (is_set_assigned_memonheap()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_memonheap:");
+      sb.append(this.assigned_memonheap);
+      first = false;
+    }
+    if (is_set_assigned_memoffheap()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_memoffheap:");
+      sb.append(this.assigned_memoffheap);
+      first = false;
+    }
+    if (is_set_assigned_cpu()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_cpu:");
+      sb.append(this.assigned_cpu);
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -1165,6 +1621,54 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 521: // REQUESTED_MEMONHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_memonheap = iprot.readDouble();
+              struct.set_requested_memonheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 522: // REQUESTED_MEMOFFHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_memoffheap = iprot.readDouble();
+              struct.set_requested_memoffheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 523: // REQUESTED_CPU
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_cpu = iprot.readDouble();
+              struct.set_requested_cpu_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 524: // ASSIGNED_MEMONHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_memonheap = iprot.readDouble();
+              struct.set_assigned_memonheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 525: // ASSIGNED_MEMOFFHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_memoffheap = iprot.readDouble();
+              struct.set_assigned_memoffheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 526: // ASSIGNED_CPU
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_cpu = iprot.readDouble();
+              struct.set_assigned_cpu_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -1224,6 +1728,36 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
         oprot.writeI32(struct.replication_count);
         oprot.writeFieldEnd();
       }
+      if (struct.is_set_requested_memonheap()) {
+        oprot.writeFieldBegin(REQUESTED_MEMONHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.requested_memonheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        oprot.writeFieldBegin(REQUESTED_MEMOFFHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.requested_memoffheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_requested_cpu()) {
+        oprot.writeFieldBegin(REQUESTED_CPU_FIELD_DESC);
+        oprot.writeDouble(struct.requested_cpu);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        oprot.writeFieldBegin(ASSIGNED_MEMONHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_memonheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        oprot.writeFieldBegin(ASSIGNED_MEMOFFHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_memoffheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_cpu()) {
+        oprot.writeFieldBegin(ASSIGNED_CPU_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_cpu);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1258,7 +1792,25 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
       if (struct.is_set_replication_count()) {
         optionals.set(2);
       }
-      oprot.writeBitSet(optionals, 3);
+      if (struct.is_set_requested_memonheap()) {
+        optionals.set(3);
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        optionals.set(4);
+      }
+      if (struct.is_set_requested_cpu()) {
+        optionals.set(5);
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        optionals.set(6);
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        optionals.set(7);
+      }
+      if (struct.is_set_assigned_cpu()) {
+        optionals.set(8);
+      }
+      oprot.writeBitSet(optionals, 9);
       if (struct.is_set_sched_status()) {
         oprot.writeString(struct.sched_status);
       }
@@ -1268,6 +1820,24 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
       if (struct.is_set_replication_count()) {
         oprot.writeI32(struct.replication_count);
       }
+      if (struct.is_set_requested_memonheap()) {
+        oprot.writeDouble(struct.requested_memonheap);
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        oprot.writeDouble(struct.requested_memoffheap);
+      }
+      if (struct.is_set_requested_cpu()) {
+        oprot.writeDouble(struct.requested_cpu);
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        oprot.writeDouble(struct.assigned_memonheap);
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        oprot.writeDouble(struct.assigned_memoffheap);
+      }
+      if (struct.is_set_assigned_cpu()) {
+        oprot.writeDouble(struct.assigned_cpu);
+      }
     }
 
     @Override
@@ -1287,7 +1857,7 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
       struct.set_uptime_secs_isSet(true);
       struct.status = iprot.readString();
       struct.set_status_isSet(true);
-      BitSet incoming = iprot.readBitSet(3);
+      BitSet incoming = iprot.readBitSet(9);
       if (incoming.get(0)) {
         struct.sched_status = iprot.readString();
         struct.set_sched_status_isSet(true);
@@ -1300,6 +1870,30 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
         struct.replication_count = iprot.readI32();
         struct.set_replication_count_isSet(true);
       }
+      if (incoming.get(3)) {
+        struct.requested_memonheap = iprot.readDouble();
+        struct.set_requested_memonheap_isSet(true);
+      }
+      if (incoming.get(4)) {
+        struct.requested_memoffheap = iprot.readDouble();
+        struct.set_requested_memoffheap_isSet(true);
+      }
+      if (incoming.get(5)) {
+        struct.requested_cpu = iprot.readDouble();
+        struct.set_requested_cpu_isSet(true);
+      }
+      if (incoming.get(6)) {
+        struct.assigned_memonheap = iprot.readDouble();
+        struct.set_assigned_memonheap_isSet(true);
+      }
+      if (incoming.get(7)) {
+        struct.assigned_memoffheap = iprot.readDouble();
+        struct.set_assigned_memoffheap_isSet(true);
+      }
+      if (incoming.get(8)) {
+        struct.assigned_cpu = iprot.readDouble();
+        struct.set_assigned_cpu_isSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/416686ec/storm-core/src/py/storm/ttypes.py
----------------------------------------------------------------------
diff --git a/storm-core/src/py/storm/ttypes.py b/storm-core/src/py/storm/ttypes.py
index f91442f..7f8adea 100644
--- a/storm-core/src/py/storm/ttypes.py
+++ b/storm-core/src/py/storm/ttypes.py
@@ -1703,6 +1703,12 @@ class TopologySummary:
    - sched_status
    - owner
    - replication_count
+   - requested_memonheap
+   - requested_memoffheap
+   - requested_cpu
+   - assigned_memonheap
+   - assigned_memoffheap
+   - assigned_cpu
   """
 
   thrift_spec = (
@@ -2222,9 +2228,20 @@ class TopologySummary:
     (513, TType.STRING, 'sched_status', None, None, ), # 513
     (514, TType.STRING, 'owner', None, None, ), # 514
     (515, TType.I32, 'replication_count', None, None, ), # 515
+    None, # 516
+    None, # 517
+    None, # 518
+    None, # 519
+    None, # 520
+    (521, TType.DOUBLE, 'requested_memonheap', None, None, ), # 521
+    (522, TType.DOUBLE, 'requested_memoffheap', None, None, ), # 522
+    (523, TType.DOUBLE, 'requested_cpu', None, None, ), # 523
+    (524, TType.DOUBLE, 'assigned_memonheap', None, None, ), # 524
+    (525, TType.DOUBLE, 'assigned_memoffheap', None, None, ), # 525
+    (526, TType.DOUBLE, 'assigned_cpu', None, None, ), # 526
   )
 
-  def __init__(self, id=None, name=None, num_tasks=None, num_executors=None, num_workers=None, uptime_secs=None, status=None, sched_status=None, owner=None, replication_count=None,):
+  def __init__(self, id=None, name=None, num_tasks=None, num_executors=None, num_workers=None, uptime_secs=None, status=None, sched_status=None, owner=None, replication_count=None, requested_memonheap=None, requested_memoffheap=None, requested_cpu=None, assigned_memonheap=None, assigned_memoffheap=None, assigned_cpu=None,):
     self.id = id
     self.name = name
     self.num_tasks = num_tasks
@@ -2235,6 +2252,12 @@ class TopologySummary:
     self.sched_status = sched_status
     self.owner = owner
     self.replication_count = replication_count
+    self.requested_memonheap = requested_memonheap
+    self.requested_memoffheap = requested_memoffheap
+    self.requested_cpu = requested_cpu
+    self.assigned_memonheap = assigned_memonheap
+    self.assigned_memoffheap = assigned_memoffheap
+    self.assigned_cpu = assigned_cpu
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -2295,6 +2318,36 @@ class TopologySummary:
           self.replication_count = iprot.readI32();
         else:
           iprot.skip(ftype)
+      elif fid == 521:
+        if ftype == TType.DOUBLE:
+          self.requested_memonheap = iprot.readDouble();
+        else:
+          iprot.skip(ftype)
+      elif fid == 522:
+        if ftype == TType.DOUBLE:
+          self.requested_memoffheap = iprot.readDouble();
+        else:
+          iprot.skip(ftype)
+      elif fid == 523:
+        if ftype == TType.DOUBLE:
+          self.requested_cpu = iprot.readDouble();
+        else:
+          iprot.skip(ftype)
+      elif fid == 524:
+        if ftype == TType.DOUBLE:
+          self.assigned_memonheap = iprot.readDouble();
+        else:
+          iprot.skip(ftype)
+      elif fid == 525:
+        if ftype == TType.DOUBLE:
+          self.assigned_memoffheap = iprot.readDouble();
+        else:
+          iprot.skip(ftype)
+      elif fid == 526:
+        if ftype == TType.DOUBLE:
+          self.assigned_cpu = iprot.readDouble();
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -2345,6 +2398,30 @@ class TopologySummary:
       oprot.writeFieldBegin('replication_count', TType.I32, 515)
       oprot.writeI32(self.replication_count)
       oprot.writeFieldEnd()
+    if self.requested_memonheap is not None:
+      oprot.writeFieldBegin('requested_memonheap', TType.DOUBLE, 521)
+      oprot.writeDouble(self.requested_memonheap)
+      oprot.writeFieldEnd()
+    if self.requested_memoffheap is not None:
+      oprot.writeFieldBegin('requested_memoffheap', TType.DOUBLE, 522)
+      oprot.writeDouble(self.requested_memoffheap)
+      oprot.writeFieldEnd()
+    if self.requested_cpu is not None:
+      oprot.writeFieldBegin('requested_cpu', TType.DOUBLE, 523)
+      oprot.writeDouble(self.requested_cpu)
+      oprot.writeFieldEnd()
+    if self.assigned_memonheap is not None:
+      oprot.writeFieldBegin('assigned_memonheap', TType.DOUBLE, 524)
+      oprot.writeDouble(self.assigned_memonheap)
+      oprot.writeFieldEnd()
+    if self.assigned_memoffheap is not None:
+      oprot.writeFieldBegin('assigned_memoffheap', TType.DOUBLE, 525)
+      oprot.writeDouble(self.assigned_memoffheap)
+      oprot.writeFieldEnd()
+    if self.assigned_cpu is not None:
+      oprot.writeFieldBegin('assigned_cpu', TType.DOUBLE, 526)
+      oprot.writeDouble(self.assigned_cpu)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -2378,6 +2455,12 @@ class TopologySummary:
     value = (value * 31) ^ hash(self.sched_status)
     value = (value * 31) ^ hash(self.owner)
     value = (value * 31) ^ hash(self.replication_count)
+    value = (value * 31) ^ hash(self.requested_memonheap)
+    value = (value * 31) ^ hash(self.requested_memoffheap)
+    value = (value * 31) ^ hash(self.requested_cpu)
+    value = (value * 31) ^ hash(self.assigned_memonheap)
+    value = (value * 31) ^ hash(self.assigned_memoffheap)
+    value = (value * 31) ^ hash(self.assigned_cpu)
     return value
 
   def __repr__(self):
@@ -3774,6 +3857,12 @@ class TopologyInfo:
    - sched_status
    - owner
    - replication_count
+   - requested_memonheap
+   - requested_memoffheap
+   - requested_cpu
+   - assigned_memonheap
+   - assigned_memoffheap
+   - assigned_cpu
   """
 
   thrift_spec = (
@@ -4293,9 +4382,20 @@ class TopologyInfo:
     (513, TType.STRING, 'sched_status', None, None, ), # 513
     (514, TType.STRING, 'owner', None, None, ), # 514
     (515, TType.I32, 'replication_count', None, None, ), # 515
+    None, # 516
+    None, # 517
+    None, # 518
+    None, # 519
+    None, # 520
+    (521, TType.DOUBLE, 'requested_memonheap', None, None, ), # 521
+    (522, TType.DOUBLE, 'requested_memoffheap', None, None, ), # 522
+    (523, TType.DOUBLE, 'requested_cpu', None, None, ), # 523
+    (524, TType.DOUBLE, 'assigned_memonheap', None, None, ), # 524
+    (525, TType.DOUBLE, 'assigned_memoffheap', None, None, ), # 525
+    (526, TType.DOUBLE, 'assigned_cpu', None, None, ), # 526
   )
 
-  def __init__(self, id=None, name=None, uptime_secs=None, executors=None, status=None, errors=None, component_debug=None, sched_status=None, owner=None, replication_count=None,):
+  def __init__(self, id=None, name=None, uptime_secs=None, executors=None, status=None, errors=None, component_debug=None, sched_status=None, owner=None, replication_count=None, requested_memonheap=None, requested_memoffheap=None, requested_cpu=None, assigned_memonheap=None, assigned_memoffheap=None, assigned_cpu=None,):
     self.id = id
     self.name = name
     self.uptime_secs = uptime_secs
@@ -4306,6 +4406,12 @@ class TopologyInfo:
     self.sched_status = sched_status
     self.owner = owner
     self.replication_count = replication_count
+    self.requested_memonheap = requested_memonheap
+    self.requested_memoffheap = requested_memoffheap
+    self.requested_cpu = requested_cpu
+    self.assigned_memonheap = assigned_memonheap
+    self.assigned_memoffheap = assigned_memoffheap
+    self.assigned_cpu = assigned_cpu
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -4391,6 +4497,36 @@ class TopologyInfo:
           self.replication_count = iprot.readI32();
         else:
           iprot.skip(ftype)
+      elif fid == 521:
+        if ftype == TType.DOUBLE:
+          self.requested_memonheap = iprot.readDouble();
+        else:
+          iprot.skip(ftype)
+      elif fid == 522:
+        if ftype == TType.DOUBLE:
+          self.requested_memoffheap = iprot.readDouble();
+        else:
+          iprot.skip(ftype)
+      elif fid == 523:
+        if ftype == TType.DOUBLE:
+          self.requested_cpu = iprot.readDouble();
+        else:
+          iprot.skip(ftype)
+      elif fid == 524:
+        if ftype == TType.DOUBLE:
+          self.assigned_memonheap = iprot.readDouble();
+        else:
+          iprot.skip(ftype)
+      elif fid == 525:
+        if ftype == TType.DOUBLE:
+          self.assigned_memoffheap = iprot.readDouble();
+        else:
+          iprot.skip(ftype)
+      elif fid == 526:
+        if ftype == TType.DOUBLE:
+          self.assigned_cpu = iprot.readDouble();
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -4455,6 +4591,30 @@ class TopologyInfo:
       oprot.writeFieldBegin('replication_count', TType.I32, 515)
       oprot.writeI32(self.replication_count)
       oprot.writeFieldEnd()
+    if self.requested_memonheap is not None:
+      oprot.writeFieldBegin('requested_memonheap', TType.DOUBLE, 521)
+      oprot.writeDouble(self.requested_memonheap)
+      oprot.writeFieldEnd()
+    if self.requested_memoffheap is not None:
+      oprot.writeFieldBegin('requested_memoffheap', TType.DOUBLE, 522)
+      oprot.writeDouble(self.requested_memoffheap)
+      oprot.writeFieldEnd()
+    if self.requested_cpu is not None:
+      oprot.writeFieldBegin('requested_cpu', TType.DOUBLE, 523)
+      oprot.writeDouble(self.requested_cpu)
+      oprot.writeFieldEnd()
+    if self.assigned_memonheap is not None:
+      oprot.writeFieldBegin('assigned_memonheap', TType.DOUBLE, 524)
+      oprot.writeDouble(self.assigned_memonheap)
+      oprot.writeFieldEnd()
+    if self.assigned_memoffheap is not None:
+      oprot.writeFieldBegin('assigned_memoffheap', TType.DOUBLE, 525)
+      oprot.writeDouble(self.assigned_memoffheap)
+      oprot.writeFieldEnd()
+    if self.assigned_cpu is not None:
+      oprot.writeFieldBegin('assigned_cpu', TType.DOUBLE, 526)
+      oprot.writeDouble(self.assigned_cpu)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -4486,6 +4646,12 @@ class TopologyInfo:
     value = (value * 31) ^ hash(self.sched_status)
     value = (value * 31) ^ hash(self.owner)
     value = (value * 31) ^ hash(self.replication_count)
+    value = (value * 31) ^ hash(self.requested_memonheap)
+    value = (value * 31) ^ hash(self.requested_memoffheap)
+    value = (value * 31) ^ hash(self.requested_cpu)
+    value = (value * 31) ^ hash(self.assigned_memonheap)
+    value = (value * 31) ^ hash(self.assigned_memoffheap)
+    value = (value * 31) ^ hash(self.assigned_cpu)
     return value
 
   def __repr__(self):
@@ -5248,6 +5414,12 @@ class TopologyPageInfo:
    - owner
    - debug_options
    - replication_count
+   - requested_memonheap
+   - requested_memoffheap
+   - requested_cpu
+   - assigned_memonheap
+   - assigned_memoffheap
+   - assigned_cpu
   """
 
   thrift_spec = (
@@ -5267,202 +5439,773 @@ class TopologyPageInfo:
     (13, TType.STRING, 'owner', None, None, ), # 13
     (14, TType.STRUCT, 'debug_options', (DebugOptions, DebugOptions.thrift_spec), None, ), # 14
     (15, TType.I32, 'replication_count', None, None, ), # 15
-  )
-
-  def __init__(self, id=None, name=None, uptime_secs=None, status=None, num_tasks=None, num_workers=None, num_executors=None, topology_conf=None, id_to_spout_agg_stats=None, id_to_bolt_agg_stats=None, sched_status=None, topology_stats=None, owner=None, debug_options=None, replication_count=None,):
-    self.id = id
-    self.name = name
-    self.uptime_secs = uptime_secs
-    self.status = status
-    self.num_tasks = num_tasks
-    self.num_workers = num_workers
-    self.num_executors = num_executors
-    self.topology_conf = topology_conf
-    self.id_to_spout_agg_stats = id_to_spout_agg_stats
-    self.id_to_bolt_agg_stats = id_to_bolt_agg_stats
-    self.sched_status = sched_status
-    self.topology_stats = topology_stats
-    self.owner = owner
-    self.debug_options = debug_options
-    self.replication_count = replication_count
-
-  def read(self, iprot):
-    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
-      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
-      return
-    iprot.readStructBegin()
-    while True:
-      (fname, ftype, fid) = iprot.readFieldBegin()
-      if ftype == TType.STOP:
-        break
-      if fid == 1:
-        if ftype == TType.STRING:
-          self.id = iprot.readString().decode('utf-8')
-        else:
-          iprot.skip(ftype)
-      elif fid == 2:
-        if ftype == TType.STRING:
-          self.name = iprot.readString().decode('utf-8')
-        else:
-          iprot.skip(ftype)
-      elif fid == 3:
-        if ftype == TType.I32:
-          self.uptime_secs = iprot.readI32();
-        else:
-          iprot.skip(ftype)
-      elif fid == 4:
-        if ftype == TType.STRING:
-          self.status = iprot.readString().decode('utf-8')
-        else:
-          iprot.skip(ftype)
-      elif fid == 5:
-        if ftype == TType.I32:
-          self.num_tasks = iprot.readI32();
-        else:
-          iprot.skip(ftype)
-      elif fid == 6:
-        if ftype == TType.I32:
-          self.num_workers = iprot.readI32();
-        else:
-          iprot.skip(ftype)
-      elif fid == 7:
-        if ftype == TType.I32:
-          self.num_executors = iprot.readI32();
-        else:
-          iprot.skip(ftype)
-      elif fid == 8:
-        if ftype == TType.STRING:
-          self.topology_conf = iprot.readString().decode('utf-8')
-        else:
-          iprot.skip(ftype)
-      elif fid == 9:
-        if ftype == TType.MAP:
-          self.id_to_spout_agg_stats = {}
-          (_ktype354, _vtype355, _size353 ) = iprot.readMapBegin()
-          for _i357 in xrange(_size353):
-            _key358 = iprot.readString().decode('utf-8')
-            _val359 = ComponentAggregateStats()
-            _val359.read(iprot)
-            self.id_to_spout_agg_stats[_key358] = _val359
-          iprot.readMapEnd()
-        else:
-          iprot.skip(ftype)
-      elif fid == 10:
-        if ftype == TType.MAP:
-          self.id_to_bolt_agg_stats = {}
-          (_ktype361, _vtype362, _size360 ) = iprot.readMapBegin()
-          for _i364 in xrange(_size360):
-            _key365 = iprot.readString().decode('utf-8')
-            _val366 = ComponentAggregateStats()
-            _val366.read(iprot)
-            self.id_to_bolt_agg_stats[_key365] = _val366
-          iprot.readMapEnd()
-        else:
-          iprot.skip(ftype)
-      elif fid == 11:
-        if ftype == TType.STRING:
-          self.sched_status = iprot.readString().decode('utf-8')
-        else:
-          iprot.skip(ftype)
-      elif fid == 12:
-        if ftype == TType.STRUCT:
-          self.topology_stats = TopologyStats()
-          self.topology_stats.read(iprot)
-        else:
-          iprot.skip(ftype)
-      elif fid == 13:
-        if ftype == TType.STRING:
-          self.owner = iprot.readString().decode('utf-8')
-        else:
-          iprot.skip(ftype)
-      elif fid == 14:
-        if ftype == TType.STRUCT:
-          self.debug_options = DebugOptions()
-          self.debug_options.read(iprot)
-        else:
-          iprot.skip(ftype)
-      elif fid == 15:
-        if ftype == TType.I32:
-          self.replication_count = iprot.readI32();
-        else:
-          iprot.skip(ftype)
-      else:
-        iprot.skip(ftype)
-      iprot.readFieldEnd()
-    iprot.readStructEnd()
-
-  def write(self, oprot):
-    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
-      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
-      return
-    oprot.writeStructBegin('TopologyPageInfo')
-    if self.id is not None:
-      oprot.writeFieldBegin('id', TType.STRING, 1)
-      oprot.writeString(self.id.encode('utf-8'))
-      oprot.writeFieldEnd()
-    if self.name is not None:
-      oprot.writeFieldBegin('name', TType.STRING, 2)
-      oprot.writeString(self.name.encode('utf-8'))
-      oprot.writeFieldEnd()
-    if self.uptime_secs is not None:
-      oprot.writeFieldBegin('uptime_secs', TType.I32, 3)
-      oprot.writeI32(self.uptime_secs)
-      oprot.writeFieldEnd()
-    if self.status is not None:
-      oprot.writeFieldBegin('status', TType.STRING, 4)
-      oprot.writeString(self.status.encode('utf-8'))
-      oprot.writeFieldEnd()
-    if self.num_tasks is not None:
-      oprot.writeFieldBegin('num_tasks', TType.I32, 5)
-      oprot.writeI32(self.num_tasks)
-      oprot.writeFieldEnd()
-    if self.num_workers is not None:
-      oprot.writeFieldBegin('num_workers', TType.I32, 6)
-      oprot.writeI32(self.num_workers)
-      oprot.writeFieldEnd()
-    if self.num_executors is not None:
-      oprot.writeFieldBegin('num_executors', TType.I32, 7)
-      oprot.writeI32(self.num_executors)
-      oprot.writeFieldEnd()
-    if self.topology_conf is not None:
-      oprot.writeFieldBegin('topology_conf', TType.STRING, 8)
-      oprot.writeString(self.topology_conf.encode('utf-8'))
-      oprot.writeFieldEnd()
-    if self.id_to_spout_agg_stats is not None:
-      oprot.writeFieldBegin('id_to_spout_agg_stats', TType.MAP, 9)
-      oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.id_to_spout_agg_stats))
-      for kiter367,viter368 in self.id_to_spout_agg_stats.items():
-        oprot.writeString(kiter367.encode('utf-8'))
-        viter368.write(oprot)
-      oprot.writeMapEnd()
-      oprot.writeFieldEnd()
-    if self.id_to_bolt_agg_stats is not None:
-      oprot.writeFieldBegin('id_to_bolt_agg_stats', TType.MAP, 10)
-      oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.id_to_bolt_agg_stats))
-      for kiter369,viter370 in self.id_to_bolt_agg_stats.items():
-        oprot.writeString(kiter369.encode('utf-8'))
-        viter370.write(oprot)
-      oprot.writeMapEnd()
-      oprot.writeFieldEnd()
-    if self.sched_status is not None:
-      oprot.writeFieldBegin('sched_status', TType.STRING, 11)
-      oprot.writeString(self.sched_status.encode('utf-8'))
-      oprot.writeFieldEnd()
-    if self.topology_stats is not None:
-      oprot.writeFieldBegin('topology_stats', TType.STRUCT, 12)
-      self.topology_stats.write(oprot)
-      oprot.writeFieldEnd()
-    if self.owner is not None:
-      oprot.writeFieldBegin('owner', TType.STRING, 13)
-      oprot.writeString(self.owner.encode('utf-8'))
-      oprot.writeFieldEnd()
-    if self.debug_options is not None:
-      oprot.writeFieldBegin('debug_options', TType.STRUCT, 14)
-      self.debug_options.write(oprot)
-      oprot.writeFieldEnd()
-    if self.replication_count is not None:
-      oprot.writeFieldBegin('replication_count', TType.I32, 15)
-      oprot.writeI32(self.replication_count)
+    None, # 16
+    None, # 17
+    None, # 18
+    None, # 19
+    None, # 20
+    None, # 21
+    None, # 22
+    None, # 23
+    None, # 24
+    None, # 25
+    None, # 26
+    None, # 27
+    None, # 28
+    None, # 29
+    None, # 30
+    None, # 31
+    None, # 32
+    None, # 33
+    None, # 34
+    None, # 35
+    None, # 36
+    None, # 37
+    None, # 38
+    None, # 39
+    None, # 40
+    None, # 41
+    None, # 42
+    None, # 43
+    None, # 44
+    None, # 45
+    None, # 46
+    None, # 47
+    None, # 48
+    None, # 49
+    None, # 50
+    None, # 51
+    None, # 52
+    None, # 53
+    None, # 54
+    None, # 55
+    None, # 56
+    None, # 57
+    None, # 58
+    None, # 59
+    None, # 60
+    None, # 61
+    None, # 62
+    None, # 63
+    None, # 64
+    None, # 65
+    None, # 66
+    None, # 67
+    None, # 68
+    None, # 69
+    None, # 70
+    None, # 71
+    None, # 72
+    None, # 73
+    None, # 74
+    None, # 75
+    None, # 76
+    None, # 77
+    None, # 78
+    None, # 79
+    None, # 80
+    None, # 81
+    None, # 82
+    None, # 83
+    None, # 84
+    None, # 85
+    None, # 86
+    None, # 87
+    None, # 88
+    None, # 89
+    None, # 90
+    None, # 91
+    None, # 92
+    None, # 93
+    None, # 94
+    None, # 95
+    None, # 96
+    None, # 97
+    None, # 98
+    None, # 99
+    None, # 100
+    None, # 101
+    None, # 102
+    None, # 103
+    None, # 104
+    None, # 105
+    None, # 106
+    None, # 107
+    None, # 108
+    None, # 109
+    None, # 110
+    None, # 111
+    None, # 112
+    None, # 113
+    None, # 114
+    None, # 115
+    None, # 116
+    None, # 117
+    None, # 118
+    None, # 119
+    None, # 120
+    None, # 121
+    None, # 122
+    None, # 123
+    None, # 124
+    None, # 125
+    None, # 126
+    None, # 127
+    None, # 128
+    None, # 129
+    None, # 130
+    None, # 131
+    None, # 132
+    None, # 133
+    None, # 134
+    None, # 135
+    None, # 136
+    None, # 137
+    None, # 138
+    None, # 139
+    None, # 140
+    None, # 141
+    None, # 142
+    None, # 143
+    None, # 144
+    None, # 145
+    None, # 146
+    None, # 147
+    None, # 148
+    None, # 149
+    None, # 150
+    None, # 151
+    None, # 152
+    None, # 153
+    None, # 154
+    None, # 155
+    None, # 156
+    None, # 157
+    None, # 158
+    None, # 159
+    None, # 160
+    None, # 161
+    None, # 162
+    None, # 163
+    None, # 164
+    None, # 165
+    None, # 166
+    None, # 167
+    None, # 168
+    None, # 169
+    None, # 170
+    None, # 171
+    None, # 172
+    None, # 173
+    None, # 174
+    None, # 175
+    None, # 176
+    None, # 177
+    None, # 178
+    None, # 179
+    None, # 180
+    None, # 181
+    None, # 182
+    None, # 183
+    None, # 184
+    None, # 185
+    None, # 186
+    None, # 187
+    None, # 188
+    None, # 189
+    None, # 190
+    None, # 191
+    None, # 192
+    None, # 193
+    None, # 194
+    None, # 195
+    None, # 196
+    None, # 197
+    None, # 198
+    None, # 199
+    None, # 200
+    None, # 201
+    None, # 202
+    None, # 203
+    None, # 204
+    None, # 205
+    None, # 206
+    None, # 207
+    None, # 208
+    None, # 209
+    None, # 210
+    None, # 211
+    None, # 212
+    None, # 213
+    None, # 214
+    None, # 215
+    None, # 216
+    None, # 217
+    None, # 218
+    None, # 219
+    None, # 220
+    None, # 221
+    None, # 222
+    None, # 223
+    None, # 224
+    None, # 225
+    None, # 226
+    None, # 227
+    None, # 228
+    None, # 229
+    None, # 230
+    None, # 231
+    None, # 232
+    None, # 233
+    None, # 234
+    None, # 235
+    None, # 236
+    None, # 237
+    None, # 238
+    None, # 239
+    None, # 240
+    None, # 241
+    None, # 242
+    None, # 243
+    None, # 244
+    None, # 245
+    None, # 246
+    None, # 247
+    None, # 248
+    None, # 249
+    None, # 250
+    None, # 251
+    None, # 252
+    None, # 253
+    None, # 254
+    None, # 255
+    None, # 256
+    None, # 257
+    None, # 258
+    None, # 259
+    None, # 260
+    None, # 261
+    None, # 262
+    None, # 263
+    None, # 264
+    None, # 265
+    None, # 266
+    None, # 267
+    None, # 268
+    None, # 269
+    None, # 270
+    None, # 271
+    None, # 272
+    None, # 273
+    None, # 274
+    None, # 275
+    None, # 276
+    None, # 277
+    None, # 278
+    None, # 279
+    None, # 280
+    None, # 281
+    None, # 282
+    None, # 283
+    None, # 284
+    None, # 285
+    None, # 286
+    None, # 287
+    None, # 288
+    None, # 289
+    None, # 290
+    None, # 291
+    None, # 292
+    None, # 293
+    None, # 294
+    None, # 295
+    None, # 296
+    None, # 297
+    None, # 298
+    None, # 299
+    None, # 300
+    None, # 301
+    None, # 302
+    None, # 303
+    None, # 304
+    None, # 305
+    None, # 306
+    None, # 307
+    None, # 308
+    None, # 309
+    None, # 310
+    None, # 311
+    None, # 312
+    None, # 313
+    None, # 314
+    None, # 315
+    None, # 316
+    None, # 317
+    None, # 318
+    None, # 319
+    None, # 320
+    None, # 321
+    None, # 322
+    None, # 323
+    None, # 324
+    None, # 325
+    None, # 326
+    None, # 327
+    None, # 328
+    None, # 329
+    None, # 330
+    None, # 331
+    None, # 332
+    None, # 333
+    None, # 334
+    None, # 335
+    None, # 336
+    None, # 337
+    None, # 338
+    None, # 339
+    None, # 340
+    None, # 341
+    None, # 342
+    None, # 343
+    None, # 344
+    None, # 345
+    None, # 346
+    None, # 347
+    None, # 348
+    None, # 349
+    None, # 350
+    None, # 351
+    None, # 352
+    None, # 353
+    None, # 354
+    None, # 355
+    None, # 356
+    None, # 357
+    None, # 358
+    None, # 359
+    None, # 360
+    None, # 361
+    None, # 362
+    None, # 363
+    None, # 364
+    None, # 365
+    None, # 366
+    None, # 367
+    None, # 368
+    None, # 369
+    None, # 370
+    None, # 371
+    None, # 372
+    None, # 373
+    None, # 374
+    None, # 375
+    None, # 376
+    None, # 377
+    None, # 378
+    None, # 379
+    None, # 380
+    None, # 381
+    None, # 382
+    None, # 383
+    None, # 384
+    None, # 385
+    None, # 386
+    None, # 387
+    None, # 388
+    None, # 389
+    None, # 390
+    None, # 391
+    None, # 392
+    None, # 393
+    None, # 394
+    None, # 395
+    None, # 396
+    None, # 397
+    None, # 398
+    None, # 399
+    None, # 400
+    None, # 401
+    None, # 402
+    None, # 403
+    None, # 404
+    None, # 405
+    None, # 406
+    None, # 407
+    None, # 408
+    None, # 409
+    None, # 410
+    None, # 411
+    None, # 412
+    None, # 413
+    None, # 414
+    None, # 415
+    None, # 416
+    None, # 417
+    None, # 418
+    None, # 419
+    None, # 420
+    None, # 421
+    None, # 422
+    None, # 423
+    None, # 424
+    None, # 425
+    None, # 426
+    None, # 427
+    None, # 428
+    None, # 429
+    None, # 430
+    None, # 431
+    None, # 432
+    None, # 433
+    None, # 434
+    None, # 435
+    None, # 436
+    None, # 437
+    None, # 438
+    None, # 439
+    None, # 440
+    None, # 441
+    None, # 442
+    None, # 443
+    None, # 444
+    None, # 445
+    None, # 446
+    None, # 447
+    None, # 448
+    None, # 449
+    None, # 450
+    None, # 451
+    None, # 452
+    None, # 453
+    None, # 454
+    None, # 455
+    None, # 456
+    None, # 457
+    None, # 458
+    None, # 459
+    None, # 460
+    None, # 461
+    None, # 462
+    None, # 463
+    None, # 464
+    None, # 465
+    None, # 466
+    None, # 467
+    None, # 468
+    None, # 469
+    None, # 470
+    None, # 471
+    None, # 472
+    None, # 473
+    None, # 474
+    None, # 475
+    None, # 476
+    None, # 477
+    None, # 478
+    None, # 479
+    None, # 480
+    None, # 481
+    None, # 482
+    None, # 483
+    None, # 484
+    None, # 485
+    None, # 486
+    None, # 487
+    None, # 488
+    None, # 489
+    None, # 490
+    None, # 491
+    None, # 492
+    None, # 493
+    None, # 494
+    None, # 495
+    None, # 496
+    None, # 497
+    None, # 498
+    None, # 499
+    None, # 500
+    None, # 501
+    None, # 502
+    None, # 503
+    None, # 504
+    None, # 505
+    None, # 506
+    None, # 507
+    None, # 508
+    None, # 509
+    None, # 510
+    None, # 511
+    None, # 512
+    None, # 513
+    None, # 514
+    None, # 515
+    None, # 516
+    None, # 517
+    None, # 518
+    None, # 519
+    None, # 520
+    (521, TType.DOUBLE, 'requested_memonheap', None, None, ), # 521
+    (522, TType.DOUBLE, 'requested_memoffheap', None, None, ), # 522
+    (523, TType.DOUBLE, 'requested_cpu', None, None, ), # 523
+    (524, TType.DOUBLE, 'assigned_memonheap', None, None, ), # 524
+    (525, TType.DOUBLE, 'assigned_memoffheap', None, None, ), # 525
+    (526, TType.DOUBLE, 'assigned_cpu', None, None, ), # 526
+  )
+
+  def __init__(self, id=None, name=None, uptime_secs=None, status=None, num_tasks=None, num_workers=None, num_executors=None, topology_conf=None, id_to_spout_agg_stats=None, id_to_bolt_agg_stats=None, sched_status=None, topology_stats=None, owner=None, debug_options=None, replication_count=None, requested_memonheap=None, requested_memoffheap=None, requested_cpu=None, assigned_memonheap=None, assigned_memoffheap=None, assigned_cpu=None,):
+    self.id = id
+    self.name = name
+    self.uptime_secs = uptime_secs
+    self.status = status
+    self.num_tasks = num_tasks
+    self.num_workers = num_workers
+    self.num_executors = num_executors
+    self.topology_conf = topology_conf
+    self.id_to_spout_agg_stats = id_to_spout_agg_stats
+    self.id_to_bolt_agg_stats = id_to_bolt_agg_stats
+    self.sched_status = sched_status
+    self.topology_stats = topology_stats
+    self.owner = owner
+    self.debug_options = debug_options
+    self.replication_count = replication_count
+    self.requested_memonheap = requested_memonheap
+    self.requested_memoffheap = requested_memoffheap
+    self.requested_cpu = requested_cpu
+    self.assigned_memonheap = assigned_memonheap
+    self.assigned_memoffheap = assigned_memoffheap
+    self.assigned_cpu = assigned_cpu
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.id = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.name = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.I32:
+          self.uptime_secs = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.STRING:
+          self.status = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.I32:
+          self.num_tasks = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.I32:
+          self.num_workers = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.I32:
+          self.num_executors = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      elif fid == 8:
+        if ftype == TType.STRING:
+          self.topology_conf = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 9:
+        if ftype == TType.MAP:
+          self.id_to_spout_agg_stats = {}
+          (_ktype354, _vtype355, _size353 ) = iprot.readMapBegin()
+          for _i357 in xrange(_size353):
+            _key358 = iprot.readString().decode('utf-8')
+            _val359 = ComponentAggregateStats()
+            _val359.read(iprot)
+            self.id_to_spout_agg_stats[_key358] = _val359
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 10:
+        if ftype == TType.MAP:
+          self.id_to_bolt_agg_stats = {}
+          (_ktype361, _vtype362, _size360 ) = iprot.readMapBegin()
+          for _i364 in xrange(_size360):
+            _key365 = iprot.readString().decode('utf-8')
+            _val366 = ComponentAggregateStats()
+            _val366.read(iprot)
+            self.id_to_bolt_agg_stats[_key365] = _val366
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 11:
+        if ftype == TType.STRING:
+          self.sched_status = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 12:
+        if ftype == TType.STRUCT:
+          self.topology_stats = TopologyStats()
+          self.topology_stats.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 13:
+        if ftype == TType.STRING:
+          self.owner = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 14:
+        if ftype == TType.STRUCT:
+          self.debug_options = DebugOptions()
+          self.debug_options.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 15:
+        if ftype == TType.I32:
+          self.replication_count = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      elif fid == 521:
+        if ftype == TType.DOUBLE:
+          self.requested_memonheap = iprot.readDouble();
+        else:
+          iprot.skip(ftype)
+      elif fid == 522:
+        if ftype == TType.DOUBLE:
+          self.requested_memoffheap = iprot.readDouble();
+        else:
+          iprot.skip(ftype)
+      elif fid == 523:
+        if ftype == TType.DOUBLE:
+          self.requested_cpu = iprot.readDouble();
+        else:
+          iprot.skip(ftype)
+      elif fid == 524:
+        if ftype == TType.DOUBLE:
+          self.assigned_memonheap = iprot.readDouble();
+        else:
+          iprot.skip(ftype)
+      elif fid == 525:
+        if ftype == TType.DOUBLE:
+          self.assigned_memoffheap = iprot.readDouble();
+        else:
+          iprot.skip(ftype)
+      elif fid == 526:
+        if ftype == TType.DOUBLE:
+          self.assigned_cpu = iprot.readDouble();
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('TopologyPageInfo')
+    if self.id is not None:
+      oprot.writeFieldBegin('id', TType.STRING, 1)
+      oprot.writeString(self.id.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.name is not None:
+      oprot.writeFieldBegin('name', TType.STRING, 2)
+      oprot.writeString(self.name.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.uptime_secs is not None:
+      oprot.writeFieldBegin('uptime_secs', TType.I32, 3)
+      oprot.writeI32(self.uptime_secs)
+      oprot.writeFieldEnd()
+    if self.status is not None:
+      oprot.writeFieldBegin('status', TType.STRING, 4)
+      oprot.writeString(self.status.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.num_tasks is not None:
+      oprot.writeFieldBegin('num_tasks', TType.I32, 5)
+      oprot.writeI32(self.num_tasks)
+      oprot.writeFieldEnd()
+    if self.num_workers is not None:
+      oprot.writeFieldBegin('num_workers', TType.I32, 6)
+      oprot.writeI32(self.num_workers)
+      oprot.writeFieldEnd()
+    if self.num_executors is not None:
+      oprot.writeFieldBegin('num_executors', TType.I32, 7)
+      oprot.writeI32(self.num_executors)
+      oprot.writeFieldEnd()
+    if self.topology_conf is not None:
+      oprot.writeFieldBegin('topology_conf', TType.STRING, 8)
+      oprot.writeString(self.topology_conf.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.id_to_spout_agg_stats is not None:
+      oprot.writeFieldBegin('id_to_spout_agg_stats', TType.MAP, 9)
+      oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.id_to_spout_agg_stats))
+      for kiter367,viter368 in self.id_to_spout_agg_stats.items():
+        oprot.writeString(kiter367.encode('utf-8'))
+        viter368.write(oprot)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.id_to_bolt_agg_stats is not None:
+      oprot.writeFieldBegin('id_to_bolt_agg_stats', TType.MAP, 10)
+      oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.id_to_bolt_agg_stats))
+      for kiter369,viter370 in self.id_to_bolt_agg_stats.items():
+        oprot.writeString(kiter369.encode('utf-8'))
+        viter370.write(oprot)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.sched_status is not None:
+      oprot.writeFieldBegin('sched_status', TType.STRING, 11)
+      oprot.writeString(self.sched_status.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.topology_stats is not None:
+      oprot.writeFieldBegin('topology_stats', TType.STRUCT, 12)
+      self.topology_stats.write(oprot)
+      oprot.writeFieldEnd()
+    if self.owner is not None:
+      oprot.writeFieldBegin('owner', TType.STRING, 13)
+      oprot.writeString(self.owner.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.debug_options is not None:
+      oprot.writeFieldBegin('debug_options', TType.STRUCT, 14)
+      self.debug_options.write(oprot)
+      oprot.writeFieldEnd()
+    if self.replication_count is not None:
+      oprot.writeFieldBegin('replication_count', TType.I32, 15)
+      oprot.writeI32(self.replication_count)
+      oprot.writeFieldEnd()
+    if self.requested_memonheap is not None:
+      oprot.writeFieldBegin('requested_memonheap', TType.DOUBLE, 521)
+      oprot.writeDouble(self.requested_memonheap)
+      oprot.writeFieldEnd()
+    if self.requested_memoffheap is not None:
+      oprot.writeFieldBegin('requested_memoffheap', TType.DOUBLE, 522)
+      oprot.writeDouble(self.requested_memoffheap)
+      oprot.writeFieldEnd()
+    if self.requested_cpu is not None:
+      oprot.writeFieldBegin('requested_cpu', TType.DOUBLE, 523)
+      oprot.writeDouble(self.requested_cpu)
+      oprot.writeFieldEnd()
+    if self.assigned_memonheap is not None:
+      oprot.writeFieldBegin('assigned_memonheap', TType.DOUBLE, 524)
+      oprot.writeDouble(self.assigned_memonheap)
+      oprot.writeFieldEnd()
+    if self.assigned_memoffheap is not None:
+      oprot.writeFieldBegin('assigned_memoffheap', TType.DOUBLE, 525)
+      oprot.writeDouble(self.assigned_memoffheap)
+      oprot.writeFieldEnd()
+    if self.assigned_cpu is not None:
+      oprot.writeFieldBegin('assigned_cpu', TType.DOUBLE, 526)
+      oprot.writeDouble(self.assigned_cpu)
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
@@ -5490,6 +6233,12 @@ class TopologyPageInfo:
     value = (value * 31) ^ hash(self.owner)
     value = (value * 31) ^ hash(self.debug_options)
     value = (value * 31) ^ hash(self.replication_count)
+    value = (value * 31) ^ hash(self.requested_memonheap)
+    value = (value * 31) ^ hash(self.requested_memoffheap)
+    value = (value * 31) ^ hash(self.requested_cpu)
+    value = (value * 31) ^ hash(self.assigned_memonheap)
+    value = (value * 31) ^ hash(self.assigned_memoffheap)
+    value = (value * 31) ^ hash(self.assigned_cpu)
     return value
 
   def __repr__(self):