You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@storm.apache.org by GitBox <gi...@apache.org> on 2020/07/21 00:23:54 UTC

[GitHub] [storm] Crim opened a new pull request #3312: [STORM-3652] Last error not displayed in Topology summary storm ui

Crim opened a new pull request #3312:
URL: https://github.com/apache/storm/pull/3312


   ## What is the purpose of the change
   
   Under storm ui "Topology summary" lastError is not getting populated, you need to go into "Component summary"
   
   ## How was the change tested
   
   ### Automated Coverage
   Added automated test coverage for `UIHelper` under `storm-webapp`.  I only added targetted test coverage for the portion I modified as adding test complete coverage to `UIHelper.getTopologySummary()` would be a pretty large project in itself.
   
   For the changes to `Nimbus.java` I was unsure how to add automated test coverage and was unable to find any prior test coverage for it to use as an example.  Open to suggestions here.
   
   ### Manual Testing
   I built a distribution from the source and deployed a test topology that had a bolt configured to periodically report errors.  I then verified the web UI directly.
   
   **Bolt Code**
   ```java
      @Override
       public void execute(Tuple tuple) {
               if (random.nextInt(10) == 0) {
                   collector.reportError(new RuntimeException("This is a random exception"));
               }
       }
   ```
   
   **Topology Page:**
   <img width="1748" alt="image" src="https://user-images.githubusercontent.com/571653/87998655-7e3e3c80-cb33-11ea-8e72-b9485d9ed711.png">
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [storm] Crim commented on a change in pull request #3312: [STORM-3652] Last error not displayed in Topology summary storm ui

Posted by GitBox <gi...@apache.org>.
Crim commented on a change in pull request #3312:
URL: https://github.com/apache/storm/pull/3312#discussion_r471186111



