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):