You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by bo...@apache.org on 2018/09/13 21:04:43 UTC

[1/3] storm git commit: STORM-3217: Fixing getComponentPage API call

Repository: storm
Updated Branches:
  refs/heads/master a1e6e98c6 -> f5a410ba3


STORM-3217: Fixing getComponentPage API call


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

Branch: refs/heads/master
Commit: a805c70b210f47cb4a5192a6692888637a81f654
Parents: 8f9061a
Author: Govind Menon <go...@gmail.com>
Authored: Wed Sep 12 15:17:43 2018 -0500
Committer: Govind Menon <go...@gmail.com>
Committed: Wed Sep 12 16:57:08 2018 -0500

----------------------------------------------------------------------
 .../org/apache/storm/daemon/ui/UIHelpers.java   | 380 ++++++++++++++++++-
 1 file changed, 361 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/a805c70b/storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIHelpers.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIHelpers.java b/storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIHelpers.java
index a080599..c280cf2 100644
--- a/storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIHelpers.java
+++ b/storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIHelpers.java
@@ -25,12 +25,14 @@ import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.net.URLEncoder;
 import java.util.ArrayList;
+import java.util.Comparator;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Set;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -49,6 +51,9 @@ import org.apache.storm.generated.ClusterSummary;
 import org.apache.storm.generated.CommonAggregateStats;
 import org.apache.storm.generated.ComponentAggregateStats;
 import org.apache.storm.generated.ComponentPageInfo;
+import org.apache.storm.generated.ComponentType;
+import org.apache.storm.generated.ErrorInfo;
+import org.apache.storm.generated.ExecutorAggregateStats;
 import org.apache.storm.generated.ExecutorInfo;
 import org.apache.storm.generated.ExecutorSummary;
 import org.apache.storm.generated.GetInfoOptions;
@@ -66,6 +71,8 @@ import org.apache.storm.generated.OwnerResourceSummary;
 import org.apache.storm.generated.ProfileAction;
 import org.apache.storm.generated.ProfileRequest;
 import org.apache.storm.generated.RebalanceOptions;
+import org.apache.storm.generated.SpecificAggregateStats;
+import org.apache.storm.generated.SpoutAggregateStats;
 import org.apache.storm.generated.SpoutSpec;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.generated.SupervisorPageInfo;
