You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2020/07/22 12:52:59 UTC

[GitHub] [nifi] simonbence opened a new pull request #4420: NIFI-7429 Adding status history for system level metrics

simonbence opened a new pull request #4420:
URL: https://github.com/apache/nifi/pull/4420


   [NIFI-7429](https://issues.apache.org/jira/browse/NIFI-7429)
   
   This is a proposal for having historical data about the NiFi node’s status appearing in the NiFi UI. The main purpose was to provide a simple tool makes it possible to check basic performance metrics on the UI.
   
   From implementation perspective the solution is based on the existing status history function which was applied for components like processors. In front end side, the existing code is reused as much as possible, only some minor extension and duplication were needed. The main differences compared to the existing uses were the different trigger (this is reachable from the global menu) and the lack of some parameters like id or group.
   
   The backend side builds on top of VolatileComponentStatusRepository which already responsible for such functions. I tried to add is as an integral part of the existing metric collection, so the frequency of the measurements and the way of triggering is not separated. The metrics themselves are distilled from the SystemDiagnostics and the already collected GarbageCollectionStatus.
   
   Creating the snapshots came with three non-trivial cases I would like to highlight:
   
   1. The GC metrics are not predefined as the type of GC is depending on the running environment and on the actual collectors. This prevented pre-defining the descriptors for them, thus these are created during requests.
   2. Also, some GC metrics (time spent, counters) are growing in monotonous way as the metric collection stores the value shows the accumulated value from the start of the instance. In order to be able to provide the increment since the last measurement, the collection of the GC metrics are using the previous snapshot as baseline.
   3. The processor load average (usually in the form of “2.3” or alike) does not fit into the “long” format used by the functionality without significant information loss. In order to avoid bigger refactors, I introduced a new formatter type, named “FRACTION”. By convention the server side multiplies these metrics using a predefined number (1_000_000 for now) and during visualisation the frontend divides the metric value with the same number. By this, we shift the relevant digits into long value range. Of course, this still comes with precision loss, but for visualisation purposes, this looks sufficient.
   
   Thank you for your time and response!
   
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced 
        in the commit message?
   
   - [ ] Does your PR title start with **NIFI-XXXX** where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically `main`)?
   
   - [ ] Is your initial contribution a single, squashed commit? _Additional commits in response to PR reviewer feedback should be made on this branch and pushed to allow change tracking. Do not `squash` or use `--force` when pushing to allow for clean monitoring of changes._
   
   ### For code changes:
   - [ ] Have you ensured that the full suite of tests is executed via `mvn -Pcontrib-check clean install` at the root `nifi` folder?
   - [ ] Have you written or updated unit tests to verify your changes?
   - [ ] Have you verified that the full build is successful on JDK 8?
   - [ ] Have you verified that the full build is successful on JDK 11?
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? 
   - [ ] If applicable, have you updated the `LICENSE` file, including the main `LICENSE` file under `nifi-assembly`?
   - [ ] If applicable, have you updated the `NOTICE` file, including the main `NOTICE` file found under `nifi-assembly`?
   - [ ] If adding new Properties, have you added `.displayName` in addition to .name (programmatic access) for each of the new properties?
   
   ### For documentation related changes:
   - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check GitHub Actions CI for build issues and submit an update to your PR as soon as possible.
   


----------------------------------------------------------------
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] [nifi] tpalfy commented on a change in pull request #4420: NIFI-7429 Adding status history for system level metrics

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4420:
URL: https://github.com/apache/nifi/pull/4420#discussion_r483607459



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java
##########
@@ -164,6 +183,182 @@ public StatusHistory getRemoteProcessGroupStatusHistory(final String remoteGroup
         return getStatusHistory(remoteGroupId, true, DEFAULT_RPG_METRICS, start, end, preferredDataPoints);
     }
 
+    @Override
+    public StatusHistory getNodeStatusHistory() {
+        final List<NodeStatus> nodeStatusList = nodeStatuses.asList();
+        final List<List<GarbageCollectionStatus>> gcStatusList = gcStatuses.asList();
+        final LinkedList<StatusSnapshot> snapshots = new LinkedList<>();
+
+        final Set<MetricDescriptor<?>> metricDescriptors = new HashSet<>(DEFAULT_NODE_METRICS);
+        final List<MetricDescriptor<List<GarbageCollectionStatus>>> gcMetricDescriptors = new LinkedList<>();
+        final List<MetricDescriptor<List<GarbageCollectionStatus>>> gcMetricDescriptorsDifferential = new LinkedList<>();
+        final List<MetricDescriptor<NodeStatus>> contentStorageStatusDescriptors = new LinkedList<>();
+        final List<MetricDescriptor<NodeStatus>> provenanceStorageStatusDescriptors = new LinkedList<>();
+
+        int ordinal = DEFAULT_NODE_METRICS.size() - 1;

Review comment:
       Instead of calculating the `counter`, 
   `final AtomicInteger index = new AtomicInteger(DEFAULT_NODE_METRICS.size());` could be used with `index.getAndIncrement()` in every `new StandardMetricDescriptor`




----------------------------------------------------------------
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] [nifi] simonbence commented on a change in pull request #4420: NIFI-7429 Adding status history for system level metrics

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4420:
URL: https://github.com/apache/nifi/pull/4420#discussion_r482934126



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
##########
@@ -2691,6 +2691,39 @@ public Response getAction(
         return generateOkResponse(entity).build();
     }
 
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("node/status/history")
+    @ApiOperation(
+            value = "Gets configuration history for the node",
+            notes = NON_GUARANTEED_ENDPOINT,
+            response = ComponentHistoryEntity.class,
+            authorizations = {
+                    @Authorization(value = "Read - /flow")
+            }
+    )
+    @ApiResponses(
+            value = {
+                    @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+                    @ApiResponse(code = 401, message = "Client could not be authenticated."),
+                    @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+                    @ApiResponse(code = 404, message = "The specified resource could not be found."),
+                    @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+            }
+    )
+    public Response getNodeHistory() {
+        authorizeFlow();

Review comment:
       Good point, thanks for highlighting! I moved it to the ControllerResource.




----------------------------------------------------------------
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] [nifi] asfgit closed pull request #4420: NIFI-7429 Adding status history for system level metrics

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #4420:
URL: https://github.com/apache/nifi/pull/4420


   