##########
File path: storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIHelpers.java
##########
@@ -1488,16 +1495,21 @@ private static Double nullToZero(Double value) {
      * @return getTopologySpoutAggStatsMap
      */
     private static Map<String, Object> getTopologySpoutAggStatsMap(ComponentAggregateStats componentAggregateStats,
-                                                                   String spoutId) {
+                                                                   String spoutId, Map<String, Object> config, String topologyId) {
         Map<String, Object> result = new HashMap();
         CommonAggregateStats commonStats = componentAggregateStats.get_common_stats();
         result.putAll(getCommonAggStatsMap(commonStats));
         result.put("spoutId", spoutId);
         result.put("encodedSpoutId", Utils.urlEncodeUtf8(spoutId));
         SpoutAggregateStats spoutAggregateStats = componentAggregateStats.get_specific_stats().get_spout();
-        result.put("completeLatency", spoutAggregateStats.get_complete_latency_ms());
+        result.put("completeLatency", StatsUtil.floatStr(spoutAggregateStats.get_complete_latency_ms()));
         ErrorInfo lastError = componentAggregateStats.get_last_error();
-        result.put("lastError", Objects.isNull(lastError) ?  "" : getTruncatedErrorString(lastError.get_error()));
+        if (!Objects.isNull(lastError)) {
+            result.putAll(getComponentErrorInfo(lastError, config, topologyId, true));
+        } else {
+            // Maintain backwards compatibility in the API response by setting empty string value

Review comment:
       Updated the code as well as the API docs.
   
   Thanks!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [storm] Crim commented on pull request #3312: [STORM-3652] Last error not displayed in Topology summary storm ui

Posted by GitBox <gi...@apache.org>.
Crim commented on pull request #3312:
URL: https://github.com/apache/storm/pull/3312#issuecomment-668857893


   > > @Ethanlm When you get a free moment could you take a look at this, or let me know who would be best to review the PR?
   > > Thanks
   > > Stephen
   > 
   > Thanks @Crim. I will try to find some time to review. Sorry for the delay. I have been too busy recently
   
   Its all good!  Take your time,  Appreciate it!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [storm] Crim commented on a change in pull request #3312: [STORM-3652] Last error not displayed in Topology summary storm ui

Posted by GitBox <gi...@apache.org>.
Crim commented on a change in pull request #3312:
URL: https://github.com/apache/storm/pull/3312#discussion_r467363825



##########
File path: storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIHelpers.java
##########
@@ -1488,16 +1495,21 @@ private static Double nullToZero(Double value) {
      * @return getTopologySpoutAggStatsMap
      */
     private static Map<String, Object> getTopologySpoutAggStatsMap(ComponentAggregateStats componentAggregateStats,
-                                                                   String spoutId) {
+                                                                   String spoutId, Map<String, Object> config, String topologyId) {
         Map<String, Object> result = new HashMap();
         CommonAggregateStats commonStats = componentAggregateStats.get_common_stats();
         result.putAll(getCommonAggStatsMap(commonStats));
         result.put("spoutId", spoutId);
         result.put("encodedSpoutId", Utils.urlEncodeUtf8(spoutId));
         SpoutAggregateStats spoutAggregateStats = componentAggregateStats.get_specific_stats().get_spout();
-        result.put("completeLatency", spoutAggregateStats.get_complete_latency_ms());
+        result.put("completeLatency", StatsUtil.floatStr(spoutAggregateStats.get_complete_latency_ms()));
         ErrorInfo lastError = componentAggregateStats.get_last_error();
-        result.put("lastError", Objects.isNull(lastError) ?  "" : getTruncatedErrorString(lastError.get_error()));
+        if (!Objects.isNull(lastError)) {
+            result.putAll(getComponentErrorInfo(lastError, config, topologyId, true));
+        } else {
+            // Maintain backwards compatibility in the API response by setting empty string value

Review comment:
       Since the previous behavior always defaulted `lastError` to empty string, and the API docs show it defaulting to empty string, I've left this here for backwards compatibility. 
   
   If you think it'd be better to just exclude it (as we do with the other keys like `errorPort`, `errorHost`, `errorTime` etc.. let me know and I can remove the else block.
   
   Or alternatively, we could default the missing keys (`errorPort`, `errorHost`, `errorTime`) to null/empty string when no error has been reported.
   
   Thoughts?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [storm] Crim commented on a change in pull request #3312: [STORM-3652] Last error not displayed in Topology summary storm ui

Posted by GitBox <gi...@apache.org>.
Crim commented on a change in pull request #3312:
URL: https://github.com/apache/storm/pull/3312#discussion_r457765625



##########
File path: storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java
##########
@@ -4363,6 +4363,7 @@ private void addBoltAggStats(TopologyPageInfo topoPageInfo, StormTopology topolo
                 CommonAggregateStats commonStats = entry.getValue().get_common_stats();

Review comment:
       Populates the thrift response with the last error for this bolt, if it exists.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [storm] Crim commented on pull request #3312: [STORM-3652] Last error not displayed in Topology summary storm ui

Posted by GitBox <gi...@apache.org>.
Crim commented on pull request #3312:
URL: https://github.com/apache/storm/pull/3312#issuecomment-663322118


   Realized this was also broken for errors reported for Spouts.  Updated the PR with additional fix for spouts and test coverage.  This should be good for review/merge.
   
   Thanks!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [storm] Crim commented on a change in pull request #3312: [STORM-3652] Last error not displayed in Topology summary storm ui

Posted by GitBox <gi...@apache.org>.
Crim commented on a change in pull request #3312:
URL: https://github.com/apache/storm/pull/3312#discussion_r467363825



##########
File path: storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIHelpers.java
##########
@@ -1488,16 +1495,21 @@ private static Double nullToZero(Double value) {
      * @return getTopologySpoutAggStatsMap
      */
     private static Map<String, Object> getTopologySpoutAggStatsMap(ComponentAggregateStats componentAggregateStats,
-                                                                   String spoutId) {
+                                                                   String spoutId, Map<String, Object> config, String topologyId) {
         Map<String, Object> result = new HashMap();
         CommonAggregateStats commonStats = componentAggregateStats.get_common_stats();
         result.putAll(getCommonAggStatsMap(commonStats));
         result.put("spoutId", spoutId);
         result.put("encodedSpoutId", Utils.urlEncodeUtf8(spoutId));
         SpoutAggregateStats spoutAggregateStats = componentAggregateStats.get_specific_stats().get_spout();
-        result.put("completeLatency", spoutAggregateStats.get_complete_latency_ms());
+        result.put("completeLatency", StatsUtil.floatStr(spoutAggregateStats.get_complete_latency_ms()));
         ErrorInfo lastError = componentAggregateStats.get_last_error();
-        result.put("lastError", Objects.isNull(lastError) ?  "" : getTruncatedErrorString(lastError.get_error()));
+        if (!Objects.isNull(lastError)) {
+            result.putAll(getComponentErrorInfo(lastError, config, topologyId, true));
+        } else {
+            // Maintain backwards compatibility in the API response by setting empty string value

Review comment:
       Since the previous behavior always defaulted `lastError` to empty string, and the API docs show it defaulting to empty string, I've left this here for backwards compatibility. 
   
   If you think it'd be better to just exclude it (as we do with the other keys like `errorPort`, `errorHost`, `errorTime` etc.. let me know and I can remove the else block.
   
   Or alternatively, we could default the missing keys (`errorPort`, `errorHost`, `errorTime`) to null/empty string when no error has been reported to make their behavior consistent with `lastError`
   
   Thoughts?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [storm] Ethanlm commented on a change in pull request #3312: [STORM-3652] Last error not displayed in Topology summary storm ui

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3312:
URL: https://github.com/apache/storm/pull/3312#discussion_r467246987



##########
File path: storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java
##########
@@ -4309,6 +4309,7 @@ private void addSpoutAggStats(TopologyPageInfo topoPageInfo, StormTopology topol
                 CommonAggregateStats commonStats = entry.getValue().get_common_stats();
                 setResourcesDefaultIfNotSet(spoutResources, entry.getKey(), topoConf);
                 commonStats.set_resources_map(spoutResources.get(entry.getKey()).toNormalizedMap());
+                entry.getValue().set_last_error(stormClusterState.lastError(topoPageInfo.get_id(), entry.getKey()));

Review comment:
       I believe the bug is here:
   
   it puts "last-error" in the stat (wrongly),
   https://github.com/apache/storm/blob/e82b9a707db38a814dbd2d8f85eeaac0fda0b75b/storm-server/src/main/java/org/apache/storm/stats/StatsUtil.java#L663
   
   but retrieves it with "lastError"
   https://github.com/apache/storm/blob/e82b9a707db38a814dbd2d8f85eeaac0fda0b75b/storm-server/src/main/java/org/apache/storm/stats/StatsUtil.java#L1907
   
   
   This is a translation error from clojure to java since in clojure (1.x):
   https://github.com/apache/storm/blob/1.x-branch/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj#L2269
   
   https://github.com/apache/storm/blob/1.x-branch/storm-core/src/clj/org/apache/storm/stats.clj#L915
   
   

##########
File path: storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIHelpers.java
##########
@@ -1519,8 +1518,12 @@ private static Double nullToZero(Double value) {
         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()));
+        if (!Objects.isNull(lastError)) {

Review comment:
       We can add a parameter to `getComponentErrorInfo(xxx, boolean asLastError)` and when it is lastError, 
   ```
   if (asLastError) {
   result.put("lastError",getTruncatedErrorString)errorInfo.get_error()));  
   } else {
   result.put("error", errorInfo.get_error());
   }
   ```
    to avoid unnecessary results

##########
File path: storm-webapp/src/test/java/org/apache/storm/daemon/ui/UIHelpersTest.java
##########
@@ -0,0 +1,543 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.daemon.ui;
+
+import org.apache.storm.Config;
+import org.apache.storm.Constants;
+import org.apache.storm.generated.BoltAggregateStats;
+import org.apache.storm.generated.CommonAggregateStats;
+import org.apache.storm.generated.ComponentAggregateStats;
+import org.apache.storm.generated.ErrorInfo;
+import org.apache.storm.generated.SpecificAggregateStats;
+import org.apache.storm.generated.SpoutAggregateStats;
+import org.apache.storm.generated.TopologyPageInfo;
+import org.apache.storm.generated.TopologyStats;
+import org.json.simple.JSONValue;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+class UIHelpersTest {
+    private static final String TOPOLOGY_ID = "Test-Topology-Id";
+    private static final long TOPOLOGY_MESSAGE_TIMEOUT_SECS = 100L;
+    private static final String WINDOW = ":all-time";
+
+    /**
+     * Default empty TopologyPageInfo instance to be extended in each test case.
+     */
+    private TopologyPageInfo topoPageInfo;
+
+    /**
+     * Setups up bare minimum TopologyPageInfo instance such that we can pass to
+     * UIHelpers.getTopologySummary() without it throwing a NPE.
+     *
+     * This should provide a base for which other tests can be written, but will
+     * require populating additional values as needed for each test case.
+     */
+    @BeforeEach
+    void setup() {
+        // Create topology config and serialize to JSON.
+        final Map<String, Object> topologyConfig = new HashMap<>();
+        topologyConfig.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, TOPOLOGY_MESSAGE_TIMEOUT_SECS);
+        final String topoConfigJson = JSONValue.toJSONString(topologyConfig);
+
+        // Create empty TopologyStats instance
+        final TopologyStats topologyStats = new TopologyStats();
+        topologyStats.set_window_to_emitted(new HashMap<>());
+        topologyStats.set_window_to_transferred(new HashMap<>());
+        topologyStats.set_window_to_acked(new HashMap<>());
+        topologyStats.set_window_to_complete_latencies_ms(new HashMap<>());
+        topologyStats.set_window_to_failed(new HashMap<>());
+
+        // Create empty AggregateStats instances.
+        final Map<String,ComponentAggregateStats> idToSpoutAggStats = new HashMap<>();
+
+        final Map<String,ComponentAggregateStats> idToBoltAggStats = new HashMap<>();
+
+        // Build up TopologyPageInfo instance
+        topoPageInfo = new TopologyPageInfo();
+        topoPageInfo.set_topology_conf(topoConfigJson);
+        topoPageInfo.set_id(TOPOLOGY_ID);
+        topoPageInfo.set_topology_stats(topologyStats);
+        topoPageInfo.set_id_to_spout_agg_stats(idToSpoutAggStats);
+        topoPageInfo.set_id_to_bolt_agg_stats(idToBoltAggStats);
+    }
+
+    /**
+     * Very narrow test case to validate that 'last error' fields are populated for a bolt
+     * with an error is present.
+     */
+    @Test
+    void test_getTopologyBoltAggStatsMap_includesLastError() {
+        // Define inputs
+        final String expectedBoltId = "MyBoltId";
+        final String expectedErrorMsg = "This is my test error message";
+        final int expectedErrorTime = (int) TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis());
+        final int expectedErrorPort = 4321;
+        final String expectedErrorHost = "my.errored.host";
+
+        // Define our Last Error
+        final ErrorInfo expectedLastError = new ErrorInfo(expectedErrorMsg, expectedErrorTime);
+        expectedLastError.set_port(expectedErrorPort);
+        expectedLastError.set_host(expectedErrorHost);
+
+        // Build stats instance for our bolt
+        final ComponentAggregateStats aggregateStats = buildBoltAggregateStatsBase();
+        aggregateStats.set_last_error(expectedLastError);
+        addBoltStats(expectedBoltId, aggregateStats);
+
+        // Call method under test.
+        final Map<String, Object> result = UIHelpers.getTopologySummary(
+            topoPageInfo,
+            WINDOW,
+            new HashMap<>(),
+            "spp"
+        );
+
+        // Validate
+        assertNotNull(result, "Should never return null");
+
+        // Validate our Bolt result
+        final Map<String, Object> boltResult = getBoltStatsFromTopologySummaryResult(result, expectedBoltId);
+        assertNotNull(boltResult, "Should have an entry for bolt");
+
+        // Verify each piece
+        assertEquals(expectedBoltId, boltResult.get("boltId"));
+        assertEquals(expectedBoltId, boltResult.get("encodedBoltId"));
+
+        // Verify error
+        assertEquals(expectedErrorMsg, boltResult.get("error"));
+        assertEquals(expectedErrorMsg, boltResult.get("lastError"));
+        assertEquals(expectedErrorPort, boltResult.get("errorPort"));
+        assertEquals(expectedErrorHost, boltResult.get("errorHost"));
+        assertEquals(expectedErrorTime, boltResult.get("errorTime"));
+
+        // Fuzzy matching

Review comment:
       You can use simulatedTime https://github.com/apache/storm/blob/master/storm-client/src/jvm/org/apache/storm/utils/Time.java#L30. 
   
   And then the check can be accurate. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [storm] Crim commented on a change in pull request #3312: [STORM-3652] Last error not displayed in Topology summary storm ui

Posted by GitBox <gi...@apache.org>.
Crim commented on a change in pull request #3312:
URL: https://github.com/apache/storm/pull/3312#discussion_r469190952



##########
File path: storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIHelpers.java
##########
@@ -1488,16 +1495,21 @@ private static Double nullToZero(Double value) {
      * @return getTopologySpoutAggStatsMap
      */
     private static Map<String, Object> getTopologySpoutAggStatsMap(ComponentAggregateStats componentAggregateStats,
-                                                                   String spoutId) {
+                                                                   String spoutId, Map<String, Object> config, String topologyId) {
         Map<String, Object> result = new HashMap();
         CommonAggregateStats commonStats = componentAggregateStats.get_common_stats();
         result.putAll(getCommonAggStatsMap(commonStats));
         result.put("spoutId", spoutId);
         result.put("encodedSpoutId", Utils.urlEncodeUtf8(spoutId));
         SpoutAggregateStats spoutAggregateStats = componentAggregateStats.get_specific_stats().get_spout();
-        result.put("completeLatency", spoutAggregateStats.get_complete_latency_ms());
+        result.put("completeLatency", StatsUtil.floatStr(spoutAggregateStats.get_complete_latency_ms()));
         ErrorInfo lastError = componentAggregateStats.get_last_error();
-        result.put("lastError", Objects.isNull(lastError) ?  "" : getTruncatedErrorString(lastError.get_error()));
+        if (!Objects.isNull(lastError)) {
+            result.putAll(getComponentErrorInfo(lastError, config, topologyId, true));
+        } else {
+            // Maintain backwards compatibility in the API response by setting empty string value

Review comment:
       Sorry, it's unclear, would you prefer that I also default `errorPort`, `errorHost`, and `errorTime` to empty string?
   
   Thanks!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [storm] Crim commented on a change in pull request #3312: [STORM-3652] Last error not displayed in Topology summary storm ui

Posted by GitBox <gi...@apache.org>.
Crim commented on a change in pull request #3312:
URL: https://github.com/apache/storm/pull/3312#discussion_r467363825



##########
File path: storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIHelpers.java
##########
@@ -1488,16 +1495,21 @@ private static Double nullToZero(Double value) {
      * @return getTopologySpoutAggStatsMap
      */
     private static Map<String, Object> getTopologySpoutAggStatsMap(ComponentAggregateStats componentAggregateStats,
-                                                                   String spoutId) {
+                                                                   String spoutId, Map<String, Object> config, String topologyId) {
         Map<String, Object> result = new HashMap();
         CommonAggregateStats commonStats = componentAggregateStats.get_common_stats();
         result.putAll(getCommonAggStatsMap(commonStats));
         result.put("spoutId", spoutId);
         result.put("encodedSpoutId", Utils.urlEncodeUtf8(spoutId));
         SpoutAggregateStats spoutAggregateStats = componentAggregateStats.get_specific_stats().get_spout();
-        result.put("completeLatency", spoutAggregateStats.get_complete_latency_ms());
+        result.put("completeLatency", StatsUtil.floatStr(spoutAggregateStats.get_complete_latency_ms()));
         ErrorInfo lastError = componentAggregateStats.get_last_error();
-        result.put("lastError", Objects.isNull(lastError) ?  "" : getTruncatedErrorString(lastError.get_error()));
+        if (!Objects.isNull(lastError)) {
+            result.putAll(getComponentErrorInfo(lastError, config, topologyId, true));
+        } else {
+            // Maintain backwards compatibility in the API response by setting empty string value

Review comment:
       Since the previous behavior always defaulted `lastError` to empty string, and the API docs show it defaulting to empty string, I've left this here for backwards compatibility. 
   
   If you think it'd be better to just exclude it (as we do with the other keys like `errorPort`, `errorHost`, `errorTime` etc.. let me know and I can remove the else block.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [storm] Crim commented on a change in pull request #3312: [STORM-3652] Last error not displayed in Topology summary storm ui

Posted by GitBox <gi...@apache.org>.
Crim commented on a change in pull request #3312:
URL: https://github.com/apache/storm/pull/3312#discussion_r469190189



##########
File path: storm-core/test/jvm/org/apache/storm/stats/TestStatsUtil.java
##########
@@ -92,6 +113,33 @@ public void makeTopoInfo() {
         worker2Resources.put(new WorkerSlot("node3", 3), ws3);
     }
 
+    private Map<String, Object> createBeatBoltStats() {
+        return createBeatStats("bolt");
+    }
+
+    private Map<String, Object> createBeatSpoutStats() {
+        return createBeatStats("spout");
+    }
+
+    private Map<String, Object> createBeatStats(final String type) {
+        Map<String, Object> stats = new HashMap<>();
+        stats.put("type", type);
+
+        stats.put("acked", new HashMap<>());

Review comment:
       Updated to only include the fields required for each component type.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [storm] Ethanlm commented on pull request #3312: [STORM-3652] Last error not displayed in Topology summary storm ui

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on pull request #3312:
URL: https://github.com/apache/storm/pull/3312#issuecomment-668633629


   > @Ethanlm When you get a free moment could you take a look at this, or let me know who would be best to review the PR?
   > 
   > Thanks
   > Stephen
   
   Thanks @Crim. I will try to find some time to review. Sorry for the delay. I have been too busy recently


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [storm] Crim commented on pull request #3312: [STORM-3652] Last error not displayed in Topology summary storm ui

Posted by GitBox <gi...@apache.org>.
Crim commented on pull request #3312:
URL: https://github.com/apache/storm/pull/3312#issuecomment-676844261


   Thanks for the help getting it merged in!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [storm] Ethanlm merged pull request #3312: [STORM-3652] Last error not displayed in Topology summary storm ui

Posted by GitBox <gi...@apache.org>.
Ethanlm merged pull request #3312:
URL: https://github.com/apache/storm/pull/3312


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [storm] Ethanlm commented on a change in pull request #3312: [STORM-3652] Last error not displayed in Topology summary storm ui

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3312:
URL: https://github.com/apache/storm/pull/3312#discussion_r469667007



##########
File path: storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIHelpers.java
##########
@@ -1488,16 +1495,21 @@ private static Double nullToZero(Double value) {
      * @return getTopologySpoutAggStatsMap
      */
     private static Map<String, Object> getTopologySpoutAggStatsMap(ComponentAggregateStats componentAggregateStats,
-                                                                   String spoutId) {
+                                                                   String spoutId, Map<String, Object> config, String topologyId) {
         Map<String, Object> result = new HashMap();
         CommonAggregateStats commonStats = componentAggregateStats.get_common_stats();
         result.putAll(getCommonAggStatsMap(commonStats));
         result.put("spoutId", spoutId);
         result.put("encodedSpoutId", Utils.urlEncodeUtf8(spoutId));
         SpoutAggregateStats spoutAggregateStats = componentAggregateStats.get_specific_stats().get_spout();
-        result.put("completeLatency", spoutAggregateStats.get_complete_latency_ms());
+        result.put("completeLatency", StatsUtil.floatStr(spoutAggregateStats.get_complete_latency_ms()));
         ErrorInfo lastError = componentAggregateStats.get_last_error();
-        result.put("lastError", Objects.isNull(lastError) ?  "" : getTruncatedErrorString(lastError.get_error()));
+        if (!Objects.isNull(lastError)) {
+            result.putAll(getComponentErrorInfo(lastError, config, topologyId, true));
+        } else {
+            // Maintain backwards compatibility in the API response by setting empty string value

Review comment:
       Sorry for not being clear. I think `lastError`, `errorHost`, `errorPort`, `errorWorkerLogLink` can default to empty string, and `errorTime`, `errorLapsedSecs` can default to `null` . What do you think?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [storm] Ethanlm commented on a change in pull request #3312: [STORM-3652] Last error not displayed in Topology summary storm ui

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3312:
URL: https://github.com/apache/storm/pull/3312#discussion_r467261976



##########
File path: storm-webapp/src/test/java/org/apache/storm/daemon/ui/UIHelpersTest.java
##########
@@ -0,0 +1,543 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.daemon.ui;
+
+import org.apache.storm.Config;
+import org.apache.storm.Constants;
+import org.apache.storm.generated.BoltAggregateStats;
+import org.apache.storm.generated.CommonAggregateStats;
+import org.apache.storm.generated.ComponentAggregateStats;
+import org.apache.storm.generated.ErrorInfo;
+import org.apache.storm.generated.SpecificAggregateStats;
+import org.apache.storm.generated.SpoutAggregateStats;
+import org.apache.storm.generated.TopologyPageInfo;
+import org.apache.storm.generated.TopologyStats;
+import org.json.simple.JSONValue;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+class UIHelpersTest {
+    private static final String TOPOLOGY_ID = "Test-Topology-Id";
+    private static final long TOPOLOGY_MESSAGE_TIMEOUT_SECS = 100L;
+    private static final String WINDOW = ":all-time";
+
+    /**
+     * Default empty TopologyPageInfo instance to be extended in each test case.
+     */
+    private TopologyPageInfo topoPageInfo;
+
+    /**
+     * Setups up bare minimum TopologyPageInfo instance such that we can pass to
+     * UIHelpers.getTopologySummary() without it throwing a NPE.
+     *
+     * This should provide a base for which other tests can be written, but will
+     * require populating additional values as needed for each test case.
+     */
+    @BeforeEach
+    void setup() {
+        // Create topology config and serialize to JSON.
+        final Map<String, Object> topologyConfig = new HashMap<>();
+        topologyConfig.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, TOPOLOGY_MESSAGE_TIMEOUT_SECS);
+        final String topoConfigJson = JSONValue.toJSONString(topologyConfig);
+
+        // Create empty TopologyStats instance
+        final TopologyStats topologyStats = new TopologyStats();
+        topologyStats.set_window_to_emitted(new HashMap<>());
+        topologyStats.set_window_to_transferred(new HashMap<>());
+        topologyStats.set_window_to_acked(new HashMap<>());
+        topologyStats.set_window_to_complete_latencies_ms(new HashMap<>());
+        topologyStats.set_window_to_failed(new HashMap<>());
+
+        // Create empty AggregateStats instances.
+        final Map<String,ComponentAggregateStats> idToSpoutAggStats = new HashMap<>();
+
+        final Map<String,ComponentAggregateStats> idToBoltAggStats = new HashMap<>();
+
+        // Build up TopologyPageInfo instance
+        topoPageInfo = new TopologyPageInfo();
+        topoPageInfo.set_topology_conf(topoConfigJson);
+        topoPageInfo.set_id(TOPOLOGY_ID);
+        topoPageInfo.set_topology_stats(topologyStats);
+        topoPageInfo.set_id_to_spout_agg_stats(idToSpoutAggStats);
+        topoPageInfo.set_id_to_bolt_agg_stats(idToBoltAggStats);
+    }
+
+    /**
+     * Very narrow test case to validate that 'last error' fields are populated for a bolt
+     * with an error is present.
+     */
+    @Test
+    void test_getTopologyBoltAggStatsMap_includesLastError() {
+        // Define inputs
+        final String expectedBoltId = "MyBoltId";
+        final String expectedErrorMsg = "This is my test error message";
+        final int expectedErrorTime = (int) TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis());
+        final int expectedErrorPort = 4321;
+        final String expectedErrorHost = "my.errored.host";
+
+        // Define our Last Error
+        final ErrorInfo expectedLastError = new ErrorInfo(expectedErrorMsg, expectedErrorTime);
+        expectedLastError.set_port(expectedErrorPort);
+        expectedLastError.set_host(expectedErrorHost);
+
+        // Build stats instance for our bolt
+        final ComponentAggregateStats aggregateStats = buildBoltAggregateStatsBase();
+        aggregateStats.set_last_error(expectedLastError);
+        addBoltStats(expectedBoltId, aggregateStats);
+
+        // Call method under test.
+        final Map<String, Object> result = UIHelpers.getTopologySummary(
+            topoPageInfo,
+            WINDOW,
+            new HashMap<>(),
+            "spp"
+        );
+
+        // Validate
+        assertNotNull(result, "Should never return null");
+
+        // Validate our Bolt result
+        final Map<String, Object> boltResult = getBoltStatsFromTopologySummaryResult(result, expectedBoltId);
+        assertNotNull(boltResult, "Should have an entry for bolt");
+
+        // Verify each piece
+        assertEquals(expectedBoltId, boltResult.get("boltId"));
+        assertEquals(expectedBoltId, boltResult.get("encodedBoltId"));
+
+        // Verify error
+        assertEquals(expectedErrorMsg, boltResult.get("error"));
+        assertEquals(expectedErrorMsg, boltResult.get("lastError"));
+        assertEquals(expectedErrorPort, boltResult.get("errorPort"));
+        assertEquals(expectedErrorHost, boltResult.get("errorHost"));
+        assertEquals(expectedErrorTime, boltResult.get("errorTime"));
+
+        // Fuzzy matching

Review comment:
       You can use simulatedTime https://github.com/apache/storm/blob/master/storm-client/src/jvm/org/apache/storm/utils/Time.java#L30. 
   
   And then the check can be accurate. 
   
   Also please check if https://github.com/apache/storm/blob/master/docs/STORM-UI-REST-API.md#apiv1topologyid-get needs to be updated.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [storm] Crim commented on a change in pull request #3312: [STORM-3652] Last error not displayed in Topology summary storm ui

Posted by GitBox <gi...@apache.org>.
Crim commented on a change in pull request #3312:
URL: https://github.com/apache/storm/pull/3312#discussion_r471187489



##########
File path: docs/STORM-UI-REST-API.md
##########
@@ -192,7 +192,7 @@ Sample response:
             "host":"192.168.202.1",
             "port":6627,
             "nimbusLogLink":"http:\/\/192.168.202.1:8000\/log?file=nimbus.log",
-            "status":Leader,

Review comment:
       Random JSON syntax fix I spotted. :)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [storm] Crim commented on pull request #3312: [STORM-3652] Last error not displayed in Topology summary storm ui

Posted by GitBox <gi...@apache.org>.
Crim commented on pull request #3312:
URL: https://github.com/apache/storm/pull/3312#issuecomment-668343863


   @Ethanlm When you get a free moment could you take a look at this, or let me know who would be best to review the PR?
   
   Thanks
   Stephen


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [storm] Crim commented on a change in pull request #3312: [STORM-3652] Last error not displayed in Topology summary storm ui

Posted by GitBox <gi...@apache.org>.
Crim commented on a change in pull request #3312:
URL: https://github.com/apache/storm/pull/3312#discussion_r459825843



##########
File path: storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIHelpers.java
##########
@@ -1488,16 +1487,21 @@ private static Double nullToZero(Double value) {
      * @return getTopologySpoutAggStatsMap
      */
     private static Map<String, Object> getTopologySpoutAggStatsMap(ComponentAggregateStats componentAggregateStats,
-                                                                   String spoutId) {
+                                                                   String spoutId, Map<String, Object> config, String topologyId) {
         Map<String, Object> result = new HashMap();
         CommonAggregateStats commonStats = componentAggregateStats.get_common_stats();
         result.putAll(getCommonAggStatsMap(commonStats));
         result.put("spoutId", spoutId);
         result.put("encodedSpoutId", Utils.urlEncodeUtf8(spoutId));
         SpoutAggregateStats spoutAggregateStats = componentAggregateStats.get_specific_stats().get_spout();
-        result.put("completeLatency", spoutAggregateStats.get_complete_latency_ms());

Review comment:
       Added to make consistent with how other floats are returned.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [storm] Crim commented on a change in pull request #3312: [STORM-3652] Last error not displayed in Topology summary storm ui

Posted by GitBox <gi...@apache.org>.
Crim commented on a change in pull request #3312:
URL: https://github.com/apache/storm/pull/3312#discussion_r457765344



##########
File path: storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIHelpers.java
##########
@@ -1519,8 +1518,12 @@ private static Double nullToZero(Double value) {
         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()));
+        if (!Objects.isNull(lastError)) {

Review comment:
       The existing [getComponentErrorInfo()](https://github.com/Crim/storm/blob/sp/STORM-3652/storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIHelpers.java#L1430-L1443) method adds the remaining properties that the UI is expecting to properly render the error.  See [topology-page-template.html](https://github.com/Crim/storm/blob/sp/STORM-3652/storm-webapp/src/main/java/org/apache/storm/daemon/ui/WEB-INF/templates/topology-page-template.html#L565-L574)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [storm] Ethanlm commented on pull request #3312: [STORM-3652] Last error not displayed in Topology summary storm ui

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on pull request #3312:
URL: https://github.com/apache/storm/pull/3312#issuecomment-676837538


   Travis failures in Check-Updated-License-Files not related 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [storm] Ethanlm commented on a change in pull request #3312: [STORM-3652] Last error not displayed in Topology summary storm ui

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3312:
URL: https://github.com/apache/storm/pull/3312#discussion_r468301085



##########
File path: storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIHelpers.java
##########
@@ -1488,16 +1495,21 @@ private static Double nullToZero(Double value) {
      * @return getTopologySpoutAggStatsMap
      */
     private static Map<String, Object> getTopologySpoutAggStatsMap(ComponentAggregateStats componentAggregateStats,
-                                                                   String spoutId) {
+                                                                   String spoutId, Map<String, Object> config, String topologyId) {
         Map<String, Object> result = new HashMap();
         CommonAggregateStats commonStats = componentAggregateStats.get_common_stats();
         result.putAll(getCommonAggStatsMap(commonStats));
         result.put("spoutId", spoutId);
         result.put("encodedSpoutId", Utils.urlEncodeUtf8(spoutId));
         SpoutAggregateStats spoutAggregateStats = componentAggregateStats.get_specific_stats().get_spout();
-        result.put("completeLatency", spoutAggregateStats.get_complete_latency_ms());
+        result.put("completeLatency", StatsUtil.floatStr(spoutAggregateStats.get_complete_latency_ms()));
         ErrorInfo lastError = componentAggregateStats.get_last_error();
-        result.put("lastError", Objects.isNull(lastError) ?  "" : getTruncatedErrorString(lastError.get_error()));
+        if (!Objects.isNull(lastError)) {
+            result.putAll(getComponentErrorInfo(lastError, config, topologyId, true));
+        } else {
+            // Maintain backwards compatibility in the API response by setting empty string value

Review comment:
       You are right. Sorry I overlooked. I think we can default those keys to empty like you said

##########
File path: storm-core/test/jvm/org/apache/storm/stats/TestStatsUtil.java
##########
@@ -92,6 +113,33 @@ public void makeTopoInfo() {
         worker2Resources.put(new WorkerSlot("node3", 3), ws3);
     }
 
+    private Map<String, Object> createBeatBoltStats() {
+        return createBeatStats("bolt");
+    }
+
+    private Map<String, Object> createBeatSpoutStats() {
+        return createBeatStats("spout");
+    }
+
+    private Map<String, Object> createBeatStats(final String type) {
+        Map<String, Object> stats = new HashMap<>();
+        stats.put("type", type);
+
+        stats.put("acked", new HashMap<>());

Review comment:
       Bolt and spout have different stats
   https://github.com/apache/storm/blob/master/storm-server/src/main/java/org/apache/storm/stats/StatsUtil.java#L2049-L2085
   
   It would be nice to reflect that here even though the information is not really used  .
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [storm] Crim commented on a change in pull request #3312: [STORM-3652] Last error not displayed in Topology summary storm ui

Posted by GitBox <gi...@apache.org>.
Crim commented on a change in pull request #3312:
URL: https://github.com/apache/storm/pull/3312#discussion_r469672682



##########
File path: storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIHelpers.java
##########
@@ -1488,16 +1495,21 @@ private static Double nullToZero(Double value) {
      * @return getTopologySpoutAggStatsMap
      */
     private static Map<String, Object> getTopologySpoutAggStatsMap(ComponentAggregateStats componentAggregateStats,
-                                                                   String spoutId) {
+                                                                   String spoutId, Map<String, Object> config, String topologyId) {
         Map<String, Object> result = new HashMap();
         CommonAggregateStats commonStats = componentAggregateStats.get_common_stats();
         result.putAll(getCommonAggStatsMap(commonStats));
         result.put("spoutId", spoutId);
         result.put("encodedSpoutId", Utils.urlEncodeUtf8(spoutId));
         SpoutAggregateStats spoutAggregateStats = componentAggregateStats.get_specific_stats().get_spout();
-        result.put("completeLatency", spoutAggregateStats.get_complete_latency_ms());
+        result.put("completeLatency", StatsUtil.floatStr(spoutAggregateStats.get_complete_latency_ms()));
         ErrorInfo lastError = componentAggregateStats.get_last_error();
-        result.put("lastError", Objects.isNull(lastError) ?  "" : getTruncatedErrorString(lastError.get_error()));
+        if (!Objects.isNull(lastError)) {
+            result.putAll(getComponentErrorInfo(lastError, config, topologyId, true));
+        } else {
+            // Maintain backwards compatibility in the API response by setting empty string value

Review comment:
       Sounds good.  I'll update the code and API docs to reflect that.
   
   Thanks!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org