@@ -975,7 +982,8 @@ public class UIHelpers {
      * @param config config
      * @return getSupervisorsMap
      */
-    private static List<Map> getSupervisorsMap(List<SupervisorSummary> supervisors, Map<String, Object> config) {
+    private static List<Map> getSupervisorsMap(List<SupervisorSummary> supervisors,
+                                               Map<String, Object> config) {
         List<Map> supervisorMaps = new ArrayList();
         for (SupervisorSummary supervisorSummary : supervisors) {
             supervisorMaps.add(getPrettifiedSupervisorMap(supervisorSummary, config));
@@ -1080,6 +1088,11 @@ public class UIHelpers {
         return result;
     }
 
+    /**
+     * getStatDisplayMapLong.
+     * @param windowToTransferred windowToTransferred
+     * @return getStatDisplayMapLong
+     */
     private static Map<String, Long> getStatDisplayMapLong(Map<String,Long> windowToTransferred) {
         Map<String, Long> result = new HashMap();
         for (Map.Entry<String, Long> entry : windowToTransferred.entrySet()) {
@@ -1088,6 +1101,11 @@ public class UIHelpers {
         return result;
     }
 
+    /**
+     * getCommonAggStatsMap.
+     * @param commonAggregateStats commonAggregateStats
+     * @return getCommonAggStatsMap
+     */
     private static Map<String, Object> getCommonAggStatsMap(CommonAggregateStats commonAggregateStats) {
         Map<String, Object> result = new HashMap();
         result.put("executors", commonAggregateStats.get_num_executors());
@@ -1096,34 +1114,272 @@ public class UIHelpers {
         result.put("transferred", commonAggregateStats.get_transferred());
         result.put("acked", commonAggregateStats.get_acked());
         result.put("failed", commonAggregateStats.get_failed());
+        if (commonAggregateStats.is_set_resources_map()) {
+            result.put(
+                    "requestedMemOnHeap",
+                    commonAggregateStats.get_resources_map().get(Constants.COMMON_ONHEAP_MEMORY_RESOURCE_NAME)
+            );
+            result.put(
+                    "requestedMemOffHeap",
+                    commonAggregateStats.get_resources_map().get(Constants.COMMON_OFFHEAP_MEMORY_RESOURCE_NAME));
+            result.put(
+                    "requestedCpu",
+                    commonAggregateStats.get_resources_map().get(Constants.COMMON_CPU_RESOURCE_NAME));
+        }
+        return result;
+    }
+
+    /**
+     * getTruncatedErrorString.
+     * @param errorString errorString
+     * @return getTruncatedErrorString
+     */
+    private static String getTruncatedErrorString(String errorString) {
+        return errorString.substring(0, Math.min(errorString.length(), 200));
+    }
+
+    /**
+     * getSpoutAggStatsMap.
+     * @param componentAggregateStats componentAggregateStats
+     * @param window window
+     * @return getSpoutAggStatsMap
+     */
+    private static Map<String, Object> getSpoutAggStatsMap(
+            ComponentAggregateStats componentAggregateStats, String window) {
+        Map<String, Object> result = new HashMap();
+        SpoutAggregateStats spoutAggregateStats = componentAggregateStats.get_specific_stats().get_spout();
+        CommonAggregateStats commonStats = componentAggregateStats.get_common_stats();
+        result.put("window", window);
+        result.put("windowPretty", getWindowHint(window));
+        result.put("emitted", commonStats.get_emitted());
+        result.put("transferred", commonStats.get_transferred());
+        result.put("acked", commonStats.get_acked());
+        result.put("failed", commonStats.get_failed());
+        result.put("completeLatency", spoutAggregateStats.get_complete_latency_ms());
+
+
+        ErrorInfo lastError = componentAggregateStats.get_last_error();
+        result.put("lastError", Objects.isNull(lastError) ?  "" : getTruncatedErrorString(lastError.get_error()));
+        return result;
+    }
+
+    /**
+     * getBoltAggStatsMap.
+     * @param componentAggregateStats componentAggregateStats
+     * @param window window
+     * @return getBoltAggStatsMap
+     */
+    private static Map<String, Object> getBoltAggStatsMap(
+            ComponentAggregateStats componentAggregateStats, String window) {
+        Map<String, Object> result = new HashMap();
+        CommonAggregateStats commonStats = componentAggregateStats.get_common_stats();
+        result.put("window", window);
+        result.put("windowPretty", getWindowHint(window));
+        result.put("emitted", commonStats.get_emitted());
+        result.put("transferred", commonStats.get_transferred());
+        result.put("acked", commonStats.get_acked());
+        result.put("failed", commonStats.get_failed());
+        BoltAggregateStats boltAggregateStats = componentAggregateStats.get_specific_stats().get_bolt();
+        result.put("executeLatency", StatsUtil.floatStr(boltAggregateStats.get_execute_latency_ms()));
+        result.put("executed", boltAggregateStats.get_executed());
+        result.put("processLatency", StatsUtil.floatStr(boltAggregateStats.get_process_latency_ms()));
+        result.put("capacity", StatsUtil.floatStr(boltAggregateStats.get_capacity()));
+        return result;
+    }
+
+    /**
+     * nullToZero.
+     * @param value value
+     * @return nullToZero
+     */
+    private static Long nullToZero(Long value) {
+        return Objects.isNull(value) ? value : 0;
+    }
+
+    /**
+     * nullToZero.
+     * @param value value
+     * @return nullToZero
+     */
+    private static Double nullToZero(Double value) {
+        return Objects.isNull(value) ? value : 0;
+    }
+
+    /**
+     * getBoltInputStats.
+     * @param globalStreamId globalStreamId
+     * @param componentAggregateStats componentAggregateStats
+     * @return getBoltInputStats
+     */
+    private static Map<String, Object> getBoltInputStats(GlobalStreamId globalStreamId,
+                                                         ComponentAggregateStats componentAggregateStats) {
+        Map<String, Object> result = new HashMap();
+        SpecificAggregateStats specificAggregateStats = componentAggregateStats.get_specific_stats();
+        BoltAggregateStats boltAggregateStats = specificAggregateStats.get_bolt();
+        CommonAggregateStats commonAggregateStats = componentAggregateStats.get_common_stats();
+        String componentId = globalStreamId.get_componentId();
+        result.put("component", componentId);
+        result.put("encodedComponentId", URLEncoder.encode(componentId));
+        result.put("stream", globalStreamId.get_streamId());
+        result.put("executeLatency", StatsUtil.floatStr(boltAggregateStats.get_execute_latency_ms()));
+        result.put("processLatency", StatsUtil.floatStr(boltAggregateStats.get_process_latency_ms()));
+        result.put("executed", nullToZero(boltAggregateStats.get_executed()));
+        result.put("acked", nullToZero(commonAggregateStats.get_acked()));
+        result.put("failed", nullToZero(commonAggregateStats.get_failed()));
+        return result;
+    }
+
+    /**
+     * getBoltOutputStats.
+     * @param streamId streamId
+     * @param componentAggregateStats componentAggregateStats
+     * @return getBoltOutputStats
+     */
+    private static Map<String, Object> getBoltOutputStats(String streamId,
+                                                          ComponentAggregateStats componentAggregateStats) {
+        Map<String, Object> result = new HashMap();
+        result.put("stream", streamId);
+        CommonAggregateStats commonStats = componentAggregateStats.get_common_stats();
+        result.put("emitted", nullToZero(commonStats.get_emitted()));
+        result.put("transferred", nullToZero(commonStats.get_transferred()));
+        return result;
+    }
+
+    /**
+     * getSpoutOutputStats.
+     * @param streamId streamId
+     * @param componentAggregateStats componentAggregateStats
+     * @return getSpoutOutputStats
+     */
+    private static Map<String, Object> getSpoutOutputStats(String streamId,
+                                                           ComponentAggregateStats componentAggregateStats) {
+        SpecificAggregateStats specificAggregateStats = componentAggregateStats.get_specific_stats();
+        SpoutAggregateStats spoutAggregateStats = specificAggregateStats.get_spout();
+        Map<String, Object> result = new HashMap();
+        result.put("stream", streamId);
+        CommonAggregateStats commonStats = componentAggregateStats.get_common_stats();
+        result.put("emitted", nullToZero(commonStats.get_emitted()));
+        result.put("transferred", nullToZero(commonStats.get_transferred()));
+        result.put("completeLatency", StatsUtil.floatStr(spoutAggregateStats.get_complete_latency_ms()));
+        result.put("acked", nullToZero(commonStats.get_acked()));
+        result.put("failed", nullToZero(commonStats.get_failed()));
+        return result;
+    }
+
+    private static Map<String, Object> getBoltExecutorStats(String topologyId, Map<String, Object> config,
+                                                            ExecutorAggregateStats executorAggregateStats) {
+        Map<String, Object> result = new HashMap();
+        ExecutorSummary executorSummary = executorAggregateStats.get_exec_summary();
+        ExecutorInfo executorInfo = executorSummary.get_executor_info();
+        ComponentAggregateStats componentAggregateStats = executorAggregateStats.get_stats();
+        SpecificAggregateStats specificAggregateStats = componentAggregateStats.get_specific_stats();
+        BoltAggregateStats boltAggregateStats = specificAggregateStats.get_bolt();
+        CommonAggregateStats commonAggregateStats = componentAggregateStats.get_common_stats();
+        String executorId = prettyExecutorInfo(executorInfo);
+        result.put("id", executorId);
+        result.put("encodedId", URLEncoder.encode(executorId));
+        result.put("uptime", prettyUptimeSec(executorSummary.get_uptime_secs()));
+        result.put("uptimeSeconds", executorSummary.get_uptime_secs());
+        String host = executorSummary.get_host();
+        result.put("host", host);
+        int port = executorSummary.get_port();
+        result.put("port", port);
+        result.put("emitted", nullToZero(commonAggregateStats.get_emitted()));
+        result.put("transferred", nullToZero(commonAggregateStats.get_transferred()));
+        result.put("capacity",  StatsUtil.floatStr(nullToZero(boltAggregateStats.get_capacity())));
+        result.put("executeLatency", StatsUtil.floatStr(boltAggregateStats.get_execute_latency_ms()));
+        result.put("executed", nullToZero(boltAggregateStats.get_executed()));
+        result.put("processLatency", StatsUtil.floatStr(boltAggregateStats.get_process_latency_ms()));
+        result.put("acked", nullToZero(commonAggregateStats.get_acked()));
+        result.put("failed", nullToZero(commonAggregateStats.get_failed()));
+        result.put("workerLogLink", getWorkerLogLink(host, port, config, topologyId));
+        return result;
+    }
+
+    private static Map<String, Object> getSpoutExecutorStats(String topologyId, Map<String, Object> config,
+                                                             ExecutorAggregateStats executorAggregateStats) {
+        Map<String, Object> result = new HashMap();
+        ExecutorSummary executorSummary = executorAggregateStats.get_exec_summary();
+        ExecutorInfo executorInfo = executorSummary.get_executor_info();
+        ComponentAggregateStats componentAggregateStats = executorAggregateStats.get_stats();
+        SpecificAggregateStats specificAggregateStats = componentAggregateStats.get_specific_stats();
+        SpoutAggregateStats spoutAggregateStats = specificAggregateStats.get_spout();
+        CommonAggregateStats commonAggregateStats = componentAggregateStats.get_common_stats();
+        String executorId = prettyExecutorInfo(executorInfo);
+        result.put("id", executorId);
+        result.put("encodedId", URLEncoder.encode(executorId));
+        result.put("uptime", prettyUptimeSec(executorSummary.get_uptime_secs()));
+        result.put("uptimeSeconds", executorSummary.get_uptime_secs());
+        String host = executorSummary.get_host();
+        result.put("host", host);
+        int port = executorSummary.get_port();
+        result.put("port", port);
+        result.put("emitted", nullToZero(commonAggregateStats.get_emitted()));
+        result.put("transferred", nullToZero(commonAggregateStats.get_transferred()));
+        result.put("completeLatency", StatsUtil.floatStr(spoutAggregateStats.get_complete_latency_ms()));
+        result.put("acked", nullToZero(commonAggregateStats.get_acked()));
+        result.put("failed", nullToZero(commonAggregateStats.get_failed()));
+        result.put("workerLogLink", getWorkerLogLink(host, port, config, topologyId));
+        return result;
+    }
+
+    private static Map<String, Object> getComponentErrorInfo(ErrorInfo errorInfo, Map config,
+                                                             String topologyId) {
+        Map<String, Object> result = new HashMap();
+        result.put("time", 1000 * (long) errorInfo.get_error_time_secs());
+        String host = errorInfo.get_host();
+        result.put("errorHost", host);
+        int port = errorInfo.get_port();
+        result.put("errorPort", port);
+        result.put("errorWorkerLogLink", getWorkerLogLink(host, port, config, topologyId));
+        result.put("errorLapsedSecs", System.currentTimeMillis() / 1000 - errorInfo.get_error_time_secs());
+        result.put("error", errorInfo.get_error());
+        return result;
+    }
+
+    private static Map<String, Object> getComponentErrors(List<ErrorInfo> errorInfoList,
+                                                          String topologyId, Map config) {
+        Map<String, Object> result = new HashMap();
+        errorInfoList.sort(Comparator.comparingInt(ErrorInfo::get_error_time_secs));
         result.put(
-                "requestedMemOnHeap",
-                commonAggregateStats.get_resources_map().get(Constants.COMMON_ONHEAP_MEMORY_RESOURCE_NAME)
-        );
-        result.put(
-                "requestedMemOffHeap",
-                commonAggregateStats.get_resources_map().get(Constants.COMMON_OFFHEAP_MEMORY_RESOURCE_NAME));
+                "componentErrors",
+                errorInfoList.stream().map(
+                        e -> getComponentErrorInfo(e, config, topologyId)
+        ).collect(Collectors.toList()));
+        return result;
+    }
+
+    private static Map<String, Object> getTopologyErrors(List<ErrorInfo> errorInfoList,
+                                                         String topologyId, Map config) {
+        Map<String, Object> result = new HashMap();
+        errorInfoList.sort(Comparator.comparingInt(ErrorInfo::get_error_time_secs));
         result.put(
-                "requestedCpu" ,
-                commonAggregateStats.get_resources_map().get(Constants.COMMON_CPU_RESOURCE_NAME));
+                "topologyErrors",
+                errorInfoList.stream().map(
+                        e -> getComponentErrorInfo(e, config, topologyId)
+        ).collect(Collectors.toList()));
         return result;
     }
 
-    private static Map<String, Object> getSpoutAggStatsMap(
-            ComponentAggregateStats componentAggregateStats, String spoutId) {
+    private static Map<String, Object> getTopologySpoutAggStatsMap(ComponentAggregateStats componentAggregateStats,
+                                                                   String spoutId) {
         Map<String, Object> result = new HashMap();
-        result.putAll(getCommonAggStatsMap(componentAggregateStats.get_common_stats()));
+        CommonAggregateStats commonStats = componentAggregateStats.get_common_stats();
+        result.putAll(getCommonAggStatsMap(commonStats));
         result.put("spoutId", spoutId);
         result.put("encodedSpoutId", URLEncoder.encode(spoutId));
-        result.put("completeLatency",
-                componentAggregateStats.get_specific_stats().get_spout().get_complete_latency_ms());
+        SpoutAggregateStats spoutAggregateStats = componentAggregateStats.get_specific_stats().get_spout();
+        result.put("completeLatency", spoutAggregateStats.get_complete_latency_ms());
+        ErrorInfo lastError = componentAggregateStats.get_last_error();
+        result.put("lastError", Objects.isNull(lastError) ?  "" : getTruncatedErrorString(lastError.get_error()));
         return result;
     }
 
-    private static Map<String, Object> getBoltAggStatsMap(
-            ComponentAggregateStats componentAggregateStats, String boltId) {
+    private static Map<String, Object> getTopologyBoltAggStatsMap(ComponentAggregateStats componentAggregateStats,
+                                                                  String boltId) {
         Map<String, Object> result = new HashMap();
-        result.putAll(getCommonAggStatsMap(componentAggregateStats.get_common_stats()));
+        CommonAggregateStats commonStats = componentAggregateStats.get_common_stats();
+        result.putAll(getCommonAggStatsMap(commonStats));
         result.put("boltId", boltId);
         result.put("encodedBoltId", URLEncoder.encode(boltId));
         BoltAggregateStats boltAggregateStats = componentAggregateStats.get_specific_stats().get_bolt();
@@ -1131,6 +1387,8 @@ public class UIHelpers {
         result.put("executeLatency", StatsUtil.floatStr(boltAggregateStats.get_execute_latency_ms()));
         result.put("executed", boltAggregateStats.get_executed());
         result.put("processLatency", StatsUtil.floatStr(boltAggregateStats.get_process_latency_ms()));
+        ErrorInfo lastError = componentAggregateStats.get_last_error();
+        result.put("lastError", Objects.isNull(lastError) ?  "" : getTruncatedErrorString(lastError.get_error()));
         return result;
     }
 
@@ -1203,7 +1461,7 @@ public class UIHelpers {
         List<Map> spoutStats = new ArrayList();
 
         for (Map.Entry<String, ComponentAggregateStats> spoutEntry : spouts.entrySet()) {
-            spoutStats.add(getSpoutAggStatsMap(spoutEntry.getValue(), spoutEntry.getKey()));
+            spoutStats.add(getTopologySpoutAggStatsMap(spoutEntry.getValue(), spoutEntry.getKey()));
         }
         result.put("spouts", spoutStats);
 
@@ -1211,7 +1469,7 @@ public class UIHelpers {
         List<Map> boltStats = new ArrayList();
 
         for (Map.Entry<String, ComponentAggregateStats> boltEntry : bolts.entrySet()) {
-            boltStats.add(getBoltAggStatsMap(boltEntry.getValue(), boltEntry.getKey()));
+            boltStats.add(getTopologyBoltAggStatsMap(boltEntry.getValue(), boltEntry.getKey()));
         }
         result.put("bolts", boltStats);
 
@@ -1541,6 +1799,83 @@ public class UIHelpers {
     }
 
     /**
+     * unpackBoltPageInfo.
+     * @param componentPageInfo componentPageInfo
+     * @param topologyId topologyId
+     * @param window window
+     * @param sys sys
+     * @param config config
+     * @return unpackBoltPageInfo
+     */
+    public static Map<String, Object> unpackBoltPageInfo(ComponentPageInfo componentPageInfo,
+                                                         String topologyId, String window, boolean sys,
+                                                         Map config) {
+        Map<String, Object> result = new HashMap();
+
+        result.put(
+                "boltStats",
+                componentPageInfo.get_window_to_stats().entrySet().stream().map(
+                        e -> getBoltAggStatsMap(e.getValue(), e.getKey())
+                ).collect(Collectors.toList())
+        );
+        result.put(
+                "inputStats",
+                componentPageInfo.get_gsid_to_input_stats().entrySet().stream().map(
+                        e -> getBoltInputStats(e.getKey(), e.getValue())
+                ).collect(Collectors.toList())
+        );
+        result.put(
+                "outputStats",
+                componentPageInfo.get_sid_to_output_stats().entrySet().stream().map(
+                    e -> getBoltOutputStats(e.getKey(), e.getValue())
+                ).collect(Collectors.toList())
+        );
+        result.put(
+                "executorStats",
+                componentPageInfo.get_exec_stats().stream().map(
+                        e -> getBoltExecutorStats(topologyId, config, e)
+                ).collect(Collectors.toList())
+        );
+        result.putAll(getComponentErrors(componentPageInfo.get_errors(), topologyId, config));
+        return result;
+    }
+
+    /**
+     * unpackSpoutPageInfo.
+     * @param componentPageInfo componentPageInfo
+     * @param topologyId topologyId
+     * @param window window
+     * @param sys sys
+     * @param config config
+     * @return unpackSpoutPageInfo
+     */
+    public static Map<String, Object> unpackSpoutPageInfo(ComponentPageInfo componentPageInfo,
+                                                          String topologyId, String window, boolean sys,
+                                                          Map config) {
+        Map<String, Object> result = new HashMap();
+        result.put(
+                "spoutSummary",
+                componentPageInfo.get_window_to_stats().entrySet().stream().map(
+                        e -> getSpoutAggStatsMap(e.getValue(), e.getKey())
+                ).collect(Collectors.toList())
+        );
+        result.put(
+                "outputStats",
+                componentPageInfo.get_sid_to_output_stats().entrySet().stream().map(
+                        e -> getSpoutOutputStats(e.getKey(), e.getValue())
+                ).collect(Collectors.toList())
+        );
+        result.put(
+                "executorStats",
+                componentPageInfo.get_exec_stats().stream().map(
+                        e -> getSpoutExecutorStats(topologyId, config, e)
+                ).collect(Collectors.toList())
+        );
+        result.putAll(getComponentErrors(componentPageInfo.get_errors(), topologyId, config));
+        return result;
+    }
+
+    /**
      * getComponentPage.
      * @param client client
      * @param id id
@@ -1559,6 +1894,13 @@ public class UIHelpers {
         ComponentPageInfo componentPageInfo = client.getComponentPageInfo(
                 id, component, window, sys
         );
+
+        if (componentPageInfo.get_component_type().equals(ComponentType.BOLT)) {
+            result.putAll(unpackBoltPageInfo(componentPageInfo, id, window, sys, config));
+        } else if ((componentPageInfo.get_component_type().equals(ComponentType.SPOUT))) {
+            result.putAll(unpackSpoutPageInfo(componentPageInfo, id, window, sys, config));
+        }
+
         result.put("user", user);
         result.put("id" , component);
         result.put("encodedId", URLEncoder.encode(component));


[3/3] storm git commit: Merge branch 'STORM-3217' of https://github.com/govind-menon/storm into STORM-3217

Posted by bo...@apache.org.
Merge branch 'STORM-3217' of https://github.com/govind-menon/storm into STORM-3217

STORM-3217: Fixing getComponentPage API call

This closes #2828


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

Branch: refs/heads/master
Commit: f5a410ba34374023b89876b310a95c3be9dd1c3c
Parents: a1e6e98 a35459d
Author: Robert Evans <ev...@yahoo-inc.com>
Authored: Thu Sep 13 14:53:30 2018 -0500
Committer: Robert Evans <ev...@yahoo-inc.com>
Committed: Thu Sep 13 14:53:30 2018 -0500

----------------------------------------------------------------------
 storm-webapp/pom.xml                            |   2 +-
 .../logviewer/utils/LogFileDownloader.java      |   2 +-
 .../org/apache/storm/daemon/ui/UIHelpers.java   | 437 ++++++++++++++++++-
 .../DefaultExceptionMapper.java                 |   2 +-
 4 files changed, 422 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/f5a410ba/storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIHelpers.java
----------------------------------------------------------------------


[2/3] storm git commit: STORM-3217: Checkstyle fixes

Posted by bo...@apache.org.
STORM-3217: Checkstyle fixes


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

Branch: refs/heads/master
Commit: a35459d98c2c7e24d7acb5f9b90e925b1bb22be7
Parents: a805c70
Author: Govind Menon <go...@gmail.com>
Authored: Thu Sep 13 13:04:53 2018 -0500
Committer: Govind Menon <go...@gmail.com>
Committed: Thu Sep 13 13:04:53 2018 -0500

----------------------------------------------------------------------
 storm-webapp/pom.xml                            |  2 +-
 .../logviewer/utils/LogFileDownloader.java      |  2 +-
 .../org/apache/storm/daemon/ui/UIHelpers.java   | 71 ++++++++++++++++++--
 .../DefaultExceptionMapper.java                 |  2 +-
 4 files changed, 68 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/a35459d9/storm-webapp/pom.xml
----------------------------------------------------------------------
diff --git a/storm-webapp/pom.xml b/storm-webapp/pom.xml
index 649354e..84ec633 100644
--- a/storm-webapp/pom.xml
+++ b/storm-webapp/pom.xml
@@ -309,7 +309,7 @@
                 <artifactId>maven-checkstyle-plugin</artifactId>
                 <!--Note - the version would be inherited-->
                 <configuration>
-                    <maxAllowedViolations>16</maxAllowedViolations>
+                    <maxAllowedViolations>23</maxAllowedViolations>
                 </configuration>
             </plugin>
             <plugin>

http://git-wip-us.apache.org/repos/asf/storm/blob/a35459d9/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/LogFileDownloader.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/LogFileDownloader.java b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/LogFileDownloader.java
index bfb3065..75cb29f 100644
--- a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/LogFileDownloader.java
+++ b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/LogFileDownloader.java
@@ -30,7 +30,7 @@ import org.apache.storm.metric.StormMetricsRegistry;
 
 
 public class LogFileDownloader {
-    private static final Histogram fileDownloadSizeDistMB= StormMetricsRegistry.registerHistogram("logviewer:download-file-size-rounded-MB");
+    private static final Histogram fileDownloadSizeDistMB = StormMetricsRegistry.registerHistogram("logviewer:download-file-size-rounded-MB");
 
     private final String logRoot;
     private final String daemonLogRoot;

http://git-wip-us.apache.org/repos/asf/storm/blob/a35459d9/storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIHelpers.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIHelpers.java b/storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIHelpers.java
index c280cf2..9a5f3bf 100644
--- a/storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIHelpers.java
+++ b/storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIHelpers.java
@@ -1266,6 +1266,13 @@ public class UIHelpers {
         return result;
     }
 
+    /**
+     * getBoltExecutorStats.
+     * @param topologyId topologyId
+     * @param config config
+     * @param executorAggregateStats executorAggregateStats
+     * @return getBoltExecutorStats
+     */
     private static Map<String, Object> getBoltExecutorStats(String topologyId, Map<String, Object> config,
                                                             ExecutorAggregateStats executorAggregateStats) {
         Map<String, Object> result = new HashMap();
@@ -1296,6 +1303,13 @@ public class UIHelpers {
         return result;
     }
 
+    /**
+     * getSpoutExecutorStats.
+     * @param topologyId topologyId
+     * @param config config
+     * @param executorAggregateStats executorAggregateStats
+     * @return getSpoutExecutorStats
+     */
     private static Map<String, Object> getSpoutExecutorStats(String topologyId, Map<String, Object> config,
                                                              ExecutorAggregateStats executorAggregateStats) {
         Map<String, Object> result = new HashMap();
@@ -1323,6 +1337,13 @@ public class UIHelpers {
         return result;
     }
 
+    /**
+     * getComponentErrorInfo.
+     * @param errorInfo errorInfo
+     * @param config config
+     * @param topologyId topologyId
+     * @return getComponentErrorInfo
+     */
     private static Map<String, Object> getComponentErrorInfo(ErrorInfo errorInfo, Map config,
                                                              String topologyId) {
         Map<String, Object> result = new HashMap();
@@ -1337,30 +1358,50 @@ public class UIHelpers {
         return result;
     }
 
+    /**
+     * getComponentErrors.
+     * @param errorInfoList errorInfoList
+     * @param topologyId topologyId
+     * @param config config
+     * @return getComponentErrors
+     */
     private static Map<String, Object> getComponentErrors(List<ErrorInfo> errorInfoList,
                                                           String topologyId, Map config) {
         Map<String, Object> result = new HashMap();
         errorInfoList.sort(Comparator.comparingInt(ErrorInfo::get_error_time_secs));
         result.put(
                 "componentErrors",
-                errorInfoList.stream().map(
-                        e -> getComponentErrorInfo(e, config, topologyId)
-        ).collect(Collectors.toList()));
+                errorInfoList.stream().map(e -> getComponentErrorInfo(e, config, topologyId))
+                        .collect(Collectors.toList())
+        );
         return result;
     }
 
+    /**
+     * getTopologyErrors.
+     * @param errorInfoList errorInfoList
+     * @param topologyId topologyId
+     * @param config config
+     * @return getTopologyErrors
+     */
     private static Map<String, Object> getTopologyErrors(List<ErrorInfo> errorInfoList,
                                                          String topologyId, Map config) {
         Map<String, Object> result = new HashMap();
         errorInfoList.sort(Comparator.comparingInt(ErrorInfo::get_error_time_secs));
         result.put(
                 "topologyErrors",
-                errorInfoList.stream().map(
-                        e -> getComponentErrorInfo(e, config, topologyId)
-        ).collect(Collectors.toList()));
+                errorInfoList.stream().map(e -> getComponentErrorInfo(e, config, topologyId))
+                        .collect(Collectors.toList())
+        );
         return result;
     }
 
+    /**
+     * getTopologySpoutAggStatsMap.
+     * @param componentAggregateStats componentAggregateStats
+     * @param spoutId spoutId
+     * @return getTopologySpoutAggStatsMap
+     */
     private static Map<String, Object> getTopologySpoutAggStatsMap(ComponentAggregateStats componentAggregateStats,
                                                                    String spoutId) {
         Map<String, Object> result = new HashMap();
@@ -1375,6 +1416,12 @@ public class UIHelpers {
         return result;
     }
 
+    /**
+     * getTopologyBoltAggStatsMap.
+     * @param componentAggregateStats componentAggregateStats
+     * @param boltId boltId
+     * @return getTopologyBoltAggStatsMap
+     */
     private static Map<String, Object> getTopologyBoltAggStatsMap(ComponentAggregateStats componentAggregateStats,
                                                                   String boltId) {
         Map<String, Object> result = new HashMap();
@@ -1392,6 +1439,11 @@ public class UIHelpers {
         return result;
     }
 
+    /**
+     * getTopologyStatsMap.
+     * @param topologyStats topologyStats
+     * @return getTopologyStatsMap
+     */
     private static List<Map> getTopologyStatsMap(TopologyStats topologyStats) {
         List<Map> result = new ArrayList();
 
@@ -1416,6 +1468,13 @@ public class UIHelpers {
         return result;
     }
 
+    /**
+     * unpackTopologyInfo.
+     * @param topologyPageInfo topologyPageInfo
+     * @param window window
+     * @param config config
+     * @return unpackTopologyInfo
+     */
     private static Map<String,Object> unpackTopologyInfo(TopologyPageInfo topologyPageInfo, String window, Map<String,Object> config) {
         Map<String, Object> result = new HashMap();
         result.put("id", topologyPageInfo.get_id());

http://git-wip-us.apache.org/repos/asf/storm/blob/a35459d9/storm-webapp/src/main/java/org/apache/storm/daemon/ui/exceptionmappers/DefaultExceptionMapper.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/main/java/org/apache/storm/daemon/ui/exceptionmappers/DefaultExceptionMapper.java b/storm-webapp/src/main/java/org/apache/storm/daemon/ui/exceptionmappers/DefaultExceptionMapper.java
index 73aeb05..b3172e8 100644
--- a/storm-webapp/src/main/java/org/apache/storm/daemon/ui/exceptionmappers/DefaultExceptionMapper.java
+++ b/storm-webapp/src/main/java/org/apache/storm/daemon/ui/exceptionmappers/DefaultExceptionMapper.java
@@ -34,7 +34,7 @@ public class DefaultExceptionMapper implements ExceptionMapper<Throwable> {
 
     /**
      * toResponse.
-     * @param throwable
+     * @param throwable throwable
      * @return response
      */
     @Override