----------------------------------------------------------------
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] [nifi] simonbence commented on a change in pull request #4420: NIFI-7429 Adding status history for system level metrics

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4420:
URL: https://github.com/apache/nifi/pull/4420#discussion_r484819695



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/StatusHistoryUtil.java
##########
@@ -76,14 +76,14 @@ public static StatusDescriptorDTO createStatusDescriptorDto(final MetricDescript
 
     public static List<StatusDescriptorDTO> createFieldDescriptorDtos(final Collection<MetricDescriptor<?>> metricDescriptors) {
         final List<StatusDescriptorDTO> dtos = new ArrayList<>();
+        final Map<Integer, MetricDescriptor<?>> orderedDescriptors = new HashMap<>();
 
-        final Set<MetricDescriptor<?>> allDescriptors = new LinkedHashSet<>();
         for (final MetricDescriptor<?> metricDescriptor : metricDescriptors) {

Review comment:
       I see your point now. Yes, using array as intermediate data structure is a great improvement.
   
   I was thinking on that if the array based approach, or the stream is better. Here, I think being rigid is not necessarily a bad thing as the order numbers should be consecutive. If the order deviates from the expected, that might manifest as shifted metrics. I would go with fail fast, it should appear at development time.




----------------------------------------------------------------
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] [nifi] simonbence commented on a change in pull request #4420: NIFI-7429 Adding status history for system level metrics

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4420:
URL: https://github.com/apache/nifi/pull/4420#discussion_r484373564



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java
##########
@@ -164,6 +183,182 @@ public StatusHistory getRemoteProcessGroupStatusHistory(final String remoteGroup
         return getStatusHistory(remoteGroupId, true, DEFAULT_RPG_METRICS, start, end, preferredDataPoints);
     }
 
+    @Override
+    public StatusHistory getNodeStatusHistory() {
+        final List<NodeStatus> nodeStatusList = nodeStatuses.asList();
+        final List<List<GarbageCollectionStatus>> gcStatusList = gcStatuses.asList();
+        final LinkedList<StatusSnapshot> snapshots = new LinkedList<>();
+
+        final Set<MetricDescriptor<?>> metricDescriptors = new HashSet<>(DEFAULT_NODE_METRICS);
+        final List<MetricDescriptor<List<GarbageCollectionStatus>>> gcMetricDescriptors = new LinkedList<>();
+        final List<MetricDescriptor<List<GarbageCollectionStatus>>> gcMetricDescriptorsDifferential = new LinkedList<>();
+        final List<MetricDescriptor<NodeStatus>> contentStorageStatusDescriptors = new LinkedList<>();
+        final List<MetricDescriptor<NodeStatus>> provenanceStorageStatusDescriptors = new LinkedList<>();
+
+        int ordinal = DEFAULT_NODE_METRICS.size() - 1;
+
+        // Uses the first measurement as reference for repository metrics descriptors
+        if (nodeStatusList.size() > 0) {
+            final NodeStatus referenceNodeStatus = nodeStatusList.get(0);
+            int contentStorageNumber = 0;
+            int provenanceStorageNumber = 0;
+
+            for (int i = 0; i < referenceNodeStatus.getContentRepositories().size(); i++) {
+                final int storageNumber = i;
+                final int counter = metricDescriptors.size() - 1 + NUMBER_OF_STORAGE_METRICS * contentStorageNumber;
+
+                contentStorageStatusDescriptors.add(new StandardMetricDescriptor<>(

Review comment:
       I reworked this part a bit. Mainly simplification. Also I extracted the descriptor creation parts to make the flow easier to read.




----------------------------------------------------------------
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] [nifi] simonbence commented on a change in pull request #4420: NIFI-7429 Adding status history for system level metrics

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4420:
URL: https://github.com/apache/nifi/pull/4420#discussion_r484372894



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java
##########
@@ -164,6 +183,182 @@ public StatusHistory getRemoteProcessGroupStatusHistory(final String remoteGroup
         return getStatusHistory(remoteGroupId, true, DEFAULT_RPG_METRICS, start, end, preferredDataPoints);
     }
 
+    @Override
+    public StatusHistory getNodeStatusHistory() {
+        final List<NodeStatus> nodeStatusList = nodeStatuses.asList();
+        final List<List<GarbageCollectionStatus>> gcStatusList = gcStatuses.asList();
+        final LinkedList<StatusSnapshot> snapshots = new LinkedList<>();
+
+        final Set<MetricDescriptor<?>> metricDescriptors = new HashSet<>(DEFAULT_NODE_METRICS);
+        final List<MetricDescriptor<List<GarbageCollectionStatus>>> gcMetricDescriptors = new LinkedList<>();
+        final List<MetricDescriptor<List<GarbageCollectionStatus>>> gcMetricDescriptorsDifferential = new LinkedList<>();
+        final List<MetricDescriptor<NodeStatus>> contentStorageStatusDescriptors = new LinkedList<>();
+        final List<MetricDescriptor<NodeStatus>> provenanceStorageStatusDescriptors = new LinkedList<>();
+
+        int ordinal = DEFAULT_NODE_METRICS.size() - 1;

Review comment:
       Thanks, it's simplified a bit!




----------------------------------------------------------------
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] [nifi] tpalfy commented on a change in pull request #4420: NIFI-7429 Adding status history for system level metrics

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4420:
URL: https://github.com/apache/nifi/pull/4420#discussion_r483607459



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java
##########
@@ -164,6 +183,182 @@ public StatusHistory getRemoteProcessGroupStatusHistory(final String remoteGroup
         return getStatusHistory(remoteGroupId, true, DEFAULT_RPG_METRICS, start, end, preferredDataPoints);
     }
 
+    @Override
+    public StatusHistory getNodeStatusHistory() {
+        final List<NodeStatus> nodeStatusList = nodeStatuses.asList();
+        final List<List<GarbageCollectionStatus>> gcStatusList = gcStatuses.asList();
+        final LinkedList<StatusSnapshot> snapshots = new LinkedList<>();
+
+        final Set<MetricDescriptor<?>> metricDescriptors = new HashSet<>(DEFAULT_NODE_METRICS);
+        final List<MetricDescriptor<List<GarbageCollectionStatus>>> gcMetricDescriptors = new LinkedList<>();
+        final List<MetricDescriptor<List<GarbageCollectionStatus>>> gcMetricDescriptorsDifferential = new LinkedList<>();
+        final List<MetricDescriptor<NodeStatus>> contentStorageStatusDescriptors = new LinkedList<>();
+        final List<MetricDescriptor<NodeStatus>> provenanceStorageStatusDescriptors = new LinkedList<>();
+
+        int ordinal = DEFAULT_NODE_METRICS.size() - 1;

Review comment:
       Instead of calculating the `counter`, 
   `final AtomicInteger index = new AtomicInteger(DEFAULT_NODE_METRICS.size());` could be used with `index.incrementAndGet()` in every `new StandardMetricDescriptor`

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java
##########
@@ -164,6 +183,182 @@ public StatusHistory getRemoteProcessGroupStatusHistory(final String remoteGroup
         return getStatusHistory(remoteGroupId, true, DEFAULT_RPG_METRICS, start, end, preferredDataPoints);
     }
 
+    @Override
+    public StatusHistory getNodeStatusHistory() {
+        final List<NodeStatus> nodeStatusList = nodeStatuses.asList();
+        final List<List<GarbageCollectionStatus>> gcStatusList = gcStatuses.asList();
+        final LinkedList<StatusSnapshot> snapshots = new LinkedList<>();
+
+        final Set<MetricDescriptor<?>> metricDescriptors = new HashSet<>(DEFAULT_NODE_METRICS);
+        final List<MetricDescriptor<List<GarbageCollectionStatus>>> gcMetricDescriptors = new LinkedList<>();
+        final List<MetricDescriptor<List<GarbageCollectionStatus>>> gcMetricDescriptorsDifferential = new LinkedList<>();
+        final List<MetricDescriptor<NodeStatus>> contentStorageStatusDescriptors = new LinkedList<>();
+        final List<MetricDescriptor<NodeStatus>> provenanceStorageStatusDescriptors = new LinkedList<>();
+
+        int ordinal = DEFAULT_NODE_METRICS.size() - 1;
+
+        // Uses the first measurement as reference for repository metrics descriptors
+        if (nodeStatusList.size() > 0) {
+            final NodeStatus referenceNodeStatus = nodeStatusList.get(0);
+            int contentStorageNumber = 0;
+            int provenanceStorageNumber = 0;
+
+            for (int i = 0; i < referenceNodeStatus.getContentRepositories().size(); i++) {
+                final int storageNumber = i;
+                final int counter = metricDescriptors.size() - 1 + NUMBER_OF_STORAGE_METRICS * contentStorageNumber;
+
+                contentStorageStatusDescriptors.add(new StandardMetricDescriptor<>(

Review comment:
       Could use
   ```suggestion
                   metricDescriptors.add(new StandardMetricDescriptor<NodeStatus>(
   ```
   With this approach we could get rid of all the intermediary lists.




----------------------------------------------------------------
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] [nifi] simonbence commented on a change in pull request #4420: NIFI-7429 Adding status history for system level metrics

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4420:
URL: https://github.com/apache/nifi/pull/4420#discussion_r484819695



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/StatusHistoryUtil.java
##########
@@ -76,14 +76,14 @@ public static StatusDescriptorDTO createStatusDescriptorDto(final MetricDescript
 
     public static List<StatusDescriptorDTO> createFieldDescriptorDtos(final Collection<MetricDescriptor<?>> metricDescriptors) {
         final List<StatusDescriptorDTO> dtos = new ArrayList<>();
+        final Map<Integer, MetricDescriptor<?>> orderedDescriptors = new HashMap<>();
 
-        final Set<MetricDescriptor<?>> allDescriptors = new LinkedHashSet<>();
         for (final MetricDescriptor<?> metricDescriptor : metricDescriptors) {

Review comment:
       I see your point now. Yes, using array as intermediate data structure is a great improvement.




----------------------------------------------------------------
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] [nifi] simonbence commented on pull request #4420: NIFI-7429 Adding status history for system level metrics

Posted by GitBox <gi...@apache.org>.
simonbence commented on pull request #4420:
URL: https://github.com/apache/nifi/pull/4420#issuecomment-663589327


   Thanks for the feedback @pvillard31 !
   
   I fixed the order of the metrics. This is something was working the same semi-random way with other history panels as well. Now it orders based on "ordinal" of the metrics. Looks much more organized
   
   I also added the thread metrics you were asking for. I found no way to add multiple lines for the diagrams without serious refactors, so I hope it will meet your expectations the way it is. Furthermore I added detailed metrics about the given repositories as well (not only a summary for the given types)


----------------------------------------------------------------
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] [nifi] simonbence commented on a change in pull request #4420: NIFI-7429 Adding status history for system level metrics

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4420:
URL: https://github.com/apache/nifi/pull/4420#discussion_r483623029



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java
##########
@@ -164,6 +183,182 @@ public StatusHistory getRemoteProcessGroupStatusHistory(final String remoteGroup
         return getStatusHistory(remoteGroupId, true, DEFAULT_RPG_METRICS, start, end, preferredDataPoints);
     }
 
+    @Override
+    public StatusHistory getNodeStatusHistory() {

Review comment:
       It is covered in VolatileComponentStatusRepositoryTest#testNodeHistory (which should be renamed however)




----------------------------------------------------------------
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] [nifi] simonbence commented on a change in pull request #4420: NIFI-7429 Adding status history for system level metrics

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4420:
URL: https://github.com/apache/nifi/pull/4420#discussion_r484819695



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/StatusHistoryUtil.java
##########
@@ -76,14 +76,14 @@ public static StatusDescriptorDTO createStatusDescriptorDto(final MetricDescript
 
     public static List<StatusDescriptorDTO> createFieldDescriptorDtos(final Collection<MetricDescriptor<?>> metricDescriptors) {
         final List<StatusDescriptorDTO> dtos = new ArrayList<>();
+        final Map<Integer, MetricDescriptor<?>> orderedDescriptors = new HashMap<>();
 
-        final Set<MetricDescriptor<?>> allDescriptors = new LinkedHashSet<>();
         for (final MetricDescriptor<?> metricDescriptor : metricDescriptors) {

Review comment:
       I see your point now. Yes, using array as intermediate data structure is a great improvement.
   
   I was thinking on that if the array based approach, or the stream is better. Here, I think being rigid is not necessarily a bad thing as the order numbers should be consecutive, but in the end stream approach looks more elegant.




----------------------------------------------------------------
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] [nifi] simonbence commented on a change in pull request #4420: NIFI-7429 Adding status history for system level metrics

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4420:
URL: https://github.com/apache/nifi/pull/4420#discussion_r482777095



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-status-history.js
##########
@@ -78,6 +79,9 @@
         },
         'DATA_SIZE': function (d) {
             return nfCommon.formatDataSize(d);
+        },
+        'FRACTION': function (d) {
+            return nfCommon.formatFloat(d / 1000000);

Review comment:
       What I did find is the metrics functionality depends on long data type as metric data type, especially on Java side. Changing this looks as it would come with serious and possibly risky refactoring. Contrary, the processor load is usually in a range one or two digit number with fractions. I was playing with with some ideas, trying to avoid this method, but in the end they did not work well.




----------------------------------------------------------------
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] [nifi] simonbence commented on pull request #4420: NIFI-7429 Adding status history for system level metrics

Posted by GitBox <gi...@apache.org>.
simonbence commented on pull request #4420:
URL: https://github.com/apache/nifi/pull/4420#issuecomment-662435913


   @pvillard31 @markap14 May I ask you to take a look on this? Thank you very much!


----------------------------------------------------------------
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] [nifi] simonbence commented on a change in pull request #4420: NIFI-7429 Adding status history for system level metrics

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4420:
URL: https://github.com/apache/nifi/pull/4420#discussion_r483633449



##########
File path: nifi-api/src/main/java/org/apache/nifi/controller/status/NodeStatus.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.nifi.controller.status;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * The status of a NiFi node.
+ */
+public class NodeStatus implements Cloneable {

Review comment:
       I checked for StorageStatus versus StorageUsage and now I remember: StorageUsage (the original) is from the `nifi-framework-core` module, but the places we intend to use StorageStatus is in the nifi-api (as these instances are exposed, together with other metrics related DTOs), so I needed to add these.




----------------------------------------------------------------
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] [nifi] simonbence commented on a change in pull request #4420: NIFI-7429 Adding status history for system level metrics

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4420:
URL: https://github.com/apache/nifi/pull/4420#discussion_r483548523



##########
File path: nifi-api/src/main/java/org/apache/nifi/controller/status/NodeStatus.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.nifi.controller.status;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * The status of a NiFi node.
+ */
+public class NodeStatus implements Cloneable {

Review comment:
       As for SystemDiagnostics, NodeStatus consists only a part of it and also consists information from other source. As for StorageStatus, I do check on if we can spare that.




----------------------------------------------------------------
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] [nifi] simonbence edited a comment on pull request #4420: NIFI-7429 Adding status history for system level metrics

Posted by GitBox <gi...@apache.org>.
simonbence edited a comment on pull request #4420:
URL: https://github.com/apache/nifi/pull/4420#issuecomment-662435913


   @pvillard31 @mcgilman May I ask you to take a look on this? Thank you very much!


----------------------------------------------------------------
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] [nifi] pvillard31 commented on pull request #4420: NIFI-7429 Adding status history for system level metrics

Posted by GitBox <gi...@apache.org>.
pvillard31 commented on pull request #4420:
URL: https://github.com/apache/nifi/pull/4420#issuecomment-662568110


   Playing with it and it's awesome, thanks @simonbence for this pull request! Minor suggestions at the moment:
   - ordering of the metrics on the UI side, I'd probably to group things together like: load stats, heap stats, repo stats, gc stats, file handler, etc. Not sure how easy it'd be.
   - there is the total number of threads which is great, it would be nice to also have the number of threads being used from the Timer Driven Thread pool, the number of threads being used from the Event Driven Thread pool. That would be great. Not sure if we display two lines on the same graph, but if possible, displaying the size value of the pool (basically the maximum the value can be over time) would be nice. That would be particularly useful to see if the thread pool is oversized or not.


----------------------------------------------------------------
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] [nifi] pvillard31 commented on pull request #4420: NIFI-7429 Adding status history for system level metrics

Posted by GitBox <gi...@apache.org>.
pvillard31 commented on pull request #4420:
URL: https://github.com/apache/nifi/pull/4420#issuecomment-690321440


   Merged to main, thanks for this awesome improvement !


----------------------------------------------------------------
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] [nifi] tpalfy commented on a change in pull request #4420: NIFI-7429 Adding status history for system level metrics

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4420:
URL: https://github.com/apache/nifi/pull/4420#discussion_r482254565



##########
File path: nifi-api/src/main/java/org/apache/nifi/controller/status/StorageStatus.java
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.nifi.controller.status;
+
+/**
+ * The status of a storage repository.
+ */
+public class StorageStatus {

Review comment:
       Shouldn't this be `Cloneable`?

##########
File path: nifi-docs/src/main/asciidoc/user-guide.adoc
##########
@@ -3089,6 +3089,10 @@ recent change to the dataflow has caused a problem and needs to be fixed. The DF
 adjust the flow as needed to fix the problem. While NiFi does not have an "undo" feature, the DFM can make new changes to the
 dataflow that will fix the problem.
 
+Select Node Status History to view instance specific metrics from the last 24 hours or if the instance runs for less time, then
+since it has been started. The status history can help the DFM in troubleshooting performance barriers and provides a general

Review comment:
       I would use `performance issues` instead of `performance barriers` (which is not used too often in this context).

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/NodeStatusDescriptor.java
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.nifi.controller.status.history;
+
+import org.apache.nifi.controller.status.NodeStatus;
+
+import java.util.List;
+
+public enum NodeStatusDescriptor {
+    FREE_HEAP(
+            "freeHeap",
+            "Free Heap",
+            "The amount of free memory in the heap that can be used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFreeHeap()),
+    USED_HEAP(
+            "usedHeap",
+            "Used Heap",
+            "The amount of used memory in the heap that is used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getUsedHeap()),
+    HEAP_UTILIZATION(
+            "heapUtilization",
+            "Heap Utilization",
+            "The percentage of available heap currently used by the Java virtual machine.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getHeapUtilization(),
+            new ValueReducer<StatusSnapshot, Long>() {
+                @Override
+                public Long reduce(final List<StatusSnapshot> values) {
+                    long sumUtilization = 0L;
+                    int invocations = 0;
+
+                    for (final StatusSnapshot snapshot : values) {
+                        final Long utilization = snapshot.getStatusMetric(HEAP_UTILIZATION.getDescriptor());
+                        if (utilization != null) {
+                            sumUtilization += utilization.longValue();
+                            invocations++;
+                        }
+                    }
+
+                    if (invocations == 0) {
+                        return 0L;
+                    }
+
+                    return sumUtilization / invocations;
+                }
+            }),

Review comment:
       ```suggestion
               new ValueReducer<StatusSnapshot, Long>() {
                   @Override
                   public Long reduce(final List<StatusSnapshot> values) {
                       return (long) values.stream()
                           .map(snapshot -> snapshot.getStatusMetric(HEAP_UTILIZATION.getDescriptor()))
                           .filter(Objects::nonNull)
                           .mapToLong(value -> value)
                           .average()
                           .orElse(0L);
                   }
               }),
   ```

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/NodeStatusDescriptor.java
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.nifi.controller.status.history;
+
+import org.apache.nifi.controller.status.NodeStatus;
+
+import java.util.List;
+
+public enum NodeStatusDescriptor {
+    FREE_HEAP(
+            "freeHeap",
+            "Free Heap",
+            "The amount of free memory in the heap that can be used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFreeHeap()),
+    USED_HEAP(
+            "usedHeap",
+            "Used Heap",
+            "The amount of used memory in the heap that is used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getUsedHeap()),
+    HEAP_UTILIZATION(
+            "heapUtilization",
+            "Heap Utilization",
+            "The percentage of available heap currently used by the Java virtual machine.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getHeapUtilization(),
+            new ValueReducer<StatusSnapshot, Long>() {
+                @Override
+                public Long reduce(final List<StatusSnapshot> values) {
+                    long sumUtilization = 0L;
+                    int invocations = 0;
+
+                    for (final StatusSnapshot snapshot : values) {
+                        final Long utilization = snapshot.getStatusMetric(HEAP_UTILIZATION.getDescriptor());
+                        if (utilization != null) {
+                            sumUtilization += utilization.longValue();
+                            invocations++;
+                        }
+                    }
+
+                    if (invocations == 0) {
+                        return 0L;
+                    }
+
+                    return sumUtilization / invocations;
+                }
+            }),
+    FREE_NON_HEAP(
+            "freeNonHeap",
+            "Free Non Heap",
+            "The currently available non-heap memory that can be used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFreeNonHeap()),
+    USED_NON_HEAP(
+            "usedNonHeap",
+            "Used Non Heap",
+            "The current memory usage of non-heap memory that is used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getUsedNonHeap()),
+    OPEN_FILE_HANDLERS(
+            "openFileHandlers",
+            "Open File Handlers",
+            "The current number of open file descriptors used by the Java virtual machine.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getOpenFileHandlers()),
+    PROCESSOR_LOAD_AVERAGE(
+            "processorLoadAverage",
+            "Processor Load Average",
+            "The processor load. Every measurement point represents the system load average for the last minute.",
+            MetricDescriptor.Formatter.FRACTION,
+            s -> Double.valueOf(s.getProcessorLoadAverage() * MetricDescriptor.FRACTION_MULTIPLIER).longValue(),
+            new ValueReducer<StatusSnapshot, Long>() {
+                @Override
+                public Long reduce(final List<StatusSnapshot> values) {
+                    long sumLoad = 0L;
+                    int invocations = 0;
+
+                    for (final StatusSnapshot snapshot : values) {
+                        final Long load = snapshot.getStatusMetric(PROCESSOR_LOAD_AVERAGE.getDescriptor());
+                        if (load != null) {
+                            sumLoad += load.longValue();
+                            invocations++;
+                        }
+                    }
+
+                    if (invocations == 0) {
+                        return 0L;
+                    }
+
+                    return sumLoad / invocations;
+                }
+            }),
+    TOTAL_THREADS(
+            "totalThreads",
+            "Number of total threads",
+            "The current number of live threads in the Java virtual machine (both daemon and non-daemon threads).",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getTotalThreads()),
+    EVENT_DRIVEN_THREADS(
+            "eventDrivenThreads",
+            "Number of event driven threads",
+            "The current number of active threads in the event driven thread pool.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getEventDrivenThreads()),
+    TIME_DRIVEN_THREADS(
+            "timeDrivenThreads",
+            "Number of time driven threads",
+            "The current number of active threads in the time driven thread pool.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getTimeDrivenThreads()),
+    FLOW_FILE_REPOSITORY_FREE_SPACE(
+            "flowFileRepositoryFreeSpace",
+            "Flow File Repository Free Space",
+            "The usable space available for use by the underlying storage mechanism.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFlowFileRepositoryFreeSpace()),
+    FLOW_FILE_REPOSITORY_USED_SPACE(
+            "flowFileRepositoryUsedSpace",
+            "Flow File Repository Used Space",
+            "The space in use on the underlying storage mechanism.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFlowFileRepositoryUsedSpace()),
+    CONTENT_REPOSITORY_FREE_SPACE(
+            "contentRepositoryFreeSpace",
+            "Sum content Repository Free Space",
+            "The usable space available for use by the underlying storage mechanisms.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getContentRepositories().stream().map(r -> r.getFreeSpace()).reduce(0L, (a, b) -> a + b)),

Review comment:
       ```suggestion
               s -> s.getContentRepositories().stream().mapToLong(StorageStatus::getFreeSpace).sum()),
   ```

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
##########
@@ -1057,6 +1058,39 @@ public Response deleteNode(
     // history
     // -------
 
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("status/history")
+    @ApiOperation(
+            value = "Gets configuration history for the node",
+            notes = NON_GUARANTEED_ENDPOINT,
+            response = ComponentHistoryEntity.class,
+            authorizations = {
+                    @Authorization(value = "Read - /controller")
+            }
+    )
+    @ApiResponses(
+            value = {
+                    @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+                    @ApiResponse(code = 401, message = "Client could not be authenticated."),
+                    @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+                    @ApiResponse(code = 404, message = "The specified resource could not be found."),
+                    @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+            }
+    )
+    public Response getNodeHistory() {

Review comment:
       ```suggestion
       public Response getNodeStatusHistory() {
   ```

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/NodeStatusDescriptor.java
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.nifi.controller.status.history;
+
+import org.apache.nifi.controller.status.NodeStatus;
+
+import java.util.List;
+
+public enum NodeStatusDescriptor {
+    FREE_HEAP(
+            "freeHeap",
+            "Free Heap",
+            "The amount of free memory in the heap that can be used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFreeHeap()),
+    USED_HEAP(
+            "usedHeap",
+            "Used Heap",
+            "The amount of used memory in the heap that is used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getUsedHeap()),
+    HEAP_UTILIZATION(
+            "heapUtilization",
+            "Heap Utilization",
+            "The percentage of available heap currently used by the Java virtual machine.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getHeapUtilization(),
+            new ValueReducer<StatusSnapshot, Long>() {
+                @Override
+                public Long reduce(final List<StatusSnapshot> values) {
+                    long sumUtilization = 0L;
+                    int invocations = 0;
+
+                    for (final StatusSnapshot snapshot : values) {
+                        final Long utilization = snapshot.getStatusMetric(HEAP_UTILIZATION.getDescriptor());
+                        if (utilization != null) {
+                            sumUtilization += utilization.longValue();
+                            invocations++;
+                        }
+                    }
+
+                    if (invocations == 0) {
+                        return 0L;
+                    }
+
+                    return sumUtilization / invocations;
+                }
+            }),
+    FREE_NON_HEAP(
+            "freeNonHeap",
+            "Free Non Heap",
+            "The currently available non-heap memory that can be used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFreeNonHeap()),
+    USED_NON_HEAP(
+            "usedNonHeap",
+            "Used Non Heap",
+            "The current memory usage of non-heap memory that is used by the Java virtual machine.",

Review comment:
       ```suggestion
               "The current usage of non-heap memory that is used by the Java virtual machine.",
   ```

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/NodeStatusDescriptor.java
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.nifi.controller.status.history;
+
+import org.apache.nifi.controller.status.NodeStatus;
+
+import java.util.List;
+
+public enum NodeStatusDescriptor {
+    FREE_HEAP(
+            "freeHeap",
+            "Free Heap",
+            "The amount of free memory in the heap that can be used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFreeHeap()),
+    USED_HEAP(
+            "usedHeap",
+            "Used Heap",
+            "The amount of used memory in the heap that is used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getUsedHeap()),
+    HEAP_UTILIZATION(
+            "heapUtilization",
+            "Heap Utilization",
+            "The percentage of available heap currently used by the Java virtual machine.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getHeapUtilization(),
+            new ValueReducer<StatusSnapshot, Long>() {
+                @Override
+                public Long reduce(final List<StatusSnapshot> values) {
+                    long sumUtilization = 0L;
+                    int invocations = 0;
+
+                    for (final StatusSnapshot snapshot : values) {
+                        final Long utilization = snapshot.getStatusMetric(HEAP_UTILIZATION.getDescriptor());
+                        if (utilization != null) {
+                            sumUtilization += utilization.longValue();
+                            invocations++;
+                        }
+                    }
+
+                    if (invocations == 0) {
+                        return 0L;
+                    }
+
+                    return sumUtilization / invocations;
+                }
+            }),
+    FREE_NON_HEAP(
+            "freeNonHeap",
+            "Free Non Heap",
+            "The currently available non-heap memory that can be used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFreeNonHeap()),
+    USED_NON_HEAP(
+            "usedNonHeap",
+            "Used Non Heap",
+            "The current memory usage of non-heap memory that is used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getUsedNonHeap()),
+    OPEN_FILE_HANDLERS(
+            "openFileHandlers",
+            "Open File Handlers",
+            "The current number of open file descriptors used by the Java virtual machine.",

Review comment:
       `file handle` is a more accurate term than `file handler`
   ```suggestion
               "The current number of open file handles used by the Java virtual machine.",
   ```

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/NodeStatusDescriptor.java
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.nifi.controller.status.history;
+
+import org.apache.nifi.controller.status.NodeStatus;
+
+import java.util.List;
+
+public enum NodeStatusDescriptor {
+    FREE_HEAP(
+            "freeHeap",
+            "Free Heap",
+            "The amount of free memory in the heap that can be used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFreeHeap()),
+    USED_HEAP(
+            "usedHeap",
+            "Used Heap",
+            "The amount of used memory in the heap that is used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getUsedHeap()),
+    HEAP_UTILIZATION(
+            "heapUtilization",
+            "Heap Utilization",
+            "The percentage of available heap currently used by the Java virtual machine.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getHeapUtilization(),
+            new ValueReducer<StatusSnapshot, Long>() {
+                @Override
+                public Long reduce(final List<StatusSnapshot> values) {
+                    long sumUtilization = 0L;
+                    int invocations = 0;
+
+                    for (final StatusSnapshot snapshot : values) {
+                        final Long utilization = snapshot.getStatusMetric(HEAP_UTILIZATION.getDescriptor());
+                        if (utilization != null) {
+                            sumUtilization += utilization.longValue();
+                            invocations++;
+                        }
+                    }
+
+                    if (invocations == 0) {
+                        return 0L;
+                    }
+
+                    return sumUtilization / invocations;
+                }
+            }),
+    FREE_NON_HEAP(
+            "freeNonHeap",
+            "Free Non Heap",
+            "The currently available non-heap memory that can be used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFreeNonHeap()),
+    USED_NON_HEAP(
+            "usedNonHeap",
+            "Used Non Heap",
+            "The current memory usage of non-heap memory that is used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getUsedNonHeap()),
+    OPEN_FILE_HANDLERS(
+            "openFileHandlers",
+            "Open File Handlers",
+            "The current number of open file descriptors used by the Java virtual machine.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getOpenFileHandlers()),
+    PROCESSOR_LOAD_AVERAGE(
+            "processorLoadAverage",
+            "Processor Load Average",
+            "The processor load. Every measurement point represents the system load average for the last minute.",
+            MetricDescriptor.Formatter.FRACTION,
+            s -> Double.valueOf(s.getProcessorLoadAverage() * MetricDescriptor.FRACTION_MULTIPLIER).longValue(),
+            new ValueReducer<StatusSnapshot, Long>() {
+                @Override
+                public Long reduce(final List<StatusSnapshot> values) {
+                    long sumLoad = 0L;
+                    int invocations = 0;
+
+                    for (final StatusSnapshot snapshot : values) {
+                        final Long load = snapshot.getStatusMetric(PROCESSOR_LOAD_AVERAGE.getDescriptor());
+                        if (load != null) {
+                            sumLoad += load.longValue();
+                            invocations++;
+                        }
+                    }
+
+                    if (invocations == 0) {
+                        return 0L;
+                    }
+
+                    return sumLoad / invocations;
+                }
+            }),

Review comment:
       ```suggestion
               new ValueReducer<StatusSnapshot, Long>() {
                   @Override
                   public Long reduce(final List<StatusSnapshot> values) {
                       return (long) values.stream()
                           .map(snapshot -> snapshot.getStatusMetric(PROCESSOR_LOAD_AVERAGE.getDescriptor()))
                           .filter(Objects::nonNull)
                           .mapToLong(value -> value)
                           .average()
                           .orElse(0L);
                   }
               }),
   ```

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/NodeStatusDescriptor.java
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.nifi.controller.status.history;
+
+import org.apache.nifi.controller.status.NodeStatus;
+
+import java.util.List;
+
+public enum NodeStatusDescriptor {
+    FREE_HEAP(
+            "freeHeap",
+            "Free Heap",
+            "The amount of free memory in the heap that can be used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFreeHeap()),
+    USED_HEAP(
+            "usedHeap",
+            "Used Heap",
+            "The amount of used memory in the heap that is used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getUsedHeap()),
+    HEAP_UTILIZATION(
+            "heapUtilization",
+            "Heap Utilization",
+            "The percentage of available heap currently used by the Java virtual machine.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getHeapUtilization(),
+            new ValueReducer<StatusSnapshot, Long>() {
+                @Override
+                public Long reduce(final List<StatusSnapshot> values) {
+                    long sumUtilization = 0L;
+                    int invocations = 0;
+
+                    for (final StatusSnapshot snapshot : values) {
+                        final Long utilization = snapshot.getStatusMetric(HEAP_UTILIZATION.getDescriptor());
+                        if (utilization != null) {
+                            sumUtilization += utilization.longValue();
+                            invocations++;
+                        }
+                    }
+
+                    if (invocations == 0) {
+                        return 0L;
+                    }
+
+                    return sumUtilization / invocations;
+                }
+            }),
+    FREE_NON_HEAP(
+            "freeNonHeap",
+            "Free Non Heap",
+            "The currently available non-heap memory that can be used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFreeNonHeap()),
+    USED_NON_HEAP(
+            "usedNonHeap",
+            "Used Non Heap",
+            "The current memory usage of non-heap memory that is used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getUsedNonHeap()),
+    OPEN_FILE_HANDLERS(
+            "openFileHandlers",
+            "Open File Handlers",
+            "The current number of open file descriptors used by the Java virtual machine.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getOpenFileHandlers()),
+    PROCESSOR_LOAD_AVERAGE(
+            "processorLoadAverage",
+            "Processor Load Average",
+            "The processor load. Every measurement point represents the system load average for the last minute.",
+            MetricDescriptor.Formatter.FRACTION,
+            s -> Double.valueOf(s.getProcessorLoadAverage() * MetricDescriptor.FRACTION_MULTIPLIER).longValue(),
+            new ValueReducer<StatusSnapshot, Long>() {
+                @Override
+                public Long reduce(final List<StatusSnapshot> values) {
+                    long sumLoad = 0L;
+                    int invocations = 0;
+
+                    for (final StatusSnapshot snapshot : values) {
+                        final Long load = snapshot.getStatusMetric(PROCESSOR_LOAD_AVERAGE.getDescriptor());
+                        if (load != null) {
+                            sumLoad += load.longValue();
+                            invocations++;
+                        }
+                    }
+
+                    if (invocations == 0) {
+                        return 0L;
+                    }
+
+                    return sumLoad / invocations;
+                }
+            }),
+    TOTAL_THREADS(
+            "totalThreads",
+            "Number of total threads",
+            "The current number of live threads in the Java virtual machine (both daemon and non-daemon threads).",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getTotalThreads()),
+    EVENT_DRIVEN_THREADS(
+            "eventDrivenThreads",
+            "Number of event driven threads",
+            "The current number of active threads in the event driven thread pool.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getEventDrivenThreads()),
+    TIME_DRIVEN_THREADS(
+            "timeDrivenThreads",
+            "Number of time driven threads",
+            "The current number of active threads in the time driven thread pool.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getTimeDrivenThreads()),
+    FLOW_FILE_REPOSITORY_FREE_SPACE(
+            "flowFileRepositoryFreeSpace",
+            "Flow File Repository Free Space",
+            "The usable space available for use by the underlying storage mechanism.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFlowFileRepositoryFreeSpace()),
+    FLOW_FILE_REPOSITORY_USED_SPACE(
+            "flowFileRepositoryUsedSpace",
+            "Flow File Repository Used Space",
+            "The space in use on the underlying storage mechanism.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFlowFileRepositoryUsedSpace()),
+    CONTENT_REPOSITORY_FREE_SPACE(
+            "contentRepositoryFreeSpace",
+            "Sum content Repository Free Space",
+            "The usable space available for use by the underlying storage mechanisms.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getContentRepositories().stream().map(r -> r.getFreeSpace()).reduce(0L, (a, b) -> a + b)),
+    CONTENT_REPOSITORY_USED_SPACE(
+            "contentRepositoryUsedSpace",
+            "Sum content Repository Used Space",
+            "The space in use on the underlying storage mechanisms.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getContentRepositories().stream().map(r -> r.getUsedSpace()).reduce(0L, (a, b) -> a + b)),

Review comment:
       ```suggestion
               s -> s.getContentRepositories().stream().mapToLong(StorageStatus::getUsedSpace).sum()),
   ```

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java
##########
@@ -164,6 +183,182 @@ public StatusHistory getRemoteProcessGroupStatusHistory(final String remoteGroup
         return getStatusHistory(remoteGroupId, true, DEFAULT_RPG_METRICS, start, end, preferredDataPoints);
     }
 
+    @Override
+    public StatusHistory getNodeStatusHistory() {

Review comment:
       Could this be covered with unit tests?

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/NodeStatusDescriptor.java
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.nifi.controller.status.history;
+
+import org.apache.nifi.controller.status.NodeStatus;
+
+import java.util.List;
+
+public enum NodeStatusDescriptor {
+    FREE_HEAP(
+            "freeHeap",
+            "Free Heap",
+            "The amount of free memory in the heap that can be used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFreeHeap()),
+    USED_HEAP(
+            "usedHeap",
+            "Used Heap",
+            "The amount of used memory in the heap that is used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getUsedHeap()),
+    HEAP_UTILIZATION(
+            "heapUtilization",
+            "Heap Utilization",
+            "The percentage of available heap currently used by the Java virtual machine.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getHeapUtilization(),
+            new ValueReducer<StatusSnapshot, Long>() {
+                @Override
+                public Long reduce(final List<StatusSnapshot> values) {
+                    long sumUtilization = 0L;
+                    int invocations = 0;
+
+                    for (final StatusSnapshot snapshot : values) {
+                        final Long utilization = snapshot.getStatusMetric(HEAP_UTILIZATION.getDescriptor());
+                        if (utilization != null) {
+                            sumUtilization += utilization.longValue();
+                            invocations++;
+                        }
+                    }
+
+                    if (invocations == 0) {
+                        return 0L;
+                    }
+
+                    return sumUtilization / invocations;
+                }
+            }),
+    FREE_NON_HEAP(
+            "freeNonHeap",
+            "Free Non Heap",
+            "The currently available non-heap memory that can be used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFreeNonHeap()),
+    USED_NON_HEAP(
+            "usedNonHeap",
+            "Used Non Heap",
+            "The current memory usage of non-heap memory that is used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getUsedNonHeap()),
+    OPEN_FILE_HANDLERS(
+            "openFileHandlers",
+            "Open File Handlers",
+            "The current number of open file descriptors used by the Java virtual machine.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getOpenFileHandlers()),
+    PROCESSOR_LOAD_AVERAGE(
+            "processorLoadAverage",
+            "Processor Load Average",
+            "The processor load. Every measurement point represents the system load average for the last minute.",
+            MetricDescriptor.Formatter.FRACTION,
+            s -> Double.valueOf(s.getProcessorLoadAverage() * MetricDescriptor.FRACTION_MULTIPLIER).longValue(),
+            new ValueReducer<StatusSnapshot, Long>() {
+                @Override
+                public Long reduce(final List<StatusSnapshot> values) {
+                    long sumLoad = 0L;
+                    int invocations = 0;
+
+                    for (final StatusSnapshot snapshot : values) {
+                        final Long load = snapshot.getStatusMetric(PROCESSOR_LOAD_AVERAGE.getDescriptor());
+                        if (load != null) {
+                            sumLoad += load.longValue();
+                            invocations++;
+                        }
+                    }
+
+                    if (invocations == 0) {
+                        return 0L;
+                    }
+
+                    return sumLoad / invocations;
+                }
+            }),
+    TOTAL_THREADS(
+            "totalThreads",
+            "Number of total threads",
+            "The current number of live threads in the Java virtual machine (both daemon and non-daemon threads).",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getTotalThreads()),
+    EVENT_DRIVEN_THREADS(
+            "eventDrivenThreads",
+            "Number of event driven threads",
+            "The current number of active threads in the event driven thread pool.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getEventDrivenThreads()),
+    TIME_DRIVEN_THREADS(
+            "timeDrivenThreads",
+            "Number of time driven threads",
+            "The current number of active threads in the time driven thread pool.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getTimeDrivenThreads()),
+    FLOW_FILE_REPOSITORY_FREE_SPACE(
+            "flowFileRepositoryFreeSpace",
+            "Flow File Repository Free Space",
+            "The usable space available for use by the underlying storage mechanism.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFlowFileRepositoryFreeSpace()),
+    FLOW_FILE_REPOSITORY_USED_SPACE(
+            "flowFileRepositoryUsedSpace",
+            "Flow File Repository Used Space",
+            "The space in use on the underlying storage mechanism.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFlowFileRepositoryUsedSpace()),
+    CONTENT_REPOSITORY_FREE_SPACE(
+            "contentRepositoryFreeSpace",
+            "Sum content Repository Free Space",
+            "The usable space available for use by the underlying storage mechanisms.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getContentRepositories().stream().map(r -> r.getFreeSpace()).reduce(0L, (a, b) -> a + b)),
+    CONTENT_REPOSITORY_USED_SPACE(
+            "contentRepositoryUsedSpace",
+            "Sum content Repository Used Space",
+            "The space in use on the underlying storage mechanisms.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getContentRepositories().stream().map(r -> r.getUsedSpace()).reduce(0L, (a, b) -> a + b)),
+    PROVENANCE_REPOSITORY_FREE_SPACE(
+            "provenanceRepositoryFreeSpace",
+            "Sum provenance Repository Free Space",
+            "The usable space available for use by the underlying storage mechanisms.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getProvenanceRepositories().stream().map(r -> r.getFreeSpace()).reduce(0L, (a, b) -> a + b)),

Review comment:
       ```suggestion
               s -> s.getProvenanceRepositories().stream().mapToLong(StorageStatus::getFreeSpace).sum()),
   ```

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/NodeStatusDescriptor.java
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.nifi.controller.status.history;
+
+import org.apache.nifi.controller.status.NodeStatus;
+
+import java.util.List;
+
+public enum NodeStatusDescriptor {
+    FREE_HEAP(
+            "freeHeap",
+            "Free Heap",
+            "The amount of free memory in the heap that can be used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFreeHeap()),
+    USED_HEAP(
+            "usedHeap",
+            "Used Heap",
+            "The amount of used memory in the heap that is used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getUsedHeap()),
+    HEAP_UTILIZATION(
+            "heapUtilization",
+            "Heap Utilization",
+            "The percentage of available heap currently used by the Java virtual machine.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getHeapUtilization(),
+            new ValueReducer<StatusSnapshot, Long>() {
+                @Override
+                public Long reduce(final List<StatusSnapshot> values) {
+                    long sumUtilization = 0L;
+                    int invocations = 0;
+
+                    for (final StatusSnapshot snapshot : values) {
+                        final Long utilization = snapshot.getStatusMetric(HEAP_UTILIZATION.getDescriptor());
+                        if (utilization != null) {
+                            sumUtilization += utilization.longValue();
+                            invocations++;
+                        }
+                    }
+
+                    if (invocations == 0) {
+                        return 0L;
+                    }
+
+                    return sumUtilization / invocations;
+                }
+            }),
+    FREE_NON_HEAP(
+            "freeNonHeap",
+            "Free Non Heap",
+            "The currently available non-heap memory that can be used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFreeNonHeap()),
+    USED_NON_HEAP(
+            "usedNonHeap",
+            "Used Non Heap",
+            "The current memory usage of non-heap memory that is used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getUsedNonHeap()),
+    OPEN_FILE_HANDLERS(
+            "openFileHandlers",
+            "Open File Handlers",
+            "The current number of open file descriptors used by the Java virtual machine.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getOpenFileHandlers()),
+    PROCESSOR_LOAD_AVERAGE(
+            "processorLoadAverage",
+            "Processor Load Average",
+            "The processor load. Every measurement point represents the system load average for the last minute.",
+            MetricDescriptor.Formatter.FRACTION,
+            s -> Double.valueOf(s.getProcessorLoadAverage() * MetricDescriptor.FRACTION_MULTIPLIER).longValue(),
+            new ValueReducer<StatusSnapshot, Long>() {
+                @Override
+                public Long reduce(final List<StatusSnapshot> values) {
+                    long sumLoad = 0L;
+                    int invocations = 0;
+
+                    for (final StatusSnapshot snapshot : values) {
+                        final Long load = snapshot.getStatusMetric(PROCESSOR_LOAD_AVERAGE.getDescriptor());
+                        if (load != null) {
+                            sumLoad += load.longValue();
+                            invocations++;
+                        }
+                    }
+
+                    if (invocations == 0) {
+                        return 0L;
+                    }
+
+                    return sumLoad / invocations;
+                }
+            }),
+    TOTAL_THREADS(
+            "totalThreads",
+            "Number of total threads",
+            "The current number of live threads in the Java virtual machine (both daemon and non-daemon threads).",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getTotalThreads()),
+    EVENT_DRIVEN_THREADS(
+            "eventDrivenThreads",
+            "Number of event driven threads",
+            "The current number of active threads in the event driven thread pool.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getEventDrivenThreads()),
+    TIME_DRIVEN_THREADS(
+            "timeDrivenThreads",
+            "Number of time driven threads",
+            "The current number of active threads in the time driven thread pool.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getTimeDrivenThreads()),
+    FLOW_FILE_REPOSITORY_FREE_SPACE(
+            "flowFileRepositoryFreeSpace",
+            "Flow File Repository Free Space",
+            "The usable space available for use by the underlying storage mechanism.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFlowFileRepositoryFreeSpace()),
+    FLOW_FILE_REPOSITORY_USED_SPACE(
+            "flowFileRepositoryUsedSpace",
+            "Flow File Repository Used Space",
+            "The space in use on the underlying storage mechanism.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFlowFileRepositoryUsedSpace()),
+    CONTENT_REPOSITORY_FREE_SPACE(
+            "contentRepositoryFreeSpace",
+            "Sum content Repository Free Space",
+            "The usable space available for use by the underlying storage mechanisms.",

Review comment:
       Has the same description as `FLOW_FILE_REPOSITORY_FREE_SPACE`.

##########
File path: nifi-api/src/main/java/org/apache/nifi/controller/status/NodeStatus.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.nifi.controller.status;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * The status of a NiFi node.
+ */
+public class NodeStatus implements Cloneable {
+    private long createdAtInMs;
+
+    private long freeHeap;
+    private long usedHeap;
+    private long heapUtilization;
+    private long freeNonHeap;
+    private long usedNonHeap;
+
+    private long openFileHandlers;
+    private double processorLoadAverage;
+    private long uptime;
+
+    private long totalThreads;
+    private long eventDrivenThreads;
+    private long timeDrivenThreads;

Review comment:
       ```suggestion
       private long timerDrivenThreads;
   ```

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
##########
@@ -1057,6 +1058,39 @@ public Response deleteNode(
     // history
     // -------
 
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("status/history")
+    @ApiOperation(
+            value = "Gets configuration history for the node",

Review comment:
       ```suggestion
               value = "Gets status history for the node",
   ```

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/NodeStatusDescriptor.java
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.nifi.controller.status.history;
+
+import org.apache.nifi.controller.status.NodeStatus;
+
+import java.util.List;
+
+public enum NodeStatusDescriptor {
+    FREE_HEAP(
+            "freeHeap",
+            "Free Heap",
+            "The amount of free memory in the heap that can be used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFreeHeap()),
+    USED_HEAP(
+            "usedHeap",
+            "Used Heap",
+            "The amount of used memory in the heap that is used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getUsedHeap()),
+    HEAP_UTILIZATION(
+            "heapUtilization",
+            "Heap Utilization",
+            "The percentage of available heap currently used by the Java virtual machine.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getHeapUtilization(),
+            new ValueReducer<StatusSnapshot, Long>() {
+                @Override
+                public Long reduce(final List<StatusSnapshot> values) {
+                    long sumUtilization = 0L;
+                    int invocations = 0;
+
+                    for (final StatusSnapshot snapshot : values) {
+                        final Long utilization = snapshot.getStatusMetric(HEAP_UTILIZATION.getDescriptor());
+                        if (utilization != null) {
+                            sumUtilization += utilization.longValue();
+                            invocations++;
+                        }
+                    }
+
+                    if (invocations == 0) {
+                        return 0L;
+                    }
+
+                    return sumUtilization / invocations;
+                }
+            }),
+    FREE_NON_HEAP(
+            "freeNonHeap",
+            "Free Non Heap",
+            "The currently available non-heap memory that can be used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFreeNonHeap()),
+    USED_NON_HEAP(
+            "usedNonHeap",
+            "Used Non Heap",
+            "The current memory usage of non-heap memory that is used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getUsedNonHeap()),
+    OPEN_FILE_HANDLERS(
+            "openFileHandlers",
+            "Open File Handlers",
+            "The current number of open file descriptors used by the Java virtual machine.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getOpenFileHandlers()),
+    PROCESSOR_LOAD_AVERAGE(
+            "processorLoadAverage",
+            "Processor Load Average",
+            "The processor load. Every measurement point represents the system load average for the last minute.",
+            MetricDescriptor.Formatter.FRACTION,
+            s -> Double.valueOf(s.getProcessorLoadAverage() * MetricDescriptor.FRACTION_MULTIPLIER).longValue(),
+            new ValueReducer<StatusSnapshot, Long>() {
+                @Override
+                public Long reduce(final List<StatusSnapshot> values) {
+                    long sumLoad = 0L;
+                    int invocations = 0;
+
+                    for (final StatusSnapshot snapshot : values) {
+                        final Long load = snapshot.getStatusMetric(PROCESSOR_LOAD_AVERAGE.getDescriptor());
+                        if (load != null) {
+                            sumLoad += load.longValue();
+                            invocations++;
+                        }
+                    }
+
+                    if (invocations == 0) {
+                        return 0L;
+                    }
+
+                    return sumLoad / invocations;
+                }
+            }),
+    TOTAL_THREADS(
+            "totalThreads",
+            "Number of total threads",
+            "The current number of live threads in the Java virtual machine (both daemon and non-daemon threads).",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getTotalThreads()),
+    EVENT_DRIVEN_THREADS(
+            "eventDrivenThreads",
+            "Number of event driven threads",
+            "The current number of active threads in the event driven thread pool.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getEventDrivenThreads()),
+    TIME_DRIVEN_THREADS(
+            "timeDrivenThreads",
+            "Number of time driven threads",
+            "The current number of active threads in the time driven thread pool.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getTimeDrivenThreads()),
+    FLOW_FILE_REPOSITORY_FREE_SPACE(
+            "flowFileRepositoryFreeSpace",
+            "Flow File Repository Free Space",
+            "The usable space available for use by the underlying storage mechanism.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFlowFileRepositoryFreeSpace()),
+    FLOW_FILE_REPOSITORY_USED_SPACE(
+            "flowFileRepositoryUsedSpace",
+            "Flow File Repository Used Space",
+            "The space in use on the underlying storage mechanism.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFlowFileRepositoryUsedSpace()),
+    CONTENT_REPOSITORY_FREE_SPACE(
+            "contentRepositoryFreeSpace",
+            "Sum content Repository Free Space",
+            "The usable space available for use by the underlying storage mechanisms.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getContentRepositories().stream().map(r -> r.getFreeSpace()).reduce(0L, (a, b) -> a + b)),
+    CONTENT_REPOSITORY_USED_SPACE(
+            "contentRepositoryUsedSpace",
+            "Sum content Repository Used Space",
+            "The space in use on the underlying storage mechanisms.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getContentRepositories().stream().map(r -> r.getUsedSpace()).reduce(0L, (a, b) -> a + b)),
+    PROVENANCE_REPOSITORY_FREE_SPACE(
+            "provenanceRepositoryFreeSpace",
+            "Sum provenance Repository Free Space",
+            "The usable space available for use by the underlying storage mechanisms.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getProvenanceRepositories().stream().map(r -> r.getFreeSpace()).reduce(0L, (a, b) -> a + b)),
+    PROVENANCE_REPOSITORY_USED_SPACE(
+            "provenanceRepositoryUsedSpace",
+            "Sum provenance Repository Used Space",
+            "The space in use on the underlying storage mechanisms.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getProvenanceRepositories().stream().map(r -> r.getUsedSpace()).reduce(0L, (a, b) -> a + b));

Review comment:
       ```suggestion
               s -> s.getProvenanceRepositories().stream().mapToLong(StorageStatus::getUsedSpace).sum()),
   ```

##########
File path: nifi-api/src/main/java/org/apache/nifi/controller/status/NodeStatus.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.nifi.controller.status;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * The status of a NiFi node.
+ */
+public class NodeStatus implements Cloneable {

Review comment:
       Is there a reason for the new DTO classes? Couldn't we use the original `SystemDiagnostics` and `StorageStatus` instead?

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/StatusHistoryUtil.java
##########
@@ -76,14 +76,14 @@ public static StatusDescriptorDTO createStatusDescriptorDto(final MetricDescript
 
     public static List<StatusDescriptorDTO> createFieldDescriptorDtos(final Collection<MetricDescriptor<?>> metricDescriptors) {
         final List<StatusDescriptorDTO> dtos = new ArrayList<>();
+        final Map<Integer, MetricDescriptor<?>> orderedDescriptors = new HashMap<>();
 
-        final Set<MetricDescriptor<?>> allDescriptors = new LinkedHashSet<>();
         for (final MetricDescriptor<?> metricDescriptor : metricDescriptors) {

Review comment:
       Is there a reason why this logic is this complex?
   `orderedDescriptors.get(i)` indicates that `metricDescriptors` cannot have a gap in their `getMetricIdentifier()` values.
   Not sure if it's a good idea to rely on that. But if absolutely can, we could simply go with:
   ```java
       public static List<StatusDescriptorDTO> createFieldDescriptorDtos(final Collection<MetricDescriptor<?>> metricDescriptors) {
           final List<StatusDescriptorDTO> dtos = new ArrayList<>();
   
           for (final MetricDescriptor<?> metricDescriptor : metricDescriptors) {
               dtos.add(metricDescriptor.getMetricIdentifier(), createStatusDescriptorDto(metricDescriptor));
           }
   
           return dtos;
       }
   ```
   Otherwise we might simply want to call a `sort` on `dtos`.




----------------------------------------------------------------
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] [nifi] mcgilman commented on a change in pull request #4420: NIFI-7429 Adding status history for system level metrics

Posted by GitBox <gi...@apache.org>.
mcgilman commented on a change in pull request #4420:
URL: https://github.com/apache/nifi/pull/4420#discussion_r475689496



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/NodeStatusDescriptor.java
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.nifi.controller.status.history;
+
+import org.apache.nifi.controller.status.NodeStatus;
+
+import java.util.List;
+
+public enum NodeStatusDescriptor {
+    FREE_HEAP(
+            "freeHeap",
+            "Free Heap",
+            "The amount of free memory in the heap that can be used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFreeHeap()),
+    USED_HEAP(
+            "usedHeap",
+            "Used Heap",
+            "The amount of used memory in the heap that is used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getUsedHeap()),
+    HEAP_UTILIZATION(
+            "heapUtilization",
+            "Heap Utilization",
+            "The percentage of available heap currently used by the Java virtual machine.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getHeapUtilization(),
+            new ValueReducer<StatusSnapshot, Long>() {
+                @Override
+                public Long reduce(final List<StatusSnapshot> values) {
+                    long sumUtilization = 0L;
+                    int invocations = 0;
+
+                    for (final StatusSnapshot snapshot : values) {
+                        final Long utilization = snapshot.getStatusMetric(HEAP_UTILIZATION.getDescriptor());
+                        if (utilization != null) {
+                            sumUtilization += utilization.longValue();
+                            invocations++;
+                        }
+                    }
+
+                    if (invocations == 0) {
+                        return 0L;
+                    }
+
+                    return sumUtilization / invocations;
+                }
+            }),
+    FREE_NON_HEAP(
+            "freeNonHeap",
+            "Free Non Heap",
+            "The currently available non-heap memory that can be used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getFreeNonHeap()),
+    USED_NON_HEAP(
+            "usedNonHeap",
+            "Used Non Heap",
+            "The current memory usage of non-heap memory that is used by the Java virtual machine.",
+            MetricDescriptor.Formatter.DATA_SIZE,
+            s -> s.getUsedNonHeap()),
+    OPEN_FILE_HANDLERS(
+            "openFileHandlers",
+            "Open File Handlers",
+            "The current number of open file descriptors used by the Java virtual machine.",
+            MetricDescriptor.Formatter.COUNT,
+            s -> s.getOpenFileHandlers()),
+    PROCESSOR_LOAD_AVERAGE(
+            "processorLoadAverage",
+            "Processor Load Average",
+            "The processor load. Every measurement point represents te system load average for the last minute.",

Review comment:
       Typo `te`.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-status-history.js
##########
@@ -78,6 +79,9 @@
         },
         'DATA_SIZE': function (d) {
             return nfCommon.formatDataSize(d);
+        },
+        'FRACTION': function (d) {
+            return nfCommon.formatFloat(d / 1000000);

Review comment:
       I saw there is a metric multiplier that is applied server-side and the comment indicates that this operation is needed before presenting the value to a user. However, I'm not quite following upon first review. Can you elaborate on this a little more and explain why it's needed? Just a little worried that we have a magic number here with no reference to the fraction multiplier being applied server-side.




----------------------------------------------------------------
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] [nifi] simonbence commented on a change in pull request #4420: NIFI-7429 Adding status history for system level metrics

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4420:
URL: https://github.com/apache/nifi/pull/4420#discussion_r483547447



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/StatusHistoryUtil.java
##########
@@ -76,14 +76,14 @@ public static StatusDescriptorDTO createStatusDescriptorDto(final MetricDescript
 
     public static List<StatusDescriptorDTO> createFieldDescriptorDtos(final Collection<MetricDescriptor<?>> metricDescriptors) {
         final List<StatusDescriptorDTO> dtos = new ArrayList<>();
+        final Map<Integer, MetricDescriptor<?>> orderedDescriptors = new HashMap<>();
 
-        final Set<MetricDescriptor<?>> allDescriptors = new LinkedHashSet<>();
         for (final MetricDescriptor<?> metricDescriptor : metricDescriptors) {

Review comment:
       The order of the items counts, as it determines the order of the metrics in the answer JSON, thus the order of the items in the UI.
   
   The two-step mechanism was introduced to provide the ascending order specified by the identifier of the metric. As the result StatusDescriptorDTO does not contain this information, only its position carries the expected ordering, I had to ask back to the original MetricDescriptor. The incoming collection is not guaranteed to be ordered.
   
   An other possible way would be to sort, as you mention, and not in the end result but the input argument or more precisely on its copy. Copying and then sorting that collection does not look much nicer in my perspective so if you do not insist, I would keep it this way.




----------------------------------------------------------------
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] [nifi] bbende commented on a change in pull request #4420: NIFI-7429 Adding status history for system level metrics

Posted by GitBox <gi...@apache.org>.
bbende commented on a change in pull request #4420:
URL: https://github.com/apache/nifi/pull/4420#discussion_r481306447



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
##########
@@ -2691,6 +2691,39 @@ public Response getAction(
         return generateOkResponse(entity).build();
     }
 
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("node/status/history")
+    @ApiOperation(
+            value = "Gets configuration history for the node",
+            notes = NON_GUARANTEED_ENDPOINT,
+            response = ComponentHistoryEntity.class,
+            authorizations = {
+                    @Authorization(value = "Read - /flow")
+            }
+    )
+    @ApiResponses(
+            value = {
+                    @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+                    @ApiResponse(code = 401, message = "Client could not be authenticated."),
+                    @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+                    @ApiResponse(code = 404, message = "The specified resource could not be found."),
+                    @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+            }
+    )
+    public Response getNodeHistory() {
+        authorizeFlow();

Review comment:
       I think we probably want to move this end-point to the` ControllerResource` and authorize against `Read - /controller`. The reason being that most of the information in the node status history is really controller level information and is similar to what is returned from `ControllerResource` for` /controller/cluster`.




----------------------------------------------------------------
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] [nifi] pvillard31 commented on pull request #4420: NIFI-7429 Adding status history for system level metrics

Posted by GitBox <gi...@apache.org>.
pvillard31 commented on pull request #4420:
URL: https://github.com/apache/nifi/pull/4420#issuecomment-666469978


   I've played with it and it looks good to me. That is an awesome addition to NiFi, thanks @simonbence !
   @markap14 - I think it'd be best if you can have a look to the code since you're familiar with this part
   @mcgilman - as far as I can tell the UI part looks good to me, do you want to double check?


----------------------------------------------------------------
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] [nifi] tpalfy commented on a change in pull request #4420: NIFI-7429 Adding status history for system level metrics

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4420:
URL: https://github.com/apache/nifi/pull/4420#discussion_r483599157



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/StatusHistoryUtil.java
##########
@@ -76,14 +76,14 @@ public static StatusDescriptorDTO createStatusDescriptorDto(final MetricDescript
 
     public static List<StatusDescriptorDTO> createFieldDescriptorDtos(final Collection<MetricDescriptor<?>> metricDescriptors) {
         final List<StatusDescriptorDTO> dtos = new ArrayList<>();
+        final Map<Integer, MetricDescriptor<?>> orderedDescriptors = new HashMap<>();
 
-        final Set<MetricDescriptor<?>> allDescriptors = new LinkedHashSet<>();
         for (final MetricDescriptor<?> metricDescriptor : metricDescriptors) {

Review comment:
       I think I didn't explain myself clearly. I understand the reasoning and constraints but those don't deny my statements. 
   My code snippet was not 100% correct but the concept was. Here's a correct version:
   ```java
           final StatusDescriptorDTO[] dtos = new StatusDescriptorDTO[metricDescriptors.size()];
   
           for (final MetricDescriptor<?> metricDescriptor : metricDescriptors) {
               dtos[metricDescriptor.getMetricIdentifier()] = createStatusDescriptorDto(metricDescriptor);
           }
   
           return Arrays.asList(dtos);
   ```
   Here's a unit test (which - or something similar - would be useful to add):
   ```java
   public class StatusHistoryUtilTest {
       @Test
       public void testCreateFieldDescriptorDtos() throws Exception {
           // GIVEN
           Collection<MetricDescriptor<?>> metricDescriptors = Arrays.asList(
               new StandardMetricDescriptor<>(
                   () -> 1,
                   "field2",
                   "Field2",
                   "Field 2",
                   MetricDescriptor.Formatter.COUNT,
                   __ -> 2L
               ),
               new StandardMetricDescriptor<>(
                   () -> 0,
                   "field1",
                   "Field1",
                   "Field 1",
                   MetricDescriptor.Formatter.COUNT,
                   __ -> 1L
               )
           );
   
           List<StatusDescriptorDTO> expected = Arrays.asList(
               new StatusDescriptorDTO("field1", "Field1", "Field 1", MetricDescriptor.Formatter.COUNT.name()),
               new StatusDescriptorDTO("field2", "Field2", "Field 2", MetricDescriptor.Formatter.COUNT.name())
           );
   
           // WHEN
           List<StatusDescriptorDTO> actual = StatusHistoryUtil.createFieldDescriptorDtos(metricDescriptors);
   
           // THEN
           assertEquals(expected, actual);
       }
   }
   ```
   There are two metric indexes: 1, 0 (coming from `() -> 1` and `() -> 0,` respectively).
   
   - If you change `() -> 1` to `() -> 2`, the method throws an expection (yours a NullPointerException, mine an ArrayIndexOutOfBoundsException) - Because index 1 is missing. That's what I meant by "metricDescriptors cannot have a gap in their getMetricIdentifier() values"
   - All we do is make sure the order of the output is based on the metric index. That can done with the simple for iteration I presented, or simply sorting it, mapping them into a dto and collecting them into a list (which _would_ allow a gap in the getMetricIdentifier() values as well) like this:
   ```java
       public static List<StatusDescriptorDTO> createFieldDescriptorDtos(final Collection<MetricDescriptor<?>> metricDescriptors) {
           return metricDescriptors.stream()
               .sorted(Comparator.comparingInt(MetricDescriptor::getMetricIdentifier))
               .map(StatusHistoryUtil::createStatusDescriptorDto)
               .collect(Collectors.toList());
       }
   ```




----------------------------------------------------------------
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] [nifi] tpalfy commented on a change in pull request #4420: NIFI-7429 Adding status history for system level metrics

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4420:
URL: https://github.com/apache/nifi/pull/4420#discussion_r483599157



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/StatusHistoryUtil.java
##########
@@ -76,14 +76,14 @@ public static StatusDescriptorDTO createStatusDescriptorDto(final MetricDescript
 
     public static List<StatusDescriptorDTO> createFieldDescriptorDtos(final Collection<MetricDescriptor<?>> metricDescriptors) {
         final List<StatusDescriptorDTO> dtos = new ArrayList<>();
+        final Map<Integer, MetricDescriptor<?>> orderedDescriptors = new HashMap<>();
 
-        final Set<MetricDescriptor<?>> allDescriptors = new LinkedHashSet<>();
         for (final MetricDescriptor<?> metricDescriptor : metricDescriptors) {

Review comment:
       I think I didn't explain myself clearly. I understand the reasoning and constraints but those don't deny my statements. 
   My code snippet was not 100% correct but the concept was. Here's a correct version:
   ```java
           final StatusDescriptorDTO[] dtos = new StatusDescriptorDTO[metricDescriptors.size()];
   
           for (final MetricDescriptor<?> metricDescriptor : metricDescriptors) {
               dtos[metricDescriptor.getMetricIdentifier()] = createStatusDescriptorDto(metricDescriptor);
           }
   
           return Arrays.asList(dtos);
   ```
   Here's a unit test (which - or something similar - would be useful to add):
   ```java
   public class StatusHistoryUtilTest {
       @Test
       public void name() throws Exception {
           // GIVEN
           Collection<MetricDescriptor<?>> metricDescriptors = Arrays.asList(
               new StandardMetricDescriptor<>(
                   () -> 1,
                   "field2",
                   "Field2",
                   "Field 2",
                   MetricDescriptor.Formatter.COUNT,
                   __ -> 2L
               ),
               new StandardMetricDescriptor<>(
                   () -> 0,
                   "field1",
                   "Field1",
                   "Field 1",
                   MetricDescriptor.Formatter.COUNT,
                   __ -> 1L
               )
           );
   
           List<StatusDescriptorDTO> expected = Arrays.asList(
               new StatusDescriptorDTO("field1", "Field1", "Field 1", MetricDescriptor.Formatter.COUNT.name()),
               new StatusDescriptorDTO("field2", "Field2", "Field 2", MetricDescriptor.Formatter.COUNT.name())
           );
   
           // WHEN
           List<StatusDescriptorDTO> actual = StatusHistoryUtil.createFieldDescriptorDtos(metricDescriptors);
   
           // THEN
           assertEquals(expected, actual);
       }
   }
   ```
   There are two metric indexes: 1, 0 (coming from `() -> 1` and `() -> 0,` respectively).
   
   - If you change `() -> 1` to `() -> 2`, the method throws an expection (yours a NullPointerException, mine an ArrayIndexOutOfBoundsException) - Because index 1 is missing. That's what I meant by "metricDescriptors cannot have a gap in their getMetricIdentifier() values"
   - All we do is make sure the order of the output is based on the metric index. That can done with the simple for iteration I presented, or simply sorting it, mapping them into a dto and collecting them into a list (which _would_ allow a gap in the getMetricIdentifier() values as well) like this:
   ```java
       public static List<StatusDescriptorDTO> createFieldDescriptorDtos(final Collection<MetricDescriptor<?>> metricDescriptors) {
           return metricDescriptors.stream()
               .sorted((a, b) -> a.getMetricIdentifier() < b.getMetricIdentifier() ? -1 : a.getMetricIdentifier() > b.getMetricIdentifier() ? 1 :0)
               .map(StatusHistoryUtil::createStatusDescriptorDto)
               .collect(Collectors.toList());
       }
   ```




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