You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ai...@apache.org on 2019/08/19 13:53:36 UTC

[nifi] branch analytics-framework created (now 714b3f6)

This is an automated email from the ASF dual-hosted git repository.

aichrist pushed a change to branch analytics-framework
in repository https://gitbox.apache.org/repos/asf/nifi.git.


      at 714b3f6  NIFI-6510 Added property to nifi.properties - Prediction Interval for connection status analytics (#11)

This branch includes the following new commits:

     new c4cc6c9  NIFI-6510 Implement initial analytic engine
     new cfc3fdc  NIFI-6510 Implemented basic linear regression model for queue counts
     new 42b50e8  NIFI-6510 Initial analytics REST endpoint and supporting objects
     new be0993c  NIFI-6510 Connect the dots for StatusAnalytics -> API
     new bd011f3  NIFI-6510 Added poc engine with prediction model caching
     new 7252413  NIFI-6510 Updated objects and interfaces to reflect 4 prediction metrics
     new 2bcf3db  NIFI-6510 adjustments for interface updates, added call to StandardEventAccess, updated interface to use connection id
     new af7becb  NIFI-6510 Remove redundant connection prediction interfaces as we can just use ConnectionStatusAnalytics directly
     new d1a9a92  NIFI-6510 Revert "DFA-9 Remove redundant connection prediction interfaces as we can just use ConnectionStatusAnalytics directly"
     new 7adaa24  NIFI-6510 Split StatusAnalytics interface into Engine and per-Connection versions
     new 6a45d01  NIFI-6510 Added prediction fields for use by UI, still need to be populated
     new f3824a0  NIFI-6510 Analytics Framework Introduction (#10)
     new 8a48c77  NIFI-6510 Add prediction percent values and predicted interval seconds
     new 6c5eda1  NIFI-6510 Changes to inject flowManager instead of flow controller, also changes to properly reflect when predictions can be made vs not.
     new 0bdedd5  NIFI-6510 Added tests for engine
     new ecce279  NIFI-6150 Added tests for connection status analytics class, corrected variable names
     new 34e6191  NIFI-6150 Make checkstyle happy
     new 9cc3404  NIFI-6150 Fixed NaN check and refactored time prediction. Switched to use non caching engine for testing
     new 0c9e8ed  NIFI-6510 Fixed checkstyle issue in TestConnectionStatusAnalytics
     new 586185e  NIFI-6510 Adjusted interval and incorporated R-squared check
     new 714b3f6  NIFI-6510 Added property to nifi.properties - Prediction Interval for connection status analytics (#11)

The 21 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[nifi] 11/21: NIFI-6510 Added prediction fields for use by UI, still need to be populated

Posted by ai...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

aichrist pushed a commit to branch analytics-framework
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 6a45d016df7dd4226875133ae428d66fadee87b5
Author: Matthew Burgess <ma...@apache.org>
AuthorDate: Wed Jul 24 17:09:27 2019 -0400

    NIFI-6510 Added prediction fields for use by UI, still need to be populated
---
 .../nifi/controller/status/ConnectionStatus.java   |  9 ++++++
 .../dto/status/ConnectionStatusSnapshotDTO.java    | 33 ++++++++++++++++++++++
 .../apache/nifi/cluster/manager/StatusMerger.java  | 28 ++++++++++++++++++
 .../org/apache/nifi/web/api/dto/DtoFactory.java    |  4 +++
 4 files changed, 74 insertions(+)

diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/ConnectionStatus.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/ConnectionStatus.java
index 0e5d306..ee7dd45 100644
--- a/nifi-api/src/main/java/org/apache/nifi/controller/status/ConnectionStatus.java
+++ b/nifi-api/src/main/java/org/apache/nifi/controller/status/ConnectionStatus.java
@@ -40,6 +40,7 @@ public class ConnectionStatus implements Cloneable {
     private long outputBytes;
     private int maxQueuedCount;
     private long maxQueuedBytes;
+    private long predictionIntervalMillis;
     private int nextPredictedQueuedCount;
     private long nextPredictedQueuedBytes;
     private long predictedTimeToCountBackpressureMillis;
@@ -190,6 +191,14 @@ public class ConnectionStatus implements Cloneable {
         this.backPressureBytesThreshold = backPressureBytesThreshold;
     }
 
+    public long getPredictionIntervalMillis() {
+        return predictionIntervalMillis;
+    }
+
+    public void setPredictionIntervalMillis(long predictionIntervalMillis) {
+        this.predictionIntervalMillis = predictionIntervalMillis;
+    }
+
     public int getNextPredictedQueuedCount() {
         return nextPredictedQueuedCount;
     }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusSnapshotDTO.java
index 3237385..f48e145 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusSnapshotDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusSnapshotDTO.java
@@ -50,8 +50,11 @@ public class ConnectionStatusSnapshotDTO implements Cloneable {
     private Integer percentUseBytes;
     private Long predictedMillisUntilCountBackpressure = 0L;
     private Long predictedMillisUntilBytesBackpressure = 0L;
+    private Integer predictionIntervalSeconds;
     private Integer predictedCountAtNextInterval = 0;
     private Long predictedBytesAtNextInterval = 0L;
+    private Integer predictedPercentCount;
+    private Integer predictedPercentBytes;
 
     /* getters / setters */
     /**
@@ -302,6 +305,15 @@ public class ConnectionStatusSnapshotDTO implements Cloneable {
         this.predictedCountAtNextInterval = predictedCountAtNextInterval;
     }
 
+    @ApiModelProperty("The configured interval (in seconds) for predicting connection queue count and size (and percent usage).")
+    public Integer getPredictionIntervalSeconds() {
+        return predictionIntervalSeconds;
+    }
+
+    public void setPredictionIntervalSeconds(Integer predictionIntervalSeconds) {
+        this.predictionIntervalSeconds = predictionIntervalSeconds;
+    }
+
     @ApiModelProperty("The predicted total number of bytes in the queue at the next configured interval.")
     public Long getPredictedBytesAtNextInterval() {
         return predictedBytesAtNextInterval;
@@ -311,6 +323,24 @@ public class ConnectionStatusSnapshotDTO implements Cloneable {
         this.predictedBytesAtNextInterval = predictedBytesAtNextInterval;
     }
 
+    @ApiModelProperty("Predicted connection percent use regarding queued flow files count and backpressure threshold if configured.")
+    public Integer getPredictedPercentCount() {
+        return predictedPercentCount;
+    }
+
+    public void setPredictedPercentCount(Integer predictedPercentCount) {
+        this.predictedPercentCount = predictedPercentCount;
+    }
+
+    @ApiModelProperty("Predicted connection percent use regarding queued flow files size and backpressure threshold if configured.")
+    public Integer getPredictedPercentBytes() {
+        return predictedPercentBytes;
+    }
+
+    public void setPredictedPercentBytes(Integer predictedPercentBytes) {
+        this.predictedPercentBytes = predictedPercentBytes;
+    }
+
     @Override
     public ConnectionStatusSnapshotDTO clone() {
         final ConnectionStatusSnapshotDTO other = new ConnectionStatusSnapshotDTO();
@@ -337,8 +367,11 @@ public class ConnectionStatusSnapshotDTO implements Cloneable {
         other.setPercentUseCount(getPercentUseCount());
         other.setPredictedMillisUntilCountBackpressure(getPredictedMillisUntilCountBackpressure());
         other.setPredictedMillisUntilBytesBackpressure(getPredictedMillisUntilBytesBackpressure());
+        other.setPredictionIntervalSeconds(getPredictionIntervalSeconds());
         other.setPredictedCountAtNextInterval(getPredictedCountAtNextInterval());
         other.setPredictedBytesAtNextInterval(getPredictedBytesAtNextInterval());
+        other.setPredictedPercentBytes(getPredictedPercentBytes());
+        other.setPredictedPercentCount(getPredictedPercentCount());
 
         return other;
     }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/StatusMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/StatusMerger.java
index 946295a..4d21ce4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/StatusMerger.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/StatusMerger.java
@@ -500,6 +500,34 @@ public class StatusMerger {
             target.setPercentUseCount(Math.max(target.getPercentUseCount(), toMerge.getPercentUseCount()));
         }
 
+        // Merge predicted values (minimum time to backpressure, maximum percent at next interval
+
+        if (target.getPredictionIntervalSeconds() == null) {
+            target.setPredictionIntervalSeconds(toMerge.getPredictionIntervalSeconds());
+        }
+
+        if (target.getPredictedMillisUntilBytesBackpressure() == null) {
+            target.setPredictedMillisUntilBytesBackpressure(toMerge.getPredictedMillisUntilBytesBackpressure());
+        } else if (toMerge.getPredictedMillisUntilBytesBackpressure() != null) {
+            target.setPredictedMillisUntilBytesBackpressure(Math.min(target.getPredictedMillisUntilBytesBackpressure(), toMerge.getPredictedMillisUntilBytesBackpressure()));
+        }
+        if (target.getPredictedMillisUntilCountBackpressure() == null) {
+            target.setPredictedMillisUntilCountBackpressure(toMerge.getPredictedMillisUntilCountBackpressure());
+        } else if (toMerge.getPredictedMillisUntilCountBackpressure() != null) {
+            target.setPredictedMillisUntilCountBackpressure(Math.min(target.getPredictedMillisUntilCountBackpressure(), toMerge.getPredictedMillisUntilCountBackpressure()));
+        }
+
+        if (target.getPredictedPercentBytes() == null) {
+            target.setPredictedPercentBytes(toMerge.getPredictedPercentBytes());
+        } else if (toMerge.getPercentUseBytes() != null) {
+            target.setPredictedPercentBytes(Math.max(target.getPredictedPercentBytes(), toMerge.getPredictedPercentBytes()));
+        }
+        if (target.getPredictedPercentCount() == null) {
+            target.setPredictedPercentCount(toMerge.getPredictedPercentCount());
+        } else if (toMerge.getPredictedPercentCount() != null) {
+            target.setPredictedPercentCount(Math.max(target.getPredictedPercentCount(), toMerge.getPredictedPercentCount()));
+        }
+
         updatePrettyPrintedFields(target);
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
index 2ff12e6..88cc299 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
@@ -1179,11 +1179,15 @@ public final class DtoFactory {
 
         if (connectionStatus.getBackPressureObjectThreshold() > 0) {
             snapshot.setPercentUseCount(Math.min(100, StatusMerger.getUtilization(connectionStatus.getQueuedCount(), connectionStatus.getBackPressureObjectThreshold())));
+
+            snapshot.setPredictionIntervalSeconds(((Long) (connectionStatus.getPredictionIntervalMillis() / 1000L)).intValue());
             snapshot.setPredictedMillisUntilCountBackpressure(connectionStatus.getPredictedTimeToCountBackpressureMillis());
             snapshot.setPredictedCountAtNextInterval(connectionStatus.getNextPredictedQueuedCount());
         }
         if (connectionStatus.getBackPressureBytesThreshold() > 0) {
             snapshot.setPercentUseBytes(Math.min(100, StatusMerger.getUtilization(connectionStatus.getQueuedBytes(), connectionStatus.getBackPressureBytesThreshold())));
+
+            snapshot.setPredictionIntervalSeconds(((Long) (connectionStatus.getPredictionIntervalMillis() / 1000L)).intValue());
             snapshot.setPredictedMillisUntilBytesBackpressure(connectionStatus.getPredictedTimeToBytesBackpressureMillis());
             snapshot.setPredictedBytesAtNextInterval(connectionStatus.getNextPredictedQueuedBytes());
         }


[nifi] 15/21: NIFI-6510 Added tests for engine

Posted by ai...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

aichrist pushed a commit to branch analytics-framework
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 0bdedd5931f7652a8bf3f89e04d55a11a26b782b
Author: Yolanda Davis <yo...@gmail.com>
AuthorDate: Wed Jul 31 10:47:23 2019 -0400

    NIFI-6510 Added tests for engine
    
    (cherry picked from commit 6d7a13b)
---
 ...TestCachingConnectionStatusAnalyticsEngine.java | 39 ++++++++++++++
 .../TestConnectionStatusAnalyticsEngine.java       | 29 ++++++++++
 .../analytics/TestStatusAnalyticsEngine.java       | 63 ++++++++++++++++++++++
 3 files changed, 131 insertions(+)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestCachingConnectionStatusAnalyticsEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestCachingConnectionStatusAnalyticsEngine.java
new file mode 100644
index 0000000..abffafd
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestCachingConnectionStatusAnalyticsEngine.java
@@ -0,0 +1,39 @@
+/*
+ * 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.analytics;
+
+import org.apache.nifi.controller.flow.FlowManager;
+import org.apache.nifi.controller.status.history.ComponentStatusRepository;
+import org.junit.Test;
+
+public class TestCachingConnectionStatusAnalyticsEngine extends TestStatusAnalyticsEngine{
+
+    @Override
+    public StatusAnalyticsEngine getStatusAnalyticsEngine(FlowManager flowManager, ComponentStatusRepository componentStatusRepository) {
+        return new CachingConnectionStatusAnalyticsEngine(flowManager,componentStatusRepository);
+    }
+
+    @Test
+    public void testCachedStatusAnalytics(){
+        StatusAnalyticsEngine statusAnalyticsEngine = new CachingConnectionStatusAnalyticsEngine(flowManager,statusRepository);
+        StatusAnalytics statusAnalyticsA = statusAnalyticsEngine.getStatusAnalytics("A");
+        StatusAnalytics statusAnalyticsB = statusAnalyticsEngine.getStatusAnalytics("B");
+        StatusAnalytics statusAnalyticsTest = statusAnalyticsEngine.getStatusAnalytics("A");
+        assert(statusAnalyticsA.equals(statusAnalyticsTest));
+    }
+
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalyticsEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalyticsEngine.java
new file mode 100644
index 0000000..2005e04
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalyticsEngine.java
@@ -0,0 +1,29 @@
+/*
+ * 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.analytics;
+
+import org.apache.nifi.controller.flow.FlowManager;
+import org.apache.nifi.controller.status.history.ComponentStatusRepository;
+
+public class TestConnectionStatusAnalyticsEngine extends TestStatusAnalyticsEngine {
+
+    @Override
+    public StatusAnalyticsEngine getStatusAnalyticsEngine(FlowManager flowManager, ComponentStatusRepository statusRepository) {
+        return new ConnectionStatusAnalyticsEngine(flowManager, statusRepository);
+    }
+
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestStatusAnalyticsEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestStatusAnalyticsEngine.java
new file mode 100644
index 0000000..dfaaa02
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestStatusAnalyticsEngine.java
@@ -0,0 +1,63 @@
+/*
+ * 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.analytics;
+
+import static org.junit.Assert.assertNotNull;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.when;
+
+import java.util.Collections;
+
+import org.apache.nifi.controller.flow.FlowManager;
+import org.apache.nifi.controller.status.history.ComponentStatusRepository;
+import org.apache.nifi.controller.status.history.StatusHistory;
+import org.apache.nifi.controller.status.history.StatusSnapshot;
+import org.apache.nifi.groups.ProcessGroup;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public abstract class TestStatusAnalyticsEngine {
+
+    protected ComponentStatusRepository statusRepository;
+    protected FlowManager flowManager;
+
+    @Before
+    public void setup() {
+
+        statusRepository = Mockito.mock(ComponentStatusRepository.class);
+        flowManager = Mockito.mock(FlowManager.class);
+        ProcessGroup processGroup = Mockito.mock(ProcessGroup.class);
+        StatusHistory statusHistory = Mockito.mock(StatusHistory.class);
+        StatusSnapshot statusSnapshot = Mockito.mock(StatusSnapshot.class);
+        when(statusSnapshot.getMetricDescriptors()).thenReturn(Collections.emptySet());
+        when(flowManager.getRootGroup()).thenReturn(processGroup);
+        when(statusRepository.getConnectionStatusHistory(anyString(),any(),any(),anyInt())).thenReturn(statusHistory);
+    }
+
+    @Test
+    public void testGetStatusAnalytics(){
+        StatusAnalyticsEngine statusAnalyticsEngine = getStatusAnalyticsEngine(flowManager,statusRepository);
+        StatusAnalytics statusAnalytics = statusAnalyticsEngine.getStatusAnalytics("1");
+        assertNotNull(statusAnalytics);
+    }
+
+    public abstract StatusAnalyticsEngine getStatusAnalyticsEngine(FlowManager flowManager, ComponentStatusRepository componentStatusRepository);
+
+}


[nifi] 02/21: NIFI-6510 Implemented basic linear regression model for queue counts

Posted by ai...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

aichrist pushed a commit to branch analytics-framework
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit cfc3fdc1b5506b6405d3c4838514f30126f2fa2c
Author: Andrew I. Christianson <an...@andyic.org>
AuthorDate: Thu Jul 11 12:02:22 2019 -0400

    NIFI-6510 Implemented basic linear regression model for queue counts
---
 .../nifi-framework/nifi-framework-core/pom.xml     |  5 +++
 .../org/apache/nifi/controller/FlowController.java |  2 +-
 .../status/analytics/StatusAnalyticEngine.java     | 52 ++++++++++++++++++----
 3 files changed, 49 insertions(+), 10 deletions(-)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
index a1bff42..6551d54 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
@@ -134,6 +134,11 @@
             <artifactId>commons-io</artifactId>
         </dependency>
         <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-math3</artifactId>
+            <version>3.6.1</version>
+        </dependency>
+        <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-data-provenance-utils</artifactId>
             <version>1.10.0-SNAPSHOT</version>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index 4c0288f..f7ed734 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -608,7 +608,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
             @Override
             public void run() {
                 try {
-                    analyticsEngine.getMinTimeToBackpressure();
+                    analyticsEngine.getMinTimeToBackpressureMillis();
                 } catch (final Exception e) {
                     LOG.error("Failed to capture component stats for Stats History", e);
                 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java
index 8b69ebf..0602a93 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java
@@ -16,12 +16,14 @@
  */
 package org.apache.nifi.controller.status.analytics;
 
+import java.util.Date;
 import java.util.List;
-import java.util.Map.Entry;
 
+import org.apache.commons.math3.stat.regression.SimpleRegression;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.status.history.ComponentStatusRepository;
+import org.apache.nifi.controller.status.history.ConnectionStatusDescriptor;
 import org.apache.nifi.controller.status.history.StatusHistoryUtil;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
@@ -40,21 +42,53 @@ public class StatusAnalyticEngine {
         this.statusRepository = statusRepository;
     }
 
-    public long getMinTimeToBackpressure() {
+    public long getMinTimeToBackpressureMillis() {
         ProcessGroup rootGroup = controller.getFlowManager().getRootGroup();
         List<Connection> allConnections = rootGroup.findAllConnections();
+        long minTimeToBackpressure = Long.MAX_VALUE;
 
         for (Connection conn : allConnections) {
             LOG.info("Getting connection history for: " + conn.getIdentifier());
-            StatusHistoryDTO connHistory = StatusHistoryUtil.createStatusHistoryDTO(
-                    statusRepository.getConnectionStatusHistory(conn.getIdentifier(), null, null, Integer.MAX_VALUE));
-            for (StatusSnapshotDTO snap : connHistory.getAggregateSnapshots()) {
-                for (Entry<String, Long> snapEntry : snap.getStatusMetrics().entrySet()) {
-                    LOG.info("Snap " + snapEntry.getKey() + ": " + snapEntry.getValue());
-                }
+            Date minDate = new Date(System.currentTimeMillis() - (5 * 60 * 1000));
+            StatusHistoryDTO connHistory = StatusHistoryUtil.createStatusHistoryDTO(statusRepository
+                    .getConnectionStatusHistory(conn.getIdentifier(), minDate, null, Integer.MAX_VALUE));
+            List<StatusSnapshotDTO> aggregateSnapshots = connHistory.getAggregateSnapshots();
+
+            if (aggregateSnapshots.size() < 2) {
+                LOG.info("Not enough data to model time to backpressure.");
+                continue;
             }
+
+            long backPressureObjectThreshold = conn.getFlowFileQueue().getBackPressureObjectThreshold();
+            LOG.info("Connection " + conn.getIdentifier() + " backpressure object threshold is "
+                    + Long.toString(backPressureObjectThreshold));
+
+            ConnectionStatusDescriptor.QUEUED_COUNT.getField();
+
+            SimpleRegression regression = new SimpleRegression();
+
+            for (StatusSnapshotDTO snap : aggregateSnapshots) {
+                Long snapQueuedCount = snap.getStatusMetrics().get(ConnectionStatusDescriptor.QUEUED_COUNT.getField());
+                long snapTime = snap.getTimestamp().getTime();
+                regression.addData(snapTime, snapQueuedCount);
+            }
+
+            // Skip this connection if its queue is declining.
+            if (regression.getSlope() <= 0) {
+                LOG.info("Connection " + conn.getIdentifier() + " is not experiencing backpressure.");
+                continue;
+            }
+
+            // Compute time-to backpressure for this connection; Reduce total result iff
+            // this connection is lower.
+            long connTimeToBackpressure = Math
+                    .round((backPressureObjectThreshold - regression.getIntercept()) / regression.getSlope())
+                    - System.currentTimeMillis();
+            LOG.info("Connection " + conn.getIdentifier() + " time to backpressure is " + connTimeToBackpressure);
+            minTimeToBackpressure = Math.min(minTimeToBackpressure, connTimeToBackpressure);
         }
 
-        return 0;
+        LOG.info("Min time to backpressure is: " + Long.toString(minTimeToBackpressure));
+        return minTimeToBackpressure;
     }
 }


[nifi] 04/21: NIFI-6510 Connect the dots for StatusAnalytics -> API

Posted by ai...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

aichrist pushed a commit to branch analytics-framework
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit be0993cb6f759784cb51ce67b32c534d8cd2cdf1
Author: Andrew I. Christianson <an...@andyic.org>
AuthorDate: Tue Jul 16 15:57:14 2019 -0400

    NIFI-6510 Connect the dots for StatusAnalytics -> API
---
 ...alytics.java => ConnectionStatusAnalytics.java} |   9 +-
 .../status/analytics/StatusAnalytics.java          |   8 +-
 .../org/apache/nifi/controller/FlowController.java |  10 +-
 .../status/analytics/StatusAnalyticEngine.java     | 106 ++++++++++++++++-----
 .../org/apache/nifi/web/api/dto/DtoFactory.java    |   4 +-
 .../nifi/web/controller/ControllerFacade.java      |   9 +-
 6 files changed, 114 insertions(+), 32 deletions(-)

diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
similarity index 79%
copy from nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java
copy to nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
index d6ad3bc..12c8a15 100644
--- a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java
+++ b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
@@ -16,6 +16,13 @@
  */
 package org.apache.nifi.controller.status.analytics;
 
-public interface StatusAnalytics {
+public interface ConnectionStatusAnalytics {
     long getMinTimeToBackpressureMillis();
+    String getGroupId();
+    String getId();
+    String getName();
+    String getSourceId();
+    String getSourceName();
+    String getDestinationId();
+    String getDestinationName();
 }
diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java
index d6ad3bc..42c2abd 100644
--- a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java
+++ b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java
@@ -16,6 +16,12 @@
  */
 package org.apache.nifi.controller.status.analytics;
 
+/**
+ * StatusAnalytics
+ */
 public interface StatusAnalytics {
-    long getMinTimeToBackpressureMillis();
+
+    ConnectionStatusAnalytics getConnectionStatusAnalytics(String connectionId);
+
+    public long getMinTimeToBackpressureMillis();
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index 0c422b4..56272ff 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -350,6 +350,8 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
     // guarded by rwLock
     private NodeConnectionStatus connectionStatus;
 
+    private StatusAnalyticEngine analyticsEngine;
+
     // guarded by rwLock
     private String instanceId;
 
@@ -603,7 +605,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
             }
         }, snapshotMillis, snapshotMillis, TimeUnit.MILLISECONDS);
 
-        StatusAnalytics analyticsEngine = new StatusAnalyticEngine(this, componentStatusRepository);
+        analyticsEngine = new StatusAnalyticEngine(this, componentStatusRepository);
 
         timerDrivenEngineRef.get().scheduleWithFixedDelay(new Runnable() {
             @Override
@@ -614,7 +616,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
                     LOG.error("Failed to capture component stats for Stats History", e);
                 }
             }
-        }, 1000, 1000, TimeUnit.MILLISECONDS); //FIXME use a real/configured interval
+        }, 1000, 1000, TimeUnit.MILLISECONDS); //FIXME use a real/configured interval (or maybe just compute on the fly when requested)
 
         this.connectionStatus = new NodeConnectionStatus(nodeId, DisconnectionCode.NOT_YET_CONNECTED);
         heartbeatBeanRef.set(new HeartbeatBean(rootGroup, false));
@@ -1387,6 +1389,10 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
         return eventAccess;
     }
 
+    public StatusAnalytics getStatusAnalytics() {
+        return analyticsEngine;
+    }
+
     /**
      * Sets the root group to the given group
      *
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java
index 9231707..64c2065 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java
@@ -43,22 +43,28 @@ public class StatusAnalyticEngine implements StatusAnalytics {
     }
 
     @Override
-    public long getMinTimeToBackpressureMillis() {
+    public ConnectionStatusAnalytics getConnectionStatusAnalytics(String connectionId) {
         ProcessGroup rootGroup = controller.getFlowManager().getRootGroup();
-        List<Connection> allConnections = rootGroup.findAllConnections();
-        long minTimeToBackpressure = Long.MAX_VALUE;
+        return getConnectionStatusAnalytics(rootGroup.findConnection(connectionId));
+    }
 
-        for (Connection conn : allConnections) {
-            LOG.info("Getting connection history for: " + conn.getIdentifier());
-            Date minDate = new Date(System.currentTimeMillis() - (5 * 60 * 1000));
-            StatusHistoryDTO connHistory = StatusHistoryUtil.createStatusHistoryDTO(statusRepository
-                    .getConnectionStatusHistory(conn.getIdentifier(), minDate, null, Integer.MAX_VALUE));
-            List<StatusSnapshotDTO> aggregateSnapshots = connHistory.getAggregateSnapshots();
-
-            if (aggregateSnapshots.size() < 2) {
-                LOG.info("Not enough data to model time to backpressure.");
-                continue;
-            }
+    /**
+     * Finds the number of millis until the given connection will experience backpressure.
+     * @param conn the connection to run the analytic on
+     * @return
+     */
+    public ConnectionStatusAnalytics getConnectionStatusAnalytics(Connection conn) {
+        LOG.info("Getting connection history for: " + conn.getIdentifier());
+        long connTimeToBackpressure;
+        Date minDate = new Date(System.currentTimeMillis() - (5 * 60 * 1000));
+        StatusHistoryDTO connHistory = StatusHistoryUtil.createStatusHistoryDTO(
+                statusRepository.getConnectionStatusHistory(conn.getIdentifier(), minDate, null, Integer.MAX_VALUE));
+        List<StatusSnapshotDTO> aggregateSnapshots = connHistory.getAggregateSnapshots();
+
+        if (aggregateSnapshots.size() < 2) {
+            LOG.info("Not enough data to model time to backpressure.");
+            connTimeToBackpressure = Long.MAX_VALUE;
+        } else {
 
             long backPressureObjectThreshold = conn.getFlowFileQueue().getBackPressureObjectThreshold();
             LOG.info("Connection " + conn.getIdentifier() + " backpressure object threshold is "
@@ -77,16 +83,72 @@ public class StatusAnalyticEngine implements StatusAnalytics {
             // Skip this connection if its queue is declining.
             if (regression.getSlope() <= 0) {
                 LOG.info("Connection " + conn.getIdentifier() + " is not experiencing backpressure.");
-                continue;
+                connTimeToBackpressure = Long.MAX_VALUE;
+            } else {
+
+                // Compute time-to backpressure for this connection; Reduce total result iff
+                // this connection is lower.
+                connTimeToBackpressure = Math
+                        .round((backPressureObjectThreshold - regression.getIntercept()) / regression.getSlope())
+                        - System.currentTimeMillis();
+                LOG.info("Connection " + conn.getIdentifier() + " time to backpressure is " + connTimeToBackpressure);
+            }
+        }
+
+        return new ConnectionStatusAnalytics() {
+
+            @Override
+            public String getSourceName() {
+                return conn.getSource().getName();
+            }
+
+            @Override
+            public String getSourceId() {
+                return conn.getSource().getIdentifier();
+            }
+
+            @Override
+            public String getName() {
+                return conn.getName();
+            }
+
+            @Override
+            public long getMinTimeToBackpressureMillis() {
+                return connTimeToBackpressure;
+            }
+
+            @Override
+            public String getId() {
+                return conn.getIdentifier();
+            }
+
+            @Override
+            public String getGroupId() {
+                return conn.getProcessGroup().getIdentifier();
             }
 
-            // Compute time-to backpressure for this connection; Reduce total result iff
-            // this connection is lower.
-            long connTimeToBackpressure = Math
-                    .round((backPressureObjectThreshold - regression.getIntercept()) / regression.getSlope())
-                    - System.currentTimeMillis();
-            LOG.info("Connection " + conn.getIdentifier() + " time to backpressure is " + connTimeToBackpressure);
-            minTimeToBackpressure = Math.min(minTimeToBackpressure, connTimeToBackpressure);
+            @Override
+            public String getDestinationName() {
+                return conn.getDestination().getName();
+            }
+
+            @Override
+            public String getDestinationId() {
+                return conn.getDestination().getIdentifier();
+            }
+        };
+    }
+
+    @Override
+    public long getMinTimeToBackpressureMillis() {
+        ProcessGroup rootGroup = controller.getFlowManager().getRootGroup();
+        List<Connection> allConnections = rootGroup.findAllConnections();
+        rootGroup.findConnection("asdf");
+        long minTimeToBackpressure = Long.MAX_VALUE;
+
+        for (Connection conn : allConnections) {
+            ConnectionStatusAnalytics connAnalytics = getConnectionStatusAnalytics(conn);
+            minTimeToBackpressure = Math.min(minTimeToBackpressure, connAnalytics.getMinTimeToBackpressureMillis());
         }
 
         LOG.info("Min time to backpressure is: " + Long.toString(minTimeToBackpressure));
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
index 6903e44..b8fed37 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
@@ -105,7 +105,7 @@ import org.apache.nifi.controller.status.PortStatus;
 import org.apache.nifi.controller.status.ProcessGroupStatus;
 import org.apache.nifi.controller.status.ProcessorStatus;
 import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
-import org.apache.nifi.controller.status.analytics.StatusAnalytics;
+import org.apache.nifi.controller.status.analytics.ConnectionStatusAnalytics;
 import org.apache.nifi.controller.status.history.GarbageCollectionHistory;
 import org.apache.nifi.controller.status.history.GarbageCollectionStatus;
 import org.apache.nifi.diagnostics.GarbageCollection;
@@ -1189,7 +1189,7 @@ public final class DtoFactory {
         return connectionStatusDto;
     }
 
-    public ConnectionStatisticsDTO createConnectionStatisticsDto(final StatusAnalytics connectionStatistics) {
+    public ConnectionStatisticsDTO createConnectionStatisticsDto(final ConnectionStatusAnalytics connectionStatistics) {
         final ConnectionStatisticsDTO connectionStatisticsDTO = new ConnectionStatisticsDTO();
         connectionStatisticsDTO.setGroupId(connectionStatistics.getGroupId());
         connectionStatisticsDTO.setId(connectionStatistics.getId());
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
index c1b6754..367ea51 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
@@ -56,6 +56,7 @@ import org.apache.nifi.controller.status.PortStatus;
 import org.apache.nifi.controller.status.ProcessGroupStatus;
 import org.apache.nifi.controller.status.ProcessorStatus;
 import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
+import org.apache.nifi.controller.status.analytics.ConnectionStatusAnalytics;
 import org.apache.nifi.controller.status.analytics.StatusAnalytics;
 import org.apache.nifi.controller.status.history.ComponentStatusRepository;
 import org.apache.nifi.diagnostics.SystemDiagnostics;
@@ -687,7 +688,7 @@ public class ControllerFacade implements Authorizable {
      * @param connectionId connection id
      * @return the statistics for the specified connection
      */
-    public StatusAnalytics getConnectionStatistics(final String connectionId) {
+    public ConnectionStatusAnalytics getConnectionStatistics(final String connectionId) {
         final ProcessGroup root = getRootGroup();
         final Connection connection = root.findConnection(connectionId);
 
@@ -703,13 +704,13 @@ public class ControllerFacade implements Authorizable {
             throw new ResourceNotFoundException(String.format("Unable to locate group with id '%s'.", groupId));
         }
 
-        // TODO get from flow controller
-        final StatusAnalytics status;
+        // get from flow controller
+        final StatusAnalytics status = flowController.getStatusAnalytics();
         if (status == null) {
             throw new ResourceNotFoundException(String.format("Unable to locate connection with id '%s'.", connectionId));
         }
 
-        return status;
+        return status.getConnectionStatusAnalytics(connectionId);
     }
 
     /**


[nifi] 05/21: NIFI-6510 Added poc engine with prediction model caching

Posted by ai...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

aichrist pushed a commit to branch analytics-framework
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit bd011f3e27eef4d5fa620b8c2a5f3eccf7990492
Author: Yolanda Davis <yo...@gmail.com>
AuthorDate: Fri Jul 19 17:28:29 2019 -0400

    NIFI-6510 Added poc engine with prediction model caching
    
    (cherry picked from commit e013b91)
    
    DFA-9 - updated logging and corrected logic for checking if not in backpressure
    
    (cherry picked from commit a1f8e70)
---
 .../nifi-framework/nifi-framework-core/pom.xml     |   6 +
 .../org/apache/nifi/controller/FlowController.java |  77 +++++-----
 .../analytics/CachingStatusAnalyticEngine.java     | 160 +++++++++++++++++++++
 .../apache/nifi/reporting/StandardEventAccess.java |  14 +-
 4 files changed, 212 insertions(+), 45 deletions(-)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
index 6551d54..2d17086 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
@@ -241,6 +241,12 @@
             <version>1.10.0-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>com.github.ben-manes.caffeine</groupId>
+            <artifactId>caffeine</artifactId>
+            <version>1.0.1</version>
+            <scope>compile</scope>
+        </dependency>
     </dependencies>
     <build>
         <plugins>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index 56272ff..462b113 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -16,6 +16,39 @@
  */
 package org.apache.nifi.controller;
 
+import static java.util.Objects.requireNonNull;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.lang.management.GarbageCollectorMXBean;
+import java.lang.management.ManagementFactory;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import javax.net.ssl.SSLContext;
+
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.admin.service.AuditService;
 import org.apache.nifi.annotation.lifecycle.OnConfigurationRestored;
@@ -117,7 +150,7 @@ import org.apache.nifi.controller.service.StandardConfigurationContext;
 import org.apache.nifi.controller.service.StandardControllerServiceProvider;
 import org.apache.nifi.controller.state.manager.StandardStateManagerProvider;
 import org.apache.nifi.controller.state.server.ZooKeeperStateServer;
-import org.apache.nifi.controller.status.analytics.StatusAnalyticEngine;
+import org.apache.nifi.controller.status.analytics.CachingStatusAnalyticEngine;
 import org.apache.nifi.controller.status.analytics.StatusAnalytics;
 import org.apache.nifi.controller.status.history.ComponentStatusRepository;
 import org.apache.nifi.controller.status.history.GarbageCollectionHistory;
@@ -184,38 +217,6 @@ import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.net.ssl.SSLContext;
-import java.io.ByteArrayInputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.lang.management.GarbageCollectorMXBean;
-import java.lang.management.ManagementFactory;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.stream.Collectors;
-
-import static java.util.Objects.requireNonNull;
-
 public class FlowController implements ReportingTaskProvider, Authorizable, NodeTypeProvider {
 
     // default repository implementations
@@ -350,7 +351,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
     // guarded by rwLock
     private NodeConnectionStatus connectionStatus;
 
-    private StatusAnalyticEngine analyticsEngine;
+    private CachingStatusAnalyticEngine analyticsEngine;
 
     // guarded by rwLock
     private String instanceId;
@@ -605,18 +606,18 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
             }
         }, snapshotMillis, snapshotMillis, TimeUnit.MILLISECONDS);
 
-        analyticsEngine = new StatusAnalyticEngine(this, componentStatusRepository);
+        analyticsEngine = new CachingStatusAnalyticEngine(this, componentStatusRepository);
 
         timerDrivenEngineRef.get().scheduleWithFixedDelay(new Runnable() {
             @Override
             public void run() {
                 try {
-                    analyticsEngine.getMinTimeToBackpressureMillis();
+                    analyticsEngine.refreshModel();
                 } catch (final Exception e) {
-                    LOG.error("Failed to capture component stats for Stats History", e);
+                    LOG.error("Failed to refresh model:", e);
                 }
             }
-        }, 1000, 1000, TimeUnit.MILLISECONDS); //FIXME use a real/configured interval (or maybe just compute on the fly when requested)
+        }, 1, 1, TimeUnit.MINUTES); //FIXME use a real/configured interval (or maybe just compute on the fly when requested)
 
         this.connectionStatus = new NodeConnectionStatus(nodeId, DisconnectionCode.NOT_YET_CONNECTED);
         heartbeatBeanRef.set(new HeartbeatBean(rootGroup, false));
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingStatusAnalyticEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingStatusAnalyticEngine.java
new file mode 100644
index 0000000..5241c4a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingStatusAnalyticEngine.java
@@ -0,0 +1,160 @@
+package org.apache.nifi.controller.status.analytics;
+
+import java.util.Date;
+import java.util.List;
+
+import org.apache.commons.math3.stat.regression.SimpleRegression;
+import org.apache.nifi.connectable.Connection;
+import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.controller.status.history.ComponentStatusRepository;
+import org.apache.nifi.controller.status.history.ConnectionStatusDescriptor;
+import org.apache.nifi.controller.status.history.StatusHistoryUtil;
+import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
+import org.apache.nifi.web.api.dto.status.StatusSnapshotDTO;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+
+public class CachingStatusAnalyticEngine implements StatusAnalytics {
+    private ComponentStatusRepository statusRepository;
+    private FlowController controller;
+    private volatile Cache<String, SimpleRegression> cache;
+    private static final Logger LOG = LoggerFactory.getLogger(StatusAnalyticEngine.class);
+
+    public CachingStatusAnalyticEngine(FlowController controller, ComponentStatusRepository statusRepository) {
+        this.controller = controller;
+        this.statusRepository = statusRepository;
+        this.cache = Caffeine.newBuilder()
+                .build();
+    }
+
+    @Override
+    public ConnectionStatusAnalytics getConnectionStatusAnalytics(String connectionId) {
+
+        ProcessGroup rootGroup = controller.getFlowManager().getRootGroup();
+        Connection connection = rootGroup.findConnection(connectionId);
+        SimpleRegression cachedRegression = cache.getIfPresent(connection.getIdentifier());
+
+        if(cachedRegression != null) {
+            cache.put(connection.getIdentifier(), cachedRegression);
+        }
+
+        ConnectionStatusAnalytics cachedResult = calculate(cachedRegression,connection);
+        LOG.info("Connection: " + connectionId + " Cached backpressure Time: " + cachedResult.getMinTimeToBackpressureMillis() );
+        return cachedResult;
+    }
+
+    protected ConnectionStatusAnalytics calculate(SimpleRegression regression, Connection conn){
+        long backPressureObjectThreshold = conn.getFlowFileQueue().getBackPressureObjectThreshold();
+
+        final long connTimeToBackpressure;
+
+        if(regression == null){
+            connTimeToBackpressure = Long.MAX_VALUE;
+        }else{
+            //If calculation returns as negative only 0 will return
+            connTimeToBackpressure = Math.max(0, Math.round((backPressureObjectThreshold - regression.getIntercept()) / regression.getSlope())
+                    - System.currentTimeMillis());
+        }
+
+        return new ConnectionStatusAnalytics() {
+
+            @Override
+            public String getSourceName() {
+                return conn.getSource().getName();
+            }
+
+            @Override
+            public String getSourceId() {
+                return conn.getSource().getIdentifier();
+            }
+
+            @Override
+            public String getName() {
+                return conn.getName();
+            }
+
+            @Override
+            public long getMinTimeToBackpressureMillis() {
+                return connTimeToBackpressure;
+            }
+
+            @Override
+            public String getId() {
+                return conn.getIdentifier();
+            }
+
+            @Override
+            public String getGroupId() {
+                return conn.getProcessGroup().getIdentifier();
+            }
+
+            @Override
+            public String getDestinationName() {
+                return conn.getDestination().getName();
+            }
+
+            @Override
+            public String getDestinationId() {
+                return conn.getDestination().getIdentifier();
+            }
+        };
+
+    }
+
+    /**
+     * Get backpressure model based on current data
+     * @param conn the connection to run the analytic on
+     * @return
+     */
+    protected SimpleRegression getBackPressureRegressionModel(Connection conn) {
+        Date minDate = new Date(System.currentTimeMillis() - (5 * 60 * 1000));
+        StatusHistoryDTO connHistory = StatusHistoryUtil.createStatusHistoryDTO(
+                statusRepository.getConnectionStatusHistory(conn.getIdentifier(), minDate, null, Integer.MAX_VALUE));
+        List<StatusSnapshotDTO> aggregateSnapshots = connHistory.getAggregateSnapshots();
+
+        if (aggregateSnapshots.size() < 2) {
+            LOG.info("Not enough data to model time to backpressure.");
+            return null;
+        } else {
+
+            ConnectionStatusDescriptor.QUEUED_COUNT.getField();
+            SimpleRegression regression = new SimpleRegression();
+
+            for (StatusSnapshotDTO snap : aggregateSnapshots) {
+                Long snapQueuedCount = snap.getStatusMetrics().get(ConnectionStatusDescriptor.QUEUED_COUNT.getField());
+                long snapTime = snap.getTimestamp().getTime();
+                regression.addData(snapTime, snapQueuedCount);
+                LOG.info("Connection " + conn.getIdentifier() + " statistics: ("+snapTime+","+snapQueuedCount+")");
+            }
+
+            if (regression.getSlope() <= 0 && !conn.getFlowFileQueue().isFull()) {
+                LOG.info("Connection " + conn.getIdentifier() + " is not experiencing backpressure.");
+                return null;
+            } else {
+                return regression;
+            }
+        }
+
+    }
+
+    public void refreshModel() {
+        ProcessGroup rootGroup = controller.getFlowManager().getRootGroup();
+        List<Connection> allConnections = rootGroup.findAllConnections();
+        cache.invalidateAll();
+        for (Connection conn : allConnections) {
+            SimpleRegression regression = getBackPressureRegressionModel(conn);
+            if(regression != null) {
+                cache.put(conn.getIdentifier(), regression);
+            }
+        }
+    }
+
+    @Override
+    public long getMinTimeToBackpressureMillis() {
+        return 0;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java
index 095ddf8..0b4b73c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java
@@ -16,6 +16,13 @@
  */
 package org.apache.nifi.reporting;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.commons.collections4.Predicate;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.action.Action;
@@ -54,13 +61,6 @@ import org.apache.nifi.registry.flow.VersionControlInformation;
 import org.apache.nifi.remote.PublicPort;
 import org.apache.nifi.remote.RemoteGroupPort;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-
 public class StandardEventAccess implements UserAwareEventAccess {
     private final FlowFileEventRepository flowFileEventRepository;
     private final FlowController flowController;


[nifi] 13/21: NIFI-6510 Add prediction percent values and predicted interval seconds

Posted by ai...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

aichrist pushed a commit to branch analytics-framework
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 8a48c779d1e7c63caf404103851ccdff72246a7c
Author: Yolanda Davis <yo...@gmail.com>
AuthorDate: Mon Jul 29 08:21:29 2019 -0400

    NIFI-6510 Add prediction percent values and predicted interval seconds
    
    (cherry picked from commit e60015d)
---
 .../src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java       | 6 ++++++
 1 file changed, 6 insertions(+)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
index e56e99a..32de8dc 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
@@ -1184,6 +1184,9 @@ public final class DtoFactory {
             snapshot.setPredictionIntervalSeconds(((Long) (connectionStatus.getPredictionIntervalMillis() / 1000L)).intValue());
             snapshot.setPredictedMillisUntilCountBackpressure(connectionStatus.getPredictedTimeToCountBackpressureMillis());
             snapshot.setPredictedCountAtNextInterval(connectionStatus.getNextPredictedQueuedCount());
+            snapshot.setPredictedPercentCount(connectionStatus.getPredictedPercentCount());
+            snapshot.setPredictedPercentBytes(connectionStatus.getPredictedPercentBytes());
+            snapshot.setPredictionIntervalSeconds(((Long)(connectionStatus.getPredictionIntervalMillis() / 1000L)).intValue());
         }
         if (connectionStatus.getBackPressureBytesThreshold() > 0) {
             snapshot.setPercentUseBytes(Math.min(100, StatusMerger.getUtilization(connectionStatus.getQueuedBytes(), connectionStatus.getBackPressureBytesThreshold())));
@@ -1191,6 +1194,9 @@ public final class DtoFactory {
             snapshot.setPredictionIntervalSeconds(((Long) (connectionStatus.getPredictionIntervalMillis() / 1000L)).intValue());
             snapshot.setPredictedMillisUntilBytesBackpressure(connectionStatus.getPredictedTimeToBytesBackpressureMillis());
             snapshot.setPredictedBytesAtNextInterval(connectionStatus.getNextPredictedQueuedBytes());
+            snapshot.setPredictedPercentCount(connectionStatus.getPredictedPercentCount());
+            snapshot.setPredictedPercentBytes(connectionStatus.getPredictedPercentBytes());
+            snapshot.setPredictionIntervalSeconds(((Long)(connectionStatus.getPredictionIntervalMillis() / 1000L)).intValue());
         }
 
         StatusMerger.updatePrettyPrintedFields(snapshot);


[nifi] 06/21: NIFI-6510 Updated objects and interfaces to reflect 4 prediction metrics

Posted by ai...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

aichrist pushed a commit to branch analytics-framework
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 725241384814fe86f61966c57d2f98f35aaa7669
Author: Matthew Burgess <ma...@apache.org>
AuthorDate: Mon Jul 22 18:11:58 2019 -0400

    NIFI-6510 Updated objects and interfaces to reflect 4 prediction metrics
    
    (cherry picked from commit 050e0fc)
    
    (cherry picked from commit 9fd365f)
---
 .../nifi/controller/status/ConnectionStatus.java   | 48 ++++++++++++++
 .../analytics/ConnectionStatusAnalytics.java       | 26 +++++++-
 .../status/analytics/StatusAnalytics.java          | 30 ++++++++-
 .../status/ConnectionStatisticsSnapshotDTO.java    | 47 ++++++++++++--
 .../dto/status/ConnectionStatusSnapshotDTO.java    | 44 +++++++++++++
 .../entity/ConnectionStatisticsSnapshotEntity.java |  1 -
 .../org/apache/nifi/controller/FlowController.java | 75 +++++++++-------------
 .../status/analytics/StatusAnalyticEngine.java     | 42 +++++++++++-
 .../apache/nifi/reporting/StandardEventAccess.java |  9 +++
 .../org/apache/nifi/web/api/dto/DtoFactory.java    |  9 ++-
 10 files changed, 274 insertions(+), 57 deletions(-)

diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/ConnectionStatus.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/ConnectionStatus.java
index 341fda2..0e5d306 100644
--- a/nifi-api/src/main/java/org/apache/nifi/controller/status/ConnectionStatus.java
+++ b/nifi-api/src/main/java/org/apache/nifi/controller/status/ConnectionStatus.java
@@ -40,6 +40,10 @@ public class ConnectionStatus implements Cloneable {
     private long outputBytes;
     private int maxQueuedCount;
     private long maxQueuedBytes;
+    private int nextPredictedQueuedCount;
+    private long nextPredictedQueuedBytes;
+    private long predictedTimeToCountBackpressureMillis;
+    private long predictedTimeToBytesBackpressureMillis;
 
     public String getId() {
         return id;
@@ -186,6 +190,38 @@ public class ConnectionStatus implements Cloneable {
         this.backPressureBytesThreshold = backPressureBytesThreshold;
     }
 
+    public int getNextPredictedQueuedCount() {
+        return nextPredictedQueuedCount;
+    }
+
+    public void setNextPredictedQueuedCount(int nextPredictedQueuedCount) {
+        this.nextPredictedQueuedCount = nextPredictedQueuedCount;
+    }
+
+    public long getNextPredictedQueuedBytes() {
+        return nextPredictedQueuedBytes;
+    }
+
+    public void setNextPredictedQueuedBytes(long nextPredictedQueuedBytes) {
+        this.nextPredictedQueuedBytes = nextPredictedQueuedBytes;
+    }
+
+    public long getPredictedTimeToCountBackpressureMillis() {
+        return predictedTimeToCountBackpressureMillis;
+    }
+
+    public void setPredictedTimeToCountBackpressureMillis(long predictedTimeToCountBackpressureMillis) {
+        this.predictedTimeToCountBackpressureMillis = predictedTimeToCountBackpressureMillis;
+    }
+
+    public long getPredictedTimeToBytesBackpressureMillis() {
+        return predictedTimeToBytesBackpressureMillis;
+    }
+
+    public void setPredictedTimeToBytesBackpressureMillis(long predictedTimeToBytesBackpressureMillis) {
+        this.predictedTimeToBytesBackpressureMillis = predictedTimeToBytesBackpressureMillis;
+    }
+
     @Override
     public ConnectionStatus clone() {
         final ConnectionStatus clonedObj = new ConnectionStatus();
@@ -206,6 +242,10 @@ public class ConnectionStatus implements Cloneable {
         clonedObj.backPressureObjectThreshold = backPressureObjectThreshold;
         clonedObj.maxQueuedBytes = maxQueuedBytes;
         clonedObj.maxQueuedCount = maxQueuedCount;
+        clonedObj.nextPredictedQueuedBytes = nextPredictedQueuedBytes;
+        clonedObj.nextPredictedQueuedCount = nextPredictedQueuedCount;
+        clonedObj.predictedTimeToBytesBackpressureMillis = predictedTimeToBytesBackpressureMillis;
+        clonedObj.predictedTimeToCountBackpressureMillis = predictedTimeToCountBackpressureMillis;
         return clonedObj;
     }
 
@@ -246,6 +286,14 @@ public class ConnectionStatus implements Cloneable {
         builder.append(maxQueuedCount);
         builder.append(", maxQueueBytes=");
         builder.append(maxQueuedBytes);
+        builder.append(", nextPredictedQueuedBytes=");
+        builder.append(nextPredictedQueuedBytes);
+        builder.append(", nextPredictedQueuedCount=");
+        builder.append(nextPredictedQueuedCount);
+        builder.append(", predictedTimeToBytesBackpressureMillis=");
+        builder.append(predictedTimeToBytesBackpressureMillis);
+        builder.append(", predictedTimeToCountBackpressureMillis=");
+        builder.append(predictedTimeToCountBackpressureMillis);
         builder.append("]");
         return builder.toString();
     }
diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
index 12c8a15..2380d55 100644
--- a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
+++ b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
@@ -17,7 +17,31 @@
 package org.apache.nifi.controller.status.analytics;
 
 public interface ConnectionStatusAnalytics {
-    long getMinTimeToBackpressureMillis();
+
+    /**
+     * Returns the predicted time (in milliseconds) when backpressure is expected to be applied to this connection, based on the total number of bytes in the queue.
+     * @return milliseconds until backpressure is predicted to occur, based on the total number of bytes in the queue.
+     */
+    long getTimeToBytesBackpressureMillis();
+
+    /**
+     * Returns the predicted time (in milliseconds) when backpressure is expected to be applied to this connection, based on the number of objects in the queue.
+     * @return milliseconds until backpressure is predicted to occur, based on the number of objects in the queue.
+     */
+    long getTimeToCountBackpressureMillis();
+
+    /**
+     * Returns the predicted total number of bytes in the queue to occur at the next configured interval (5 mins in the future, e.g.).
+     * @return milliseconds until backpressure is predicted to occur, based on the total number of bytes in the queue.
+     */
+    long getNextIntervalBytes();
+
+    /**
+     * Returns the predicted number of objects in the queue to occur at the next configured interval (5 mins in the future, e.g.).
+     * @return milliseconds until backpressure is predicted to occur, based on the number of bytes in the queue.
+     */
+    int getNextIntervalCount();
+
     String getGroupId();
     String getId();
     String getName();
diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java
index 42c2abd..7d29314 100644
--- a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java
+++ b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java
@@ -21,7 +21,35 @@ package org.apache.nifi.controller.status.analytics;
  */
 public interface StatusAnalytics {
 
+    /**
+     * Returns a ConnectionStatusAnalytics object containing all relevant metrics and analytical & statistical objects, as well as identity information for the connection.
+     *
+     * @param connectionId The unique ID of the connection
+     * @return A ConnectionStatusAnalytics object
+     */
     ConnectionStatusAnalytics getConnectionStatusAnalytics(String connectionId);
 
-    public long getMinTimeToBackpressureMillis();
+    /**
+     * Returns the predicted time (in milliseconds) when backpressure is expected to be applied to this connection, based on the total number of bytes in the queue.
+     * @return milliseconds until backpressure is predicted to occur, based on the total number of bytes in the queue.
+     */
+    long getTimeToBytesBackpressureMillis();
+
+    /**
+     * Returns the predicted time (in milliseconds) when backpressure is expected to be applied to this connection, based on the number of objects in the queue.
+     * @return milliseconds until backpressure is predicted to occur, based on the number of objects in the queue.
+     */
+    long getTimeToCountBackpressureMillis();
+
+    /**
+     * Returns the predicted total number of bytes in the queue to occur at the next configured interval (5 mins in the future, e.g.).
+     * @return milliseconds until backpressure is predicted to occur, based on the total number of bytes in the queue.
+     */
+    long getNextIntervalBytes();
+
+    /**
+     * Returns the predicted number of objects in the queue to occur at the next configured interval (5 mins in the future, e.g.).
+     * @return milliseconds until backpressure is predicted to occur, based on the number of bytes in the queue.
+     */
+    int getNextIntervalCount();
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatisticsSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatisticsSnapshotDTO.java
index e914f74..526bdcf 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatisticsSnapshotDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatisticsSnapshotDTO.java
@@ -35,7 +35,10 @@ public class ConnectionStatisticsSnapshotDTO implements Cloneable {
     private String destinationId;
     private String destinationName;
 
-    private Long predictedMillisUntilBackpressure = 0L;
+    private Long predictedMillisUntilCountBackpressure = 0L;
+    private Long predictedMillisUntilBytesBackpressure = 0L;
+    private Integer predictedCountAtNextInterval = 0;
+    private Long predictedBytesAtNextInterval = 0L;
 
     /* getters / setters */
     /**
@@ -122,13 +125,40 @@ public class ConnectionStatisticsSnapshotDTO implements Cloneable {
         this.destinationName = destinationName;
     }
 
-    @ApiModelProperty("The predicted number of milliseconds before the connection will have backpressure applied.")
-    public Long getPredictedMillisUntilBackpressure() {
-        return predictedMillisUntilBackpressure;
+    @ApiModelProperty("The predicted number of milliseconds before the connection will have backpressure applied, based on the queued count.")
+    public Long getPredictedMillisUntilCountBackpressure() {
+        return predictedMillisUntilCountBackpressure;
     }
 
-    public void setPredictedMillisUntilBackpressure(Long predictedMillisUntilBackpressure) {
-        this.predictedMillisUntilBackpressure = predictedMillisUntilBackpressure;
+    public void setPredictedMillisUntilCountBackpressure(Long predictedMillisUntilCountBackpressure) {
+        this.predictedMillisUntilCountBackpressure = predictedMillisUntilCountBackpressure;
+    }
+
+    @ApiModelProperty("The predicted number of milliseconds before the connection will have backpressure applied, based on the total number of bytes in the queue.")
+    public Long getPredictedMillisUntilBytesBackpressure() {
+        return predictedMillisUntilBytesBackpressure;
+    }
+
+    public void setPredictedMillisUntilBytesBackpressure(Long predictedMillisUntilBytesBackpressure) {
+        this.predictedMillisUntilBytesBackpressure = predictedMillisUntilBytesBackpressure;
+    }
+
+    @ApiModelProperty("The predicted number of queued objects at the next configured interval.")
+    public Integer getPredictedCountAtNextInterval() {
+        return predictedCountAtNextInterval;
+    }
+
+    public void setPredictedCountAtNextInterval(Integer predictedCountAtNextInterval) {
+        this.predictedCountAtNextInterval = predictedCountAtNextInterval;
+    }
+
+    @ApiModelProperty("The predicted total number of bytes in the queue at the next configured interval.")
+    public Long getPredictedBytesAtNextInterval() {
+        return predictedBytesAtNextInterval;
+    }
+
+    public void setPredictedBytesAtNextInterval(Long predictedBytesAtNextInterval) {
+        this.predictedBytesAtNextInterval = predictedBytesAtNextInterval;
     }
 
     @Override
@@ -142,7 +172,10 @@ public class ConnectionStatisticsSnapshotDTO implements Cloneable {
         other.setSourceId(getSourceId());
         other.setSourceName(getSourceName());
 
-        other.setPredictedMillisUntilBackpressure(getPredictedMillisUntilBackpressure());
+        other.setPredictedMillisUntilCountBackpressure(getPredictedMillisUntilCountBackpressure());
+        other.setPredictedMillisUntilBytesBackpressure(getPredictedMillisUntilBytesBackpressure());
+        other.setPredictedCountAtNextInterval(getPredictedCountAtNextInterval());
+        other.setPredictedBytesAtNextInterval(getPredictedBytesAtNextInterval());
 
         return other;
     }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusSnapshotDTO.java
index 7ba93cc..3237385 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusSnapshotDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusSnapshotDTO.java
@@ -48,6 +48,10 @@ public class ConnectionStatusSnapshotDTO implements Cloneable {
     private String queuedCount;
     private Integer percentUseCount;
     private Integer percentUseBytes;
+    private Long predictedMillisUntilCountBackpressure = 0L;
+    private Long predictedMillisUntilBytesBackpressure = 0L;
+    private Integer predictedCountAtNextInterval = 0;
+    private Long predictedBytesAtNextInterval = 0L;
 
     /* getters / setters */
     /**
@@ -271,6 +275,42 @@ public class ConnectionStatusSnapshotDTO implements Cloneable {
         this.percentUseBytes = percentUseBytes;
     }
 
+    @ApiModelProperty("The predicted number of milliseconds before the connection will have backpressure applied, based on the queued count.")
+    public Long getPredictedMillisUntilCountBackpressure() {
+        return predictedMillisUntilCountBackpressure;
+    }
+
+    public void setPredictedMillisUntilCountBackpressure(Long predictedMillisUntilCountBackpressure) {
+        this.predictedMillisUntilCountBackpressure = predictedMillisUntilCountBackpressure;
+    }
+
+    @ApiModelProperty("The predicted number of milliseconds before the connection will have backpressure applied, based on the total number of bytes in the queue.")
+    public Long getPredictedMillisUntilBytesBackpressure() {
+        return predictedMillisUntilBytesBackpressure;
+    }
+
+    public void setPredictedMillisUntilBytesBackpressure(Long predictedMillisUntilBytesBackpressure) {
+        this.predictedMillisUntilBytesBackpressure = predictedMillisUntilBytesBackpressure;
+    }
+
+    @ApiModelProperty("The predicted number of queued objects at the next configured interval.")
+    public Integer getPredictedCountAtNextInterval() {
+        return predictedCountAtNextInterval;
+    }
+
+    public void setPredictedCountAtNextInterval(Integer predictedCountAtNextInterval) {
+        this.predictedCountAtNextInterval = predictedCountAtNextInterval;
+    }
+
+    @ApiModelProperty("The predicted total number of bytes in the queue at the next configured interval.")
+    public Long getPredictedBytesAtNextInterval() {
+        return predictedBytesAtNextInterval;
+    }
+
+    public void setPredictedBytesAtNextInterval(Long predictedBytesAtNextInterval) {
+        this.predictedBytesAtNextInterval = predictedBytesAtNextInterval;
+    }
+
     @Override
     public ConnectionStatusSnapshotDTO clone() {
         final ConnectionStatusSnapshotDTO other = new ConnectionStatusSnapshotDTO();
@@ -295,6 +335,10 @@ public class ConnectionStatusSnapshotDTO implements Cloneable {
         other.setQueuedSize(getQueuedSize());
         other.setPercentUseBytes(getPercentUseBytes());
         other.setPercentUseCount(getPercentUseCount());
+        other.setPredictedMillisUntilCountBackpressure(getPredictedMillisUntilCountBackpressure());
+        other.setPredictedMillisUntilBytesBackpressure(getPredictedMillisUntilBytesBackpressure());
+        other.setPredictedCountAtNextInterval(getPredictedCountAtNextInterval());
+        other.setPredictedBytesAtNextInterval(getPredictedBytesAtNextInterval());
 
         return other;
     }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionStatisticsSnapshotEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionStatisticsSnapshotEntity.java
index da7e5ca..6f4eee5 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionStatisticsSnapshotEntity.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionStatisticsSnapshotEntity.java
@@ -19,7 +19,6 @@ package org.apache.nifi.web.api.entity;
 import io.swagger.annotations.ApiModelProperty;
 import org.apache.nifi.web.api.dto.ReadablePermission;
 import org.apache.nifi.web.api.dto.status.ConnectionStatisticsSnapshotDTO;
-import org.apache.nifi.web.api.dto.status.ConnectionStatusSnapshotDTO;
 
 /**
  * A serialized representation of this class can be placed in the entity body of a request or response to or from the API.
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index 462b113..5f67b49 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -16,39 +16,6 @@
  */
 package org.apache.nifi.controller;
 
-import static java.util.Objects.requireNonNull;
-
-import java.io.ByteArrayInputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.lang.management.GarbageCollectorMXBean;
-import java.lang.management.ManagementFactory;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.stream.Collectors;
-
-import javax.net.ssl.SSLContext;
-
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.admin.service.AuditService;
 import org.apache.nifi.annotation.lifecycle.OnConfigurationRestored;
@@ -217,6 +184,38 @@ import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.net.ssl.SSLContext;
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.lang.management.GarbageCollectorMXBean;
+import java.lang.management.ManagementFactory;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.requireNonNull;
+
 public class FlowController implements ReportingTaskProvider, Authorizable, NodeTypeProvider {
 
     // default repository implementations
@@ -608,16 +607,6 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
 
         analyticsEngine = new CachingStatusAnalyticEngine(this, componentStatusRepository);
 
-        timerDrivenEngineRef.get().scheduleWithFixedDelay(new Runnable() {
-            @Override
-            public void run() {
-                try {
-                    analyticsEngine.refreshModel();
-                } catch (final Exception e) {
-                    LOG.error("Failed to refresh model:", e);
-                }
-            }
-        }, 1, 1, TimeUnit.MINUTES); //FIXME use a real/configured interval (or maybe just compute on the fly when requested)
 
         this.connectionStatus = new NodeConnectionStatus(nodeId, DisconnectionCode.NOT_YET_CONNECTED);
         heartbeatBeanRef.set(new HeartbeatBean(rootGroup, false));
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java
index 64c2065..024c138 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java
@@ -113,10 +113,26 @@ public class StatusAnalyticEngine implements StatusAnalytics {
             }
 
             @Override
-            public long getMinTimeToBackpressureMillis() {
+            public long getTimeToCountBackpressureMillis() {
                 return connTimeToBackpressure;
             }
 
+            // TODO - populate the other prediction fields
+            @Override
+            public long getTimeToBytesBackpressureMillis() {
+                return 0;
+            }
+
+            @Override
+            public long getNextIntervalBytes() {
+                return 0;
+            }
+
+            @Override
+            public int getNextIntervalCount() {
+                return 0;
+            }
+
             @Override
             public String getId() {
                 return conn.getIdentifier();
@@ -139,7 +155,6 @@ public class StatusAnalyticEngine implements StatusAnalytics {
         };
     }
 
-    @Override
     public long getMinTimeToBackpressureMillis() {
         ProcessGroup rootGroup = controller.getFlowManager().getRootGroup();
         List<Connection> allConnections = rootGroup.findAllConnections();
@@ -148,10 +163,31 @@ public class StatusAnalyticEngine implements StatusAnalytics {
 
         for (Connection conn : allConnections) {
             ConnectionStatusAnalytics connAnalytics = getConnectionStatusAnalytics(conn);
-            minTimeToBackpressure = Math.min(minTimeToBackpressure, connAnalytics.getMinTimeToBackpressureMillis());
+            minTimeToBackpressure = Math.min(minTimeToBackpressure, connAnalytics.getTimeToCountBackpressureMillis());
         }
 
         LOG.info("Min time to backpressure is: " + Long.toString(minTimeToBackpressure));
         return minTimeToBackpressure;
     }
+
+    // TODO - populate the prediction fields. Do we need to pass in connection ID?
+    @Override
+    public long getTimeToCountBackpressureMillis() {
+        return 0;
+    }
+
+    @Override
+    public long getTimeToBytesBackpressureMillis() {
+        return 0;
+    }
+
+    @Override
+    public long getNextIntervalBytes() {
+        return 0;
+    }
+
+    @Override
+    public int getNextIntervalCount() {
+        return 0;
+    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java
index 0b4b73c..f943856 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java
@@ -51,6 +51,7 @@ import org.apache.nifi.controller.status.ProcessorStatus;
 import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
 import org.apache.nifi.controller.status.RunStatus;
 import org.apache.nifi.controller.status.TransmissionStatus;
+import org.apache.nifi.controller.status.analytics.StatusAnalytics;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.history.History;
@@ -338,6 +339,14 @@ public class StandardEventAccess implements UserAwareEventAccess {
                 bytesTransferred += connectionStatusReport.getContentSizeIn() + connectionStatusReport.getContentSizeOut();
             }
 
+            final StatusAnalytics statusAnalytics = flowController.getStatusAnalytics();
+            if (statusAnalytics != null) {
+                connStatus.setPredictedTimeToBytesBackpressureMillis(statusAnalytics.getTimeToBytesBackpressureMillis());
+                connStatus.setPredictedTimeToCountBackpressureMillis(statusAnalytics.getTimeToCountBackpressureMillis());
+                connStatus.setNextPredictedQueuedBytes(statusAnalytics.getNextIntervalBytes());
+                connStatus.setNextPredictedQueuedCount(statusAnalytics.getNextIntervalCount());
+            }
+
             if (isConnectionAuthorized) {
                 if (StringUtils.isNotBlank(conn.getName())) {
                     connStatus.setName(conn.getName());
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
index b8fed37..2ff12e6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
@@ -1179,9 +1179,13 @@ public final class DtoFactory {
 
         if (connectionStatus.getBackPressureObjectThreshold() > 0) {
             snapshot.setPercentUseCount(Math.min(100, StatusMerger.getUtilization(connectionStatus.getQueuedCount(), connectionStatus.getBackPressureObjectThreshold())));
+            snapshot.setPredictedMillisUntilCountBackpressure(connectionStatus.getPredictedTimeToCountBackpressureMillis());
+            snapshot.setPredictedCountAtNextInterval(connectionStatus.getNextPredictedQueuedCount());
         }
         if (connectionStatus.getBackPressureBytesThreshold() > 0) {
             snapshot.setPercentUseBytes(Math.min(100, StatusMerger.getUtilization(connectionStatus.getQueuedBytes(), connectionStatus.getBackPressureBytesThreshold())));
+            snapshot.setPredictedMillisUntilBytesBackpressure(connectionStatus.getPredictedTimeToBytesBackpressureMillis());
+            snapshot.setPredictedBytesAtNextInterval(connectionStatus.getNextPredictedQueuedBytes());
         }
 
         StatusMerger.updatePrettyPrintedFields(snapshot);
@@ -1209,7 +1213,10 @@ public final class DtoFactory {
         snapshot.setSourceName(connectionStatistics.getSourceName());
         snapshot.setDestinationName(connectionStatistics.getDestinationName());
 
-        snapshot.setPredictedMillisUntilBackpressure(connectionStatistics.getMinTimeToBackpressureMillis());
+        snapshot.setPredictedMillisUntilBytesBackpressure(connectionStatistics.getTimeToBytesBackpressureMillis());
+        snapshot.setPredictedMillisUntilCountBackpressure(connectionStatistics.getTimeToCountBackpressureMillis());
+        snapshot.setPredictedBytesAtNextInterval(connectionStatistics.getNextIntervalBytes());
+        snapshot.setPredictedCountAtNextInterval(connectionStatistics.getNextIntervalCount());
 
         return connectionStatisticsDTO;
     }


[nifi] 16/21: NIFI-6150 Added tests for connection status analytics class, corrected variable names

Posted by ai...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

aichrist pushed a commit to branch analytics-framework
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit ecce2794a174c025df45c67746d6b56e99ba2f1b
Author: Yolanda Davis <yo...@gmail.com>
AuthorDate: Wed Jul 31 13:31:24 2019 -0400

    NIFI-6150 Added tests for connection status analytics class, corrected variable names
    
    (cherry picked from commit 58c7c81)
---
 .../analytics/ConnectionStatusAnalytics.java       |   6 +-
 .../analytics/TestConnectionStatusAnalytics.java   | 159 +++++++++++++++++++++
 2 files changed, 162 insertions(+), 3 deletions(-)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
index 8b7964e..6b831fa 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
@@ -250,7 +250,7 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
 
     private final ExtractFunction extract = (metric, statusHistory) -> {
 
-        List<Double> counts = new ArrayList<>();
+        List<Double> values = new ArrayList<>();
         List<Double> times = new ArrayList<>();
 
         StatusHistoryDTO statusHistoryDTO = StatusHistoryUtil.createStatusHistoryDTO(statusHistory);
@@ -258,10 +258,10 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
         for (StatusSnapshotDTO snap : statusHistoryDTO.getAggregateSnapshots()) {
             Long snapValue = snap.getStatusMetrics().get(metric);
             long snapTime = snap.getTimestamp().getTime();
-            counts.add((double) snapValue);
+            values.add((double) snapValue);
             times.add((double) snapTime);
         }
-        return new Tuple<>(times.stream(), counts.stream());
+        return new Tuple<>(times.stream(), values.stream());
 
     };
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalytics.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalytics.java
new file mode 100644
index 0000000..e17ae9a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalytics.java
@@ -0,0 +1,159 @@
+package org.apache.nifi.controller.status.analytics;
+
+import static org.junit.Assert.assertNotNull;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.when;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.nifi.connectable.Connection;
+import org.apache.nifi.controller.flow.FlowManager;
+import org.apache.nifi.controller.queue.FlowFileQueue;
+import org.apache.nifi.controller.status.history.ComponentStatusRepository;
+import org.apache.nifi.controller.status.history.ConnectionStatusDescriptor;
+import org.apache.nifi.controller.status.history.MetricDescriptor;
+import org.apache.nifi.controller.status.history.StandardStatusSnapshot;
+import org.apache.nifi.controller.status.history.StatusHistory;
+import org.apache.nifi.controller.status.history.StatusSnapshot;
+import org.apache.nifi.groups.ProcessGroup;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class TestConnectionStatusAnalytics {
+
+    private static final Set<MetricDescriptor<?>> CONNECTION_METRICS = Arrays.stream(ConnectionStatusDescriptor.values())
+            .map(ConnectionStatusDescriptor::getDescriptor)
+            .collect(Collectors.toSet());
+
+    protected ConnectionStatusAnalytics getConnectionStatusAnalytics(Long queuedBytes, Long queuedCount,String backPressureDataSizeThreshhold, Long backPressureObjectThreshold, Boolean isConstantStatus){
+        ComponentStatusRepository statusRepository = Mockito.mock(ComponentStatusRepository.class);
+        FlowManager flowManager;flowManager = Mockito.mock(FlowManager.class);
+        final ProcessGroup processGroup = Mockito.mock(ProcessGroup.class);
+        final StatusHistory statusHistory = Mockito.mock(StatusHistory.class);
+        final Connection connection = Mockito.mock(Connection.class);
+        final FlowFileQueue flowFileQueue = Mockito.mock(FlowFileQueue.class);
+        final List<Connection> connections = new ArrayList<>();
+        final String connectionIdentifier = "1";
+        connections.add(connection);
+
+        List<StatusSnapshot> snapshotList = new ArrayList<>();
+        final long startTime = System.currentTimeMillis();
+        int iterations = 10;
+
+        for (int i=0; i < iterations; i++) {
+            final StandardStatusSnapshot snapshot = new StandardStatusSnapshot(CONNECTION_METRICS);
+            snapshot.setTimestamp(new Date(startTime + i * 1000));
+            snapshot.addStatusMetric(ConnectionStatusDescriptor.QUEUED_BYTES.getDescriptor(), (isConstantStatus || i < 5) ? queuedBytes: queuedBytes * 2);
+            snapshot.addStatusMetric(ConnectionStatusDescriptor.QUEUED_COUNT.getDescriptor(), (isConstantStatus || i < 5) ? queuedCount: queuedCount * 2);
+            snapshotList.add(snapshot);
+        }
+
+        when(flowFileQueue.getBackPressureDataSizeThreshold()).thenReturn(backPressureDataSizeThreshhold);
+        when(flowFileQueue.getBackPressureObjectThreshold()).thenReturn(backPressureObjectThreshold);
+        when(connection.getIdentifier()).thenReturn(connectionIdentifier);
+        when(connection.getFlowFileQueue()).thenReturn(flowFileQueue);
+        when(processGroup.findAllConnections()).thenReturn(connections);
+        when(statusHistory.getStatusSnapshots()).thenReturn(snapshotList);
+        when(flowManager.getRootGroup()).thenReturn(processGroup);
+        when(statusRepository.getConnectionStatusHistory(anyString(),any(),any(),anyInt())).thenReturn(statusHistory);
+        ConnectionStatusAnalytics connectionStatusAnalytics = new ConnectionStatusAnalytics(statusRepository,flowManager,connectionIdentifier,false);
+        connectionStatusAnalytics.init();
+        return connectionStatusAnalytics;
+    }
+
+    @Test
+    public void testGetIntervalTimeMillis(){
+        ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, true);
+        Long interval = connectionStatusAnalytics.getIntervalTimeMillis();
+        assertNotNull(interval);
+        assert(interval == 300000);
+    }
+
+    @Test
+    public void testGetTimeToCountBackpressureMillisConstantStatus(){
+        ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, true);
+        Long countTime = connectionStatusAnalytics.getTimeToCountBackpressureMillis();
+        assertNotNull(countTime);
+        assert(countTime == -1L);
+    }
+
+    @Test
+    public void testGetTimeToCountBackpressureMillisVaryingStatus(){
+        ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, false);
+        Long countTime = connectionStatusAnalytics.getTimeToCountBackpressureMillis();
+        assertNotNull(countTime);
+        assert(countTime > -1L);
+    }
+
+    @Test
+    public void testGetTimeToBytesBackpressureMillisConstantStatus(){
+        ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, true);
+        Long bytesTime = connectionStatusAnalytics.getTimeToBytesBackpressureMillis();
+        assertNotNull(bytesTime);
+        assert(bytesTime == -1L);
+    }
+
+    @Test
+    public void testGetTimeToBytesBackpressureMillisVaryingStatus(){
+        ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, false);
+        Long bytesTime = connectionStatusAnalytics.getTimeToBytesBackpressureMillis();
+        assertNotNull(bytesTime);
+        assert(bytesTime > -1L);
+    }
+
+    @Test
+    public void testGetNextIntervalBytesConstantStatus(){
+        ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, true);
+        Long nextBytes = connectionStatusAnalytics.getNextIntervalBytes();
+        assertNotNull(nextBytes);
+        assert(nextBytes == 5000L);
+    }
+
+    @Test
+    public void testGetNextIntervalBytesVaryingStatus(){
+        ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, false);
+        Long nextBytes = connectionStatusAnalytics.getNextIntervalBytes();
+        assertNotNull(nextBytes);
+        assert(nextBytes > -1L);
+    }
+
+    @Test
+    public void testGetNextIntervalCountConstantStatus(){
+        ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, true);
+        Long nextCount = connectionStatusAnalytics.getNextIntervalCount();
+        assertNotNull(nextCount);
+        assert(nextCount == 50L);
+    }
+
+    @Test
+    public void testGetNextIntervalCountVaryingStatus(){
+        ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, true);
+        Long nextCount = connectionStatusAnalytics.getNextIntervalCount();
+        assertNotNull(nextCount);
+        assert(nextCount == 50L);
+    }
+
+    @Test
+    public void testGetNextIntervalPercentageUseBytesConstantStatus(){
+        ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(50000L, 50L, "1MB", 100L, true);
+        Long nextBytesPercentage = connectionStatusAnalytics.getNextIntervalPercentageUseBytes();
+        assertNotNull(nextBytesPercentage);
+        assert(nextBytesPercentage == 5);
+    }
+
+    @Test
+    public void testGetNextIntervalPercentageUseCountConstantStatus(){
+        ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, true);
+        Long nextCountPercentage = connectionStatusAnalytics.getNextIntervalPercentageUseCount();
+        assertNotNull(nextCountPercentage);
+        assert(nextCountPercentage == 50);
+    }
+
+}


[nifi] 18/21: NIFI-6150 Fixed NaN check and refactored time prediction. Switched to use non caching engine for testing

Posted by ai...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

aichrist pushed a commit to branch analytics-framework
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 9cc3404bab7fe8ccf1c0a63fb0240db87466de58
Author: Yolanda Davis <yo...@gmail.com>
AuthorDate: Fri Aug 2 08:05:24 2019 -0400

    NIFI-6150 Fixed NaN check and refactored time prediction. Switched to use non caching engine for testing
---
 .../org/apache/nifi/controller/FlowController.java |  4 +-
 .../analytics/ConnectionStatusAnalytics.java       | 57 ++++++++++++----------
 2 files changed, 33 insertions(+), 28 deletions(-)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index 141a6a4..33ad7f7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -150,7 +150,7 @@ import org.apache.nifi.controller.service.StandardConfigurationContext;
 import org.apache.nifi.controller.service.StandardControllerServiceProvider;
 import org.apache.nifi.controller.state.manager.StandardStateManagerProvider;
 import org.apache.nifi.controller.state.server.ZooKeeperStateServer;
-import org.apache.nifi.controller.status.analytics.CachingConnectionStatusAnalyticsEngine;
+import org.apache.nifi.controller.status.analytics.ConnectionStatusAnalyticsEngine;
 import org.apache.nifi.controller.status.analytics.StatusAnalyticsEngine;
 import org.apache.nifi.controller.status.history.ComponentStatusRepository;
 import org.apache.nifi.controller.status.history.GarbageCollectionHistory;
@@ -594,7 +594,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
         }
 
         componentStatusRepository = createComponentStatusRepository();
-        analyticsEngine = new CachingConnectionStatusAnalyticsEngine(flowManager, componentStatusRepository);
+        analyticsEngine = new ConnectionStatusAnalyticsEngine(flowManager, componentStatusRepository);
         eventAccess = new StandardEventAccess(this, flowFileEventRepository);
 
         timerDrivenEngineRef.get().scheduleWithFixedDelay(new Runnable() {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
index 6b831fa..0cb6b5d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
@@ -73,10 +73,12 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
 
     public void refresh() {
 
-
         LOG.debug("Refreshing model with new data for connection id: {} ", connectionIdentifier);
-
-        this.queryWindow = new QueryWindow(System.currentTimeMillis() - getIntervalTimeMillis(), System.currentTimeMillis());
+        if (this.queryWindow == null) {
+            this.queryWindow = new QueryWindow(System.currentTimeMillis() - getIntervalTimeMillis(), System.currentTimeMillis());
+        } else if (supportsOnlineLearning()) {
+            this.queryWindow = new QueryWindow(queryWindow.getEndTimeMillis(), System.currentTimeMillis());
+        }
         modelMap.forEach((metric, modelFunction) -> {
 
             StatusAnalyticsModel model = modelFunction.getKey();
@@ -91,6 +93,18 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
         });
     }
 
+    protected Long getTimePrediction(Double prediction, Long timeMillis) {
+
+        if (Double.isNaN(prediction) || Double.isInfinite(prediction)) {
+            return -1L;
+        } else if (prediction < timeMillis) {
+            return 0L;
+        } else {
+            return Math.max(0, Math.round(prediction) - timeMillis);
+        }
+
+    }
+
     /**
      * Returns the predicted time (in milliseconds) when backpressure is expected to be applied to this connection, based on the total number of bytes in the queue.
      *
@@ -106,11 +120,7 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
         final String backPressureDataSize = connection.getFlowFileQueue().getBackPressureDataSizeThreshold();
         final double backPressureBytes = DataUnit.parseDataSize(backPressureDataSize, DataUnit.B);
         final double prediction = bytesModel.predictX(backPressureBytes);
-        if (prediction != Double.NaN && prediction >= System.currentTimeMillis() && !Double.isInfinite(prediction)) {
-            return Math.max(0, Math.round(prediction) - System.currentTimeMillis());
-        } else {
-            return -1L;
-        }
+        return getTimePrediction(prediction, System.currentTimeMillis());
 
     }
 
@@ -127,13 +137,8 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
             throw new NoSuchElementException("Connection with the following id cannot be found:" + connectionIdentifier + ". Model should be invalidated!");
         }
         final double backPressureCountThreshold = connection.getFlowFileQueue().getBackPressureObjectThreshold();
-        final Double prediction = countModel.predictX(backPressureCountThreshold);
-
-        if (prediction != Double.NaN && prediction >= System.currentTimeMillis() && !Double.isInfinite(prediction)) {
-            return Math.max(0, Math.round(prediction) - System.currentTimeMillis());
-        } else {
-            return -1L;
-        }
+        final double prediction = countModel.predictX(backPressureCountThreshold);
+        return getTimePrediction(prediction, System.currentTimeMillis());
     }
 
     /**
@@ -145,7 +150,7 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
     public Long getNextIntervalBytes() {
         final BivariateStatusAnalyticsModel bytesModel = (BivariateStatusAnalyticsModel) modelMap.get(ConnectionStatusDescriptor.QUEUED_BYTES.getField()).getKey();
         final Double prediction = bytesModel.predictY((double) System.currentTimeMillis() + getIntervalTimeMillis());
-        if (prediction != Double.NaN && prediction >= 0) {
+        if (!Double.isNaN(prediction) && prediction >= 0) {
             return Math.round(prediction);
         } else {
             return -1L;
@@ -161,14 +166,14 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
     public Long getNextIntervalCount() {
         final BivariateStatusAnalyticsModel countModel = (BivariateStatusAnalyticsModel) modelMap.get(ConnectionStatusDescriptor.QUEUED_COUNT.getField()).getKey();
         final Double prediction = countModel.predictY((double) System.currentTimeMillis() + getIntervalTimeMillis());
-        if (prediction != Double.NaN && prediction >= 0) {
+        if (!Double.isNaN(prediction) && prediction >= 0) {
             return Math.round(prediction);
         } else {
             return -1L;
         }
     }
 
-    public Long getNextIntervalPercentageUseCount(){
+    public Long getNextIntervalPercentageUseCount() {
 
         final Connection connection = getConnection();
         if (connection == null) {
@@ -177,15 +182,15 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
         final double backPressureCountThreshold = connection.getFlowFileQueue().getBackPressureObjectThreshold();
         final long nextIntervalCount = getNextIntervalCount();
 
-        if(nextIntervalCount > -1L) {
+        if (nextIntervalCount > -1L) {
             return Math.round((getNextIntervalCount() / backPressureCountThreshold) * 100);
-        }else{
+        } else {
             return -1L;
         }
 
     }
 
-    public Long getNextIntervalPercentageUseBytes(){
+    public Long getNextIntervalPercentageUseBytes() {
 
         final Connection connection = getConnection();
         if (connection == null) {
@@ -195,14 +200,14 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
         final double backPressureBytes = DataUnit.parseDataSize(backPressureDataSize, DataUnit.B);
         final long nextIntervalBytes = getNextIntervalBytes();
 
-        if(nextIntervalBytes > -1L) {
+        if (nextIntervalBytes > -1L) {
             return Math.round((getNextIntervalBytes() / backPressureBytes) * 100);
-        }else{
+        } else {
             return -1L;
         }
     }
 
-    public Long getIntervalTimeMillis(){
+    public Long getIntervalTimeMillis() {
         return (5L * 60 * 1000);
     }
 
@@ -226,8 +231,8 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
         predictions.put("nextIntervalPercentageUseBytes", getNextIntervalPercentageUseBytes());
         predictions.put("intervalTimeMillis", getIntervalTimeMillis());
 
-        predictions.forEach((key,value) -> {
-            LOG.debug("Prediction model for connection id {}: {}={} ", connectionIdentifier,key,value);
+        predictions.forEach((key, value) -> {
+            LOG.debug("Prediction model for connection id {}: {}={} ", connectionIdentifier, key, value);
         });
 
         return predictions;


[nifi] 20/21: NIFI-6510 Adjusted interval and incorporated R-squared check

Posted by ai...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

aichrist pushed a commit to branch analytics-framework
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 586185e4e5b72ffbb39dd4f6970df713a727818b
Author: Yolanda Davis <yo...@gmail.com>
AuthorDate: Thu Aug 8 19:54:46 2019 -0400

    NIFI-6510 Adjusted interval and incorporated R-squared check
    
    Updates to support multiple variables for features, clearing cached regression model based on r-squared values
    
    Added ordinary least squares model, which truly uses multivariable regression. Refactor of interfaces to include more general interface for variate models (that include scoring support).
    
    Ratcheck fixes
    
    Added test for SimpleRegression. Minor fix for OLS model
    
    fixed test errors
    
    fixed checkstyle errors
    
    (cherry picked from commit fab411b)
---
 .../status/analytics/StatusAnalyticsModel.java     |   4 +-
 .../org/apache/nifi/controller/FlowController.java |   4 +-
 .../CachingConnectionStatusAnalyticsEngine.java    |  17 +-
 .../analytics/ConnectionStatusAnalytics.java       | 190 ++++++++++++++++-----
 .../analytics/ConnectionStatusAnalyticsEngine.java |  12 +-
 .../BivariateStatusAnalyticsModel.java             |  22 ++-
 .../MultivariateStatusAnalyticsModel.java}         |  28 ++-
 .../analytics/models/OrdinaryLeastSquaresMSAM.java | 103 +++++++++++
 .../{ => models}/SimpleRegressionBSAM.java         |  56 ++++--
 .../models/VariateStatusAnalyticsModel.java        |  18 +-
 ...TestCachingConnectionStatusAnalyticsEngine.java |   7 +-
 .../analytics/TestConnectionStatusAnalytics.java   |  41 ++++-
 .../TestConnectionStatusAnalyticsEngine.java       |   5 +-
 .../analytics/TestStatusAnalyticsEngine.java       |   6 +-
 .../models/TestOrdinaryLeastSqauresMSAM.java       | 159 +++++++++++++++++
 .../analytics/models/TestSimpleRegressionBSAM.java | 102 +++++++++++
 16 files changed, 666 insertions(+), 108 deletions(-)

diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticsModel.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticsModel.java
index 72c81b1..4869a0f 100644
--- a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticsModel.java
+++ b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticsModel.java
@@ -20,8 +20,8 @@ import java.util.stream.Stream;
 
 public interface StatusAnalyticsModel {
 
-    void learn(Stream<Double> features, Stream<Double> labels);
-    Double predict(Double feature);
+    void learn(Stream<Double[]> features, Stream<Double> labels);
+    Double predict(Double[] feature);
     Boolean supportsOnlineLearning();
 
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index 33ad7f7..bf1d06c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -55,7 +55,6 @@ import org.apache.nifi.annotation.lifecycle.OnConfigurationRestored;
 import org.apache.nifi.annotation.lifecycle.OnShutdown;
 import org.apache.nifi.annotation.notification.OnPrimaryNodeStateChange;
 import org.apache.nifi.annotation.notification.PrimaryNodeState;
-import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.Resource;
 import org.apache.nifi.authorization.resource.Authorizable;
@@ -174,6 +173,7 @@ import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.nar.NarThreadContextClassLoader;
 import org.apache.nifi.parameter.ParameterContextManager;
+import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.parameter.StandardParameterContextManager;
 import org.apache.nifi.processor.Processor;
 import org.apache.nifi.processor.Relationship;
@@ -594,7 +594,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
         }
 
         componentStatusRepository = createComponentStatusRepository();
-        analyticsEngine = new ConnectionStatusAnalyticsEngine(flowManager, componentStatusRepository);
+        analyticsEngine = new ConnectionStatusAnalyticsEngine(flowManager, componentStatusRepository,flowFileEventRepository);
         eventAccess = new StandardEventAccess(this, flowFileEventRepository);
 
         timerDrivenEngineRef.get().scheduleWithFixedDelay(new Runnable() {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingConnectionStatusAnalyticsEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingConnectionStatusAnalyticsEngine.java
index f69ed33..747c496 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingConnectionStatusAnalyticsEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingConnectionStatusAnalyticsEngine.java
@@ -19,6 +19,7 @@ package org.apache.nifi.controller.status.analytics;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.nifi.controller.flow.FlowManager;
+import org.apache.nifi.controller.repository.FlowFileEventRepository;
 import org.apache.nifi.controller.status.history.ComponentStatusRepository;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -27,14 +28,16 @@ import com.github.benmanes.caffeine.cache.Cache;
 import com.github.benmanes.caffeine.cache.Caffeine;
 
 public class CachingConnectionStatusAnalyticsEngine implements StatusAnalyticsEngine {
-    private ComponentStatusRepository statusRepository;
-    private FlowManager flowManager;
+    private final ComponentStatusRepository statusRepository;
+    private final FlowManager flowManager;
+    private final FlowFileEventRepository flowFileEventRepository;
     private volatile Cache<String, ConnectionStatusAnalytics> cache;
     private static final Logger LOG = LoggerFactory.getLogger(CachingConnectionStatusAnalyticsEngine.class);
 
-    public CachingConnectionStatusAnalyticsEngine(FlowManager flowManager, ComponentStatusRepository statusRepository) {
+    public CachingConnectionStatusAnalyticsEngine(FlowManager flowManager, ComponentStatusRepository statusRepository, FlowFileEventRepository flowFileEventRepository) {
         this.flowManager = flowManager;
         this.statusRepository = statusRepository;
+        this.flowFileEventRepository = flowFileEventRepository;
         this.cache = Caffeine.newBuilder()
                 .expireAfterWrite(30, TimeUnit.MINUTES)
                 .build();
@@ -44,12 +47,12 @@ public class CachingConnectionStatusAnalyticsEngine implements StatusAnalyticsEn
     public StatusAnalytics getStatusAnalytics(String identifier) {
 
         ConnectionStatusAnalytics connectionStatusAnalytics = cache.getIfPresent(identifier);
-        if(connectionStatusAnalytics == null){
+        if (connectionStatusAnalytics == null) {
             LOG.debug("Creating new status analytics object for connection id: {}", identifier);
-            connectionStatusAnalytics = new ConnectionStatusAnalytics(statusRepository,flowManager,identifier,true);
+            connectionStatusAnalytics = new ConnectionStatusAnalytics(statusRepository, flowManager,flowFileEventRepository, identifier, true);
             connectionStatusAnalytics.init();
-            cache.put(identifier,connectionStatusAnalytics);
-        }else{
+            cache.put(identifier, connectionStatusAnalytics);
+        } else {
             LOG.debug("Pulled existing analytics from cache for connection id: {}", identifier);
             connectionStatusAnalytics.refresh();
         }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
index 0cb6b5d..aa19b1d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
@@ -22,10 +22,18 @@ import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.Optional;
+import java.util.Random;
 import java.util.stream.Stream;
 
+import org.apache.commons.lang3.ArrayUtils;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.controller.flow.FlowManager;
+import org.apache.nifi.controller.repository.FlowFileEvent;
+import org.apache.nifi.controller.repository.FlowFileEventRepository;
+import org.apache.nifi.controller.repository.RepositoryStatusReport;
+import org.apache.nifi.controller.status.analytics.models.MultivariateStatusAnalyticsModel;
+import org.apache.nifi.controller.status.analytics.models.OrdinaryLeastSquaresMSAM;
+import org.apache.nifi.controller.status.analytics.models.VariateStatusAnalyticsModel;
 import org.apache.nifi.controller.status.history.ComponentStatusRepository;
 import org.apache.nifi.controller.status.history.ConnectionStatusDescriptor;
 import org.apache.nifi.controller.status.history.StatusHistory;
@@ -38,19 +46,26 @@ import org.apache.nifi.web.api.dto.status.StatusSnapshotDTO;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.primitives.Doubles;
+
 public class ConnectionStatusAnalytics implements StatusAnalytics {
 
     private static final Logger LOG = LoggerFactory.getLogger(ConnectionStatusAnalytics.class);
     private Map<String, Tuple<StatusAnalyticsModel, ExtractFunction>> modelMap;
     private QueryWindow queryWindow;
     private final ComponentStatusRepository componentStatusRepository;
+    private final FlowFileEventRepository flowFileEventRepository;
     private final String connectionIdentifier;
     private final FlowManager flowManager;
     private final Boolean supportOnlineLearning;
+    private Boolean extendWindow = false;
+    private static double SCORE_THRESHOLD = .90;
 
-    public ConnectionStatusAnalytics(ComponentStatusRepository componentStatusRepository, FlowManager flowManager, String connectionIdentifier, Boolean supportOnlineLearning) {
+    public ConnectionStatusAnalytics(ComponentStatusRepository componentStatusRepository, FlowManager flowManager, FlowFileEventRepository flowFileEventRepository, String connectionIdentifier,
+                                     Boolean supportOnlineLearning) {
         this.componentStatusRepository = componentStatusRepository;
         this.flowManager = flowManager;
+        this.flowFileEventRepository = flowFileEventRepository;
         this.connectionIdentifier = connectionIdentifier;
         this.supportOnlineLearning = supportOnlineLearning;
     }
@@ -60,8 +75,8 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
         LOG.debug("Initialize analytics connection id: {} ", connectionIdentifier);
 
         if (this.modelMap == null || this.modelMap.isEmpty()) {
-            Tuple<StatusAnalyticsModel, ExtractFunction> countModelFunction = new Tuple<>(new SimpleRegressionBSAM(!supportsOnlineLearning()), extract);
-            Tuple<StatusAnalyticsModel, ExtractFunction> byteModelFunction = new Tuple<>(new SimpleRegressionBSAM(!supportsOnlineLearning()), extract);
+            Tuple<StatusAnalyticsModel, ExtractFunction> countModelFunction = new Tuple<>(new OrdinaryLeastSquaresMSAM(), extract);
+            Tuple<StatusAnalyticsModel, ExtractFunction> byteModelFunction = new Tuple<>(new OrdinaryLeastSquaresMSAM(), extract);
             this.modelMap = new HashMap<>();
             //TODO: Should change keys used here
             this.modelMap.put(ConnectionStatusDescriptor.QUEUED_COUNT.getField(), countModelFunction);
@@ -73,38 +88,39 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
 
     public void refresh() {
 
-        LOG.debug("Refreshing model with new data for connection id: {} ", connectionIdentifier);
         if (this.queryWindow == null) {
+            //Set query window to fresh value
             this.queryWindow = new QueryWindow(System.currentTimeMillis() - getIntervalTimeMillis(), System.currentTimeMillis());
-        } else if (supportsOnlineLearning()) {
-            this.queryWindow = new QueryWindow(queryWindow.getEndTimeMillis(), System.currentTimeMillis());
+        } else if (supportOnlineLearning) {
+            //Obtain latest observations when available, extend window if needed to obtain minimum observations
+            this.queryWindow = new QueryWindow(extendWindow ? queryWindow.getStartTimeMillis() : queryWindow.getEndTimeMillis(), System.currentTimeMillis());
+
         }
         modelMap.forEach((metric, modelFunction) -> {
 
             StatusAnalyticsModel model = modelFunction.getKey();
             ExtractFunction extract = modelFunction.getValue();
             StatusHistory statusHistory = componentStatusRepository.getConnectionStatusHistory(connectionIdentifier, queryWindow.getStartDateTime(), queryWindow.getEndDateTime(), Integer.MAX_VALUE);
-            Tuple<Stream<Double>, Stream<Double>> modelData = extract.extractMetric(metric, statusHistory);
-            Stream<Double> times = modelData.getKey();
-            Stream<Double> values = modelData.getValue();
-            //times are the X axis and values are on the y axis
-            model.learn(times, values);
+            Tuple<Stream<Double[]>, Stream<Double>> modelData = extract.extractMetric(metric, statusHistory);
+            Double[][] features = modelData.getKey().toArray(size -> new Double[size][1]);
+            Double[] values = modelData.getValue().toArray(size -> new Double[size]);
+
+            if (ArrayUtils.isNotEmpty(features)) {
+                try {
+                    LOG.debug("Refreshing model with new data for connection id: {} ", connectionIdentifier);
+                    model.learn(Stream.of(features), Stream.of(values));
+                    extendWindow = false;
+                } catch (Exception ex) {
+                    LOG.debug("Exception encountered while training model for connection id {}: {}", connectionIdentifier, ex.getMessage());
+                    extendWindow = true;
+                }
+            } else {
+                extendWindow = true;
+            }
 
         });
     }
 
-    protected Long getTimePrediction(Double prediction, Long timeMillis) {
-
-        if (Double.isNaN(prediction) || Double.isInfinite(prediction)) {
-            return -1L;
-        } else if (prediction < timeMillis) {
-            return 0L;
-        } else {
-            return Math.max(0, Math.round(prediction) - timeMillis);
-        }
-
-    }
-
     /**
      * Returns the predicted time (in milliseconds) when backpressure is expected to be applied to this connection, based on the total number of bytes in the queue.
      *
@@ -112,16 +128,24 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
      */
     public Long getTimeToBytesBackpressureMillis() {
 
-        final BivariateStatusAnalyticsModel bytesModel = (BivariateStatusAnalyticsModel) modelMap.get(ConnectionStatusDescriptor.QUEUED_BYTES.getField()).getKey();
+        final MultivariateStatusAnalyticsModel bytesModel = (MultivariateStatusAnalyticsModel) modelMap.get(ConnectionStatusDescriptor.QUEUED_BYTES.getField()).getKey();
+        FlowFileEvent flowFileEvent = getStatusReport();
+
         final Connection connection = getConnection();
         if (connection == null) {
             throw new NoSuchElementException("Connection with the following id cannot be found:" + connectionIdentifier + ". Model should be invalidated!");
         }
         final String backPressureDataSize = connection.getFlowFileQueue().getBackPressureDataSizeThreshold();
         final double backPressureBytes = DataUnit.parseDataSize(backPressureDataSize, DataUnit.B);
-        final double prediction = bytesModel.predictX(backPressureBytes);
-        return getTimePrediction(prediction, System.currentTimeMillis());
 
+        if(validModel(bytesModel) && flowFileEvent != null) {
+            List<Tuple<Integer, Double>> predictFeatures = new ArrayList<>();
+            Double inOutRatio = (flowFileEvent.getContentSizeOut() / (double)flowFileEvent.getContentSizeIn());
+            predictFeatures.add(new Tuple<>(1, inOutRatio));
+            return getTimePrediction(bytesModel.predictVariable(0, predictFeatures, backPressureBytes), System.currentTimeMillis());
+        }else{
+            return -1L;
+        }
     }
 
     /**
@@ -131,14 +155,24 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
      */
     public Long getTimeToCountBackpressureMillis() {
 
-        final BivariateStatusAnalyticsModel countModel = (BivariateStatusAnalyticsModel) modelMap.get(ConnectionStatusDescriptor.QUEUED_COUNT.getField()).getKey();
+        final MultivariateStatusAnalyticsModel countModel = (MultivariateStatusAnalyticsModel) modelMap.get(ConnectionStatusDescriptor.QUEUED_COUNT.getField()).getKey();
+        FlowFileEvent flowFileEvent = getStatusReport();
+
         final Connection connection = getConnection();
         if (connection == null) {
             throw new NoSuchElementException("Connection with the following id cannot be found:" + connectionIdentifier + ". Model should be invalidated!");
         }
+
         final double backPressureCountThreshold = connection.getFlowFileQueue().getBackPressureObjectThreshold();
-        final double prediction = countModel.predictX(backPressureCountThreshold);
-        return getTimePrediction(prediction, System.currentTimeMillis());
+
+        if(validModel(countModel) && flowFileEvent != null) {
+            List<Tuple<Integer, Double>> predictFeatures = new ArrayList<>();
+            Double inOutRatio = (flowFileEvent.getFlowFilesOut() / (double)flowFileEvent.getFlowFilesIn());
+            predictFeatures.add(new Tuple<>(1, inOutRatio));
+            return getTimePrediction(countModel.predictVariable(0, predictFeatures, backPressureCountThreshold), System.currentTimeMillis());
+        }else{
+            return -1L;
+        }
     }
 
     /**
@@ -148,11 +182,17 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
      */
 
     public Long getNextIntervalBytes() {
-        final BivariateStatusAnalyticsModel bytesModel = (BivariateStatusAnalyticsModel) modelMap.get(ConnectionStatusDescriptor.QUEUED_BYTES.getField()).getKey();
-        final Double prediction = bytesModel.predictY((double) System.currentTimeMillis() + getIntervalTimeMillis());
-        if (!Double.isNaN(prediction) && prediction >= 0) {
-            return Math.round(prediction);
-        } else {
+        final VariateStatusAnalyticsModel bytesModel = (VariateStatusAnalyticsModel) modelMap.get(ConnectionStatusDescriptor.QUEUED_BYTES.getField()).getKey();
+        FlowFileEvent flowFileEvent = getStatusReport();
+
+        if(validModel(bytesModel) && flowFileEvent != null) {
+            List<Double> predictFeatures = new ArrayList<>();
+            Long nextInterval =  System.currentTimeMillis() + getIntervalTimeMillis();
+            Double inOutRatio = flowFileEvent.getContentSizeOut() / (double)flowFileEvent.getContentSizeIn();
+            predictFeatures.add(nextInterval.doubleValue());
+            predictFeatures.add(inOutRatio);
+            return  (bytesModel.predict(predictFeatures.toArray(new Double[2]))).longValue();
+        }else{
             return -1L;
         }
     }
@@ -164,13 +204,20 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
      */
 
     public Long getNextIntervalCount() {
-        final BivariateStatusAnalyticsModel countModel = (BivariateStatusAnalyticsModel) modelMap.get(ConnectionStatusDescriptor.QUEUED_COUNT.getField()).getKey();
-        final Double prediction = countModel.predictY((double) System.currentTimeMillis() + getIntervalTimeMillis());
-        if (!Double.isNaN(prediction) && prediction >= 0) {
-            return Math.round(prediction);
-        } else {
+        final VariateStatusAnalyticsModel countModel = (VariateStatusAnalyticsModel) modelMap.get(ConnectionStatusDescriptor.QUEUED_COUNT.getField()).getKey();
+        FlowFileEvent flowFileEvent = getStatusReport();
+
+        if(validModel(countModel) && flowFileEvent != null) {
+            List<Double> predictFeatures = new ArrayList<>();
+            Long nextInterval =  System.currentTimeMillis() + getIntervalTimeMillis();
+            Double inOutRatio = flowFileEvent.getFlowFilesOut()/ (double)flowFileEvent.getFlowFilesIn();
+            predictFeatures.add(nextInterval.doubleValue());
+            predictFeatures.add(inOutRatio);
+            return (countModel.predict(predictFeatures.toArray(new Double[2]))).longValue();
+        }else{
             return -1L;
         }
+
     }
 
     public Long getNextIntervalPercentageUseCount() {
@@ -183,7 +230,7 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
         final long nextIntervalCount = getNextIntervalCount();
 
         if (nextIntervalCount > -1L) {
-            return Math.round((getNextIntervalCount() / backPressureCountThreshold) * 100);
+            return Math.min(100, Math.round((nextIntervalCount / backPressureCountThreshold) * 100));
         } else {
             return -1L;
         }
@@ -201,14 +248,14 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
         final long nextIntervalBytes = getNextIntervalBytes();
 
         if (nextIntervalBytes > -1L) {
-            return Math.round((getNextIntervalBytes() / backPressureBytes) * 100);
+            return Math.min(100, Math.round((getNextIntervalBytes() / backPressureBytes) * 100));
         } else {
             return -1L;
         }
     }
 
     public Long getIntervalTimeMillis() {
-        return (5L * 60 * 1000);
+        return 3L * 60 * 1000;
     }
 
     @Override
@@ -249,24 +296,73 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
         return connection.orElse(null);
     }
 
+    private FlowFileEvent getStatusReport(){
+        RepositoryStatusReport statusReport =  flowFileEventRepository.reportTransferEvents(System.currentTimeMillis());
+        return statusReport.getReportEntry(this.connectionIdentifier);
+    }
+
     private interface ExtractFunction {
-        Tuple<Stream<Double>, Stream<Double>> extractMetric(String metric, StatusHistory statusHistory);
+        Tuple<Stream<Double[]>, Stream<Double>> extractMetric(String metric, StatusHistory statusHistory);
+    }
+
+    private Long getTimePrediction(Double prediction, Long timeMillis) {
+
+        if (Double.isNaN(prediction) || Double.isInfinite(prediction)) {
+            return -1L;
+        } else if (prediction < timeMillis) {
+            return 0L;
+        } else {
+            return Math.max(0, Math.round(prediction) - timeMillis);
+        }
+    }
+
+    private boolean validModel(VariateStatusAnalyticsModel model){
+
+        Double rSquared = model.getRSquared();
+
+        if (rSquared == null || (Doubles.isFinite(rSquared) && !Double.isNaN(rSquared) && rSquared < SCORE_THRESHOLD)) {
+            if(supportOnlineLearning && model.supportsOnlineLearning()){
+                model.clear();
+            }
+            return false;
+        }else {
+            return true;
+        }
     }
 
     private final ExtractFunction extract = (metric, statusHistory) -> {
 
         List<Double> values = new ArrayList<>();
-        List<Double> times = new ArrayList<>();
-
+        List<Double[]> features = new ArrayList<>();
+        Random rand = new Random();
         StatusHistoryDTO statusHistoryDTO = StatusHistoryUtil.createStatusHistoryDTO(statusHistory);
 
         for (StatusSnapshotDTO snap : statusHistoryDTO.getAggregateSnapshots()) {
+            List<Double> featureArray = new ArrayList<>();
             Long snapValue = snap.getStatusMetrics().get(metric);
             long snapTime = snap.getTimestamp().getTime();
+
+            featureArray.add((double) snapTime);
+            Double randomError = + (rand.nextInt(1000) * .0000001);
+            if (metric.equals(ConnectionStatusDescriptor.QUEUED_COUNT.getField())) {
+
+                Long inputCount = snap.getStatusMetrics().get(ConnectionStatusDescriptor.INPUT_COUNT.getField());
+                Long outputCount = snap.getStatusMetrics().get(ConnectionStatusDescriptor.OUTPUT_COUNT.getField());
+                Double inOutRatio = ((double)outputCount /(double)inputCount) + randomError;
+                featureArray.add(Double.isNaN(inOutRatio)? randomError : inOutRatio);
+
+            } else {
+                Long inputBytes = snap.getStatusMetrics().get(ConnectionStatusDescriptor.INPUT_BYTES.getField());
+                Long outputBytes = snap.getStatusMetrics().get(ConnectionStatusDescriptor.OUTPUT_BYTES.getField());
+                Double inOutRatio = ((double)outputBytes/(double)inputBytes) + randomError;
+                featureArray.add(Double.isNaN(inOutRatio)? randomError : inOutRatio);
+            }
+
             values.add((double) snapValue);
-            times.add((double) snapTime);
+            features.add(featureArray.toArray(new Double[featureArray.size()]));
+
         }
-        return new Tuple<>(times.stream(), values.stream());
+        return new Tuple<Stream<Double[]>, Stream<Double>>(features.stream(), values.stream());
 
     };
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalyticsEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalyticsEngine.java
index 8f1222c..6f261a0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalyticsEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalyticsEngine.java
@@ -17,24 +17,26 @@
 package org.apache.nifi.controller.status.analytics;
 
 import org.apache.nifi.controller.flow.FlowManager;
+import org.apache.nifi.controller.repository.FlowFileEventRepository;
 import org.apache.nifi.controller.status.history.ComponentStatusRepository;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class ConnectionStatusAnalyticsEngine implements StatusAnalyticsEngine {
-    private ComponentStatusRepository statusRepository;
-    private FlowManager flowManager;
-
     private static final Logger LOG = LoggerFactory.getLogger(ConnectionStatusAnalyticsEngine.class);
+    private final ComponentStatusRepository statusRepository;
+    private final FlowManager flowManager;
+    private final FlowFileEventRepository flowFileEventRepository;
 
-    public ConnectionStatusAnalyticsEngine(FlowManager flowManager, ComponentStatusRepository statusRepository) {
+    public ConnectionStatusAnalyticsEngine(FlowManager flowManager, ComponentStatusRepository statusRepository, FlowFileEventRepository flowFileEventRepository) {
         this.flowManager = flowManager;
         this.statusRepository = statusRepository;
+        this.flowFileEventRepository = flowFileEventRepository;
     }
 
     @Override
     public StatusAnalytics getStatusAnalytics(String identifier) {
-        ConnectionStatusAnalytics connectionStatusAnalytics = new ConnectionStatusAnalytics(statusRepository,flowManager,identifier,false);
+        ConnectionStatusAnalytics connectionStatusAnalytics = new ConnectionStatusAnalytics(statusRepository, flowManager, flowFileEventRepository, identifier, false);
         connectionStatusAnalytics.init();
         return connectionStatusAnalytics;
     }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/BivariateStatusAnalyticsModel.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/models/BivariateStatusAnalyticsModel.java
similarity index 68%
copy from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/BivariateStatusAnalyticsModel.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/models/BivariateStatusAnalyticsModel.java
index eff661b..a1e2729 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/BivariateStatusAnalyticsModel.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/models/BivariateStatusAnalyticsModel.java
@@ -14,25 +14,33 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.controller.status.analytics;
+package org.apache.nifi.controller.status.analytics.models;
 
+import java.util.Map;
 import java.util.stream.Stream;
 
-public abstract class BivariateStatusAnalyticsModel implements StatusAnalyticsModel {
+public abstract class BivariateStatusAnalyticsModel implements VariateStatusAnalyticsModel {
 
+    public abstract void learn(Stream<Double[]> features, Stream<Double> labels);
 
-    public abstract void learn(Stream<Double> features, Stream<Double> labels);
+    public  Double predict(Double[] feature){
+        return predictY(feature[0]);
+    }
 
-    public abstract Double predict(Double feature);
+    public Boolean supportsOnlineLearning() {
+        return false;
+    }
 
     public abstract Double predictX(Double y);
 
     public abstract Double predictY(Double x);
 
+    public abstract Double getRSquared();
+
+    public abstract Map<String,Double> getScores();
+
     @Override
-    public Boolean supportsOnlineLearning() {
-        return false;
+    public void clear() {
     }
 
-
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/BivariateStatusAnalyticsModel.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/models/MultivariateStatusAnalyticsModel.java
similarity index 58%
rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/BivariateStatusAnalyticsModel.java
rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/models/MultivariateStatusAnalyticsModel.java
index eff661b..f4a98ab 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/BivariateStatusAnalyticsModel.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/models/MultivariateStatusAnalyticsModel.java
@@ -14,25 +14,37 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.controller.status.analytics;
 
-import java.util.stream.Stream;
-
-public abstract class BivariateStatusAnalyticsModel implements StatusAnalyticsModel {
+package org.apache.nifi.controller.status.analytics.models;
 
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Stream;
 
-    public abstract void learn(Stream<Double> features, Stream<Double> labels);
+import org.apache.nifi.util.Tuple;
 
-    public abstract Double predict(Double feature);
+public abstract class MultivariateStatusAnalyticsModel implements VariateStatusAnalyticsModel {
 
-    public abstract Double predictX(Double y);
+    public abstract void learn(Stream<Double[]> features, Stream<Double> labels);
 
-    public abstract Double predictY(Double x);
+    public abstract Double predict(Double[] feature);
 
     @Override
     public Boolean supportsOnlineLearning() {
         return false;
     }
 
+    public abstract Double predictVariable(Integer variableIndex, List<Tuple<Integer,Double>> predictorVariablesWithIndex, Double label);
+
+    public abstract Double getRSquared();
+
+    public abstract Map<String,Double> getScores();
+
+    @Override
+    public void clear() {
+
+    }
 
 }
+
+
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/models/OrdinaryLeastSquaresMSAM.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/models/OrdinaryLeastSquaresMSAM.java
new file mode 100644
index 0000000..4340a8c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/models/OrdinaryLeastSquaresMSAM.java
@@ -0,0 +1,103 @@
+/*
+ * 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.analytics.models;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.commons.math3.stat.regression.OLSMultipleLinearRegression;
+import org.apache.nifi.util.Tuple;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OrdinaryLeastSquaresMSAM extends MultivariateStatusAnalyticsModel {
+
+    private static final Logger LOG = LoggerFactory.getLogger(OrdinaryLeastSquaresMSAM.class);
+    private OLSMultipleLinearRegression olsModel;
+    private double[] coefficients;
+
+    public OrdinaryLeastSquaresMSAM() {
+        this.olsModel = new OLSMultipleLinearRegression();
+    }
+
+    @Override
+    public void learn(Stream<Double[]> features, Stream<Double> labels) {
+        double[] labelArray = ArrayUtils.toPrimitive(labels.toArray(Double[]::new));
+        double[][] featuresMatrix = features.map(feature -> ArrayUtils.toPrimitive(feature)).toArray(double[][]::new);
+        this.olsModel.newSampleData(labelArray, featuresMatrix);
+        this.coefficients = olsModel.estimateRegressionParameters();
+    }
+
+    @Override
+    public Double predict(Double[] feature) {
+        if (coefficients != null) {
+            final double intercept = olsModel.isNoIntercept() ? 0 : coefficients[0];
+            double sumX = 0;
+
+            for (int i = 0; i < feature.length; i++) {
+                sumX += coefficients[i + 1] * feature[i];
+            }
+
+            return sumX + intercept;
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public Double predictVariable(Integer variableIndex, List<Tuple<Integer, Double>> predictorVariablesWithIndex, Double label) {
+        if (coefficients != null) {
+            final double intercept = olsModel.isNoIntercept() ? 0 : coefficients[0];
+            final double predictorCoeff = coefficients[variableIndex + 1];
+            double sumX = 0;
+            if (predictorVariablesWithIndex.size() > 0) {
+                sumX = predictorVariablesWithIndex.stream().map(featureTuple -> coefficients[olsModel.isNoIntercept() ? featureTuple.getKey() : featureTuple.getKey() + 1] * featureTuple.getValue())
+                                                           .collect(Collectors.summingDouble(Double::doubleValue));
+            }
+            return (label - intercept - sumX) / predictorCoeff;
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public Map<String, Double> getScores() {
+        if (coefficients != null) {
+            Map<String, Double> scores = new HashMap<>();
+            scores.put("rSquared", olsModel.calculateRSquared());
+            scores.put("totalSumOfSquares", olsModel.calculateTotalSumOfSquares());
+            return scores;
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public Double getRSquared() {
+        if (coefficients != null) {
+            return olsModel.calculateRSquared();
+        } else {
+            return null;
+        }
+    }
+
+
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/SimpleRegressionBSAM.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/models/SimpleRegressionBSAM.java
similarity index 54%
rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/SimpleRegressionBSAM.java
rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/models/SimpleRegressionBSAM.java
index 21c0370..39ffce3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/SimpleRegressionBSAM.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/models/SimpleRegressionBSAM.java
@@ -14,12 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.controller.status.analytics;
+package org.apache.nifi.controller.status.analytics.models;
 
 
+import java.util.HashMap;
+import java.util.Map;
 import java.util.stream.Stream;
 
 import org.apache.commons.lang3.ArrayUtils;
+import org.apache.commons.math3.stat.regression.RegressionResults;
 import org.apache.commons.math3.stat.regression.SimpleRegression;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -29,6 +32,7 @@ public class SimpleRegressionBSAM extends BivariateStatusAnalyticsModel {
     private static final Logger LOG = LoggerFactory.getLogger(SimpleRegressionBSAM.class);
     private final SimpleRegression regression;
     private final Boolean clearObservationsOnLearn;
+    private RegressionResults results;
 
     public SimpleRegressionBSAM(Boolean clearObservationsOnLearn) {
 
@@ -37,25 +41,22 @@ public class SimpleRegressionBSAM extends BivariateStatusAnalyticsModel {
     }
 
     @Override
-    public void learn(Stream<Double> features, Stream<Double> labels) {
+    public void learn(Stream<Double[]> features, Stream<Double> labels) {
         double[] labelArray = ArrayUtils.toPrimitive(labels.toArray(Double[]::new));
-        double[][] featuresMatrix = features.map(feature -> new double[]{feature}).toArray(double[][]::new);
+        double[][] featuresMatrix = features.map(feature -> ArrayUtils.toPrimitive(feature)).toArray(double[][]::new);
 
-        if(clearObservationsOnLearn) {
+        if (clearObservationsOnLearn) {
             regression.clear();
         }
 
         regression.addObservations(featuresMatrix, labelArray);
-        LOG.debug("Model is using equation: y = {}x + {}", regression.getSlope(), regression.getIntercept());
+        results = regression.regress();
+        LOG.debug("Model is using equation: y = {}x + {}, with R-squared {}, RMSE {}", regression.getSlope(), regression.getIntercept(),
+                                                                                       results.getRSquared(), Math.sqrt(results.getMeanSquareError()));
 
     }
 
     @Override
-    public Double predict(Double feature) {
-        return predictY(feature);
-    }
-
-    @Override
     public Double predictX(Double y) {
         return (y - regression.getIntercept()) / regression.getSlope();
     }
@@ -64,4 +65,39 @@ public class SimpleRegressionBSAM extends BivariateStatusAnalyticsModel {
     public Double predictY(Double x) {
         return regression.getSlope() * x + regression.getIntercept();
     }
+
+    @Override
+    public Double getRSquared() {
+        if (results != null) {
+            return results.getRSquared();
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public Map<String, Double> getScores() {
+        if(results == null){
+            return null;
+        }else{
+            Map<String,Double> scores = new HashMap<>();
+            scores.put("rSquared",results.getRSquared());
+            scores.put("adjustedRSquared",results.getAdjustedRSquared());
+            scores.put("residualSumSquares",results.getErrorSumSquares());
+            scores.put("meanSquareError",results.getMeanSquareError());
+            return scores;
+        }
+    }
+
+    @Override
+    public void clear() {
+        results = null;
+        regression.clear();
+    }
+
+    @Override
+    public Boolean supportsOnlineLearning() {
+        return true;
+    }
+
 }
diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticsModel.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/models/VariateStatusAnalyticsModel.java
similarity index 67%
copy from nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticsModel.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/models/VariateStatusAnalyticsModel.java
index 72c81b1..44ce6b5 100644
--- a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticsModel.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/models/VariateStatusAnalyticsModel.java
@@ -14,14 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.controller.status.analytics;
+package org.apache.nifi.controller.status.analytics.models;
 
+import java.util.Map;
 import java.util.stream.Stream;
 
-public interface StatusAnalyticsModel {
+import org.apache.nifi.controller.status.analytics.StatusAnalyticsModel;
+
+public interface VariateStatusAnalyticsModel extends StatusAnalyticsModel {
+
+    void learn(Stream<Double[]> features, Stream<Double> labels);
+
+    Double predict(Double[] feature);
 
-    void learn(Stream<Double> features, Stream<Double> labels);
-    Double predict(Double feature);
     Boolean supportsOnlineLearning();
 
+    Double getRSquared();
+
+    Map<String,Double> getScores();
+
+    void clear();
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestCachingConnectionStatusAnalyticsEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestCachingConnectionStatusAnalyticsEngine.java
index 9fb67db..77ffa9b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestCachingConnectionStatusAnalyticsEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestCachingConnectionStatusAnalyticsEngine.java
@@ -17,19 +17,20 @@
 package org.apache.nifi.controller.status.analytics;
 
 import org.apache.nifi.controller.flow.FlowManager;
+import org.apache.nifi.controller.repository.FlowFileEventRepository;
 import org.apache.nifi.controller.status.history.ComponentStatusRepository;
 import org.junit.Test;
 
 public class TestCachingConnectionStatusAnalyticsEngine extends TestStatusAnalyticsEngine {
 
     @Override
-    public StatusAnalyticsEngine getStatusAnalyticsEngine(FlowManager flowManager, ComponentStatusRepository componentStatusRepository) {
-        return new CachingConnectionStatusAnalyticsEngine(flowManager, componentStatusRepository);
+    public StatusAnalyticsEngine getStatusAnalyticsEngine(FlowManager flowManager, FlowFileEventRepository flowFileEventRepository, ComponentStatusRepository componentStatusRepository) {
+        return new CachingConnectionStatusAnalyticsEngine(flowManager, componentStatusRepository, flowFileEventRepository);
     }
 
     @Test
     public void testCachedStatusAnalytics() {
-        StatusAnalyticsEngine statusAnalyticsEngine = new CachingConnectionStatusAnalyticsEngine(flowManager, statusRepository);
+        StatusAnalyticsEngine statusAnalyticsEngine = new CachingConnectionStatusAnalyticsEngine(flowManager, statusRepository, flowFileEventRepository);
         StatusAnalytics statusAnalyticsA = statusAnalyticsEngine.getStatusAnalytics("A");
         StatusAnalytics statusAnalyticsB = statusAnalyticsEngine.getStatusAnalytics("B");
         StatusAnalytics statusAnalyticsTest = statusAnalyticsEngine.getStatusAnalytics("A");
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalytics.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalytics.java
index 1f5d520..b013ed4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalytics.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalytics.java
@@ -19,6 +19,7 @@ package org.apache.nifi.controller.status.analytics;
 import static org.junit.Assert.assertNotNull;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.anyLong;
 import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.Mockito.when;
 
@@ -32,6 +33,9 @@ import java.util.stream.Collectors;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.controller.flow.FlowManager;
 import org.apache.nifi.controller.queue.FlowFileQueue;
+import org.apache.nifi.controller.repository.FlowFileEvent;
+import org.apache.nifi.controller.repository.FlowFileEventRepository;
+import org.apache.nifi.controller.repository.RepositoryStatusReport;
 import org.apache.nifi.controller.status.history.ComponentStatusRepository;
 import org.apache.nifi.controller.status.history.ConnectionStatusDescriptor;
 import org.apache.nifi.controller.status.history.MetricDescriptor;
@@ -57,6 +61,9 @@ public class TestConnectionStatusAnalytics {
         final StatusHistory statusHistory = Mockito.mock(StatusHistory.class);
         final Connection connection = Mockito.mock(Connection.class);
         final FlowFileQueue flowFileQueue = Mockito.mock(FlowFileQueue.class);
+        final FlowFileEventRepository flowFileEventRepository = Mockito.mock(FlowFileEventRepository.class);
+        final RepositoryStatusReport repositoryStatusReport = Mockito.mock(RepositoryStatusReport.class);
+        final FlowFileEvent flowFileEvent = Mockito.mock(FlowFileEvent.class);
         final List<Connection> connections = new ArrayList<>();
         final String connectionIdentifier = "1";
         connections.add(connection);
@@ -65,11 +72,20 @@ public class TestConnectionStatusAnalytics {
         final long startTime = System.currentTimeMillis();
         int iterations = 10;
 
+        Long inputBytes = queuedBytes * 2;
+        Long outputBytes = inputBytes - queuedBytes;
+        Long inputCount = queuedCount * 2;
+        Long outputCount = inputCount - queuedCount;
+
         for (int i = 0; i < iterations; i++) {
             final StandardStatusSnapshot snapshot = new StandardStatusSnapshot(CONNECTION_METRICS);
             snapshot.setTimestamp(new Date(startTime + i * 1000));
             snapshot.addStatusMetric(ConnectionStatusDescriptor.QUEUED_BYTES.getDescriptor(), (isConstantStatus || i < 5) ? queuedBytes : queuedBytes * 2);
             snapshot.addStatusMetric(ConnectionStatusDescriptor.QUEUED_COUNT.getDescriptor(), (isConstantStatus || i < 5) ? queuedCount : queuedCount * 2);
+            snapshot.addStatusMetric(ConnectionStatusDescriptor.INPUT_BYTES.getDescriptor(), (isConstantStatus || i < 5) ? inputBytes : inputBytes * 2);
+            snapshot.addStatusMetric(ConnectionStatusDescriptor.INPUT_COUNT.getDescriptor(), (isConstantStatus || i < 5) ? inputCount : inputCount * 2);
+            snapshot.addStatusMetric(ConnectionStatusDescriptor.OUTPUT_BYTES.getDescriptor(), (isConstantStatus || i < 5) ? outputBytes : outputBytes * 2);
+            snapshot.addStatusMetric(ConnectionStatusDescriptor.OUTPUT_COUNT.getDescriptor(), (isConstantStatus || i < 5) ? outputCount : outputCount * 2);
             snapshotList.add(snapshot);
         }
 
@@ -80,8 +96,14 @@ public class TestConnectionStatusAnalytics {
         when(processGroup.findAllConnections()).thenReturn(connections);
         when(statusHistory.getStatusSnapshots()).thenReturn(snapshotList);
         when(flowManager.getRootGroup()).thenReturn(processGroup);
+        when(flowFileEvent.getContentSizeIn()).thenReturn(inputBytes);
+        when(flowFileEvent.getContentSizeOut()).thenReturn(outputBytes);
+        when(flowFileEvent.getFlowFilesIn()).thenReturn(inputCount.intValue());
+        when(flowFileEvent.getFlowFilesOut()).thenReturn(outputCount.intValue());
+        when(flowFileEventRepository.reportTransferEvents(anyLong())).thenReturn(repositoryStatusReport);
+        when(repositoryStatusReport.getReportEntry(anyString())).thenReturn(flowFileEvent);
         when(statusRepository.getConnectionStatusHistory(anyString(), any(), any(), anyInt())).thenReturn(statusHistory);
-        ConnectionStatusAnalytics connectionStatusAnalytics = new ConnectionStatusAnalytics(statusRepository, flowManager, connectionIdentifier, false);
+        ConnectionStatusAnalytics connectionStatusAnalytics = new ConnectionStatusAnalytics(statusRepository, flowManager,flowFileEventRepository, connectionIdentifier, false);
         connectionStatusAnalytics.init();
         return connectionStatusAnalytics;
     }
@@ -91,7 +113,7 @@ public class TestConnectionStatusAnalytics {
         ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, true);
         Long interval = connectionStatusAnalytics.getIntervalTimeMillis();
         assertNotNull(interval);
-        assert (interval == 300000);
+        assert (interval == 180000);
     }
 
     @Test
@@ -99,7 +121,7 @@ public class TestConnectionStatusAnalytics {
         ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, true);
         Long countTime = connectionStatusAnalytics.getTimeToCountBackpressureMillis();
         assertNotNull(countTime);
-        assert (countTime == -1L);
+        assert (countTime > 0);
     }
 
     @Test
@@ -107,7 +129,7 @@ public class TestConnectionStatusAnalytics {
         ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, false);
         Long countTime = connectionStatusAnalytics.getTimeToCountBackpressureMillis();
         assertNotNull(countTime);
-        assert (countTime > -1L);
+        assert (countTime == -1L);
     }
 
     @Test
@@ -115,7 +137,7 @@ public class TestConnectionStatusAnalytics {
         ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, true);
         Long bytesTime = connectionStatusAnalytics.getTimeToBytesBackpressureMillis();
         assertNotNull(bytesTime);
-        assert (bytesTime == -1L);
+        assert (bytesTime == -1L || bytesTime == 0);
     }
 
     @Test
@@ -123,7 +145,7 @@ public class TestConnectionStatusAnalytics {
         ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, false);
         Long bytesTime = connectionStatusAnalytics.getTimeToBytesBackpressureMillis();
         assertNotNull(bytesTime);
-        assert (bytesTime > -1L);
+        assert (bytesTime == -1L);
     }
 
     @Test
@@ -139,7 +161,7 @@ public class TestConnectionStatusAnalytics {
         ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, false);
         Long nextBytes = connectionStatusAnalytics.getNextIntervalBytes();
         assertNotNull(nextBytes);
-        assert (nextBytes > -1L);
+        assert (nextBytes == -1L);
     }
 
     @Test
@@ -152,10 +174,11 @@ public class TestConnectionStatusAnalytics {
 
     @Test
     public void testGetNextIntervalCountVaryingStatus() {
-        ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, true);
+
+        ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, false);
         Long nextCount = connectionStatusAnalytics.getNextIntervalCount();
         assertNotNull(nextCount);
-        assert (nextCount == 50L);
+        assert (nextCount == -1L);
     }
 
     @Test
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalyticsEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalyticsEngine.java
index 2005e04..99c9fae 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalyticsEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalyticsEngine.java
@@ -17,13 +17,14 @@
 package org.apache.nifi.controller.status.analytics;
 
 import org.apache.nifi.controller.flow.FlowManager;
+import org.apache.nifi.controller.repository.FlowFileEventRepository;
 import org.apache.nifi.controller.status.history.ComponentStatusRepository;
 
 public class TestConnectionStatusAnalyticsEngine extends TestStatusAnalyticsEngine {
 
     @Override
-    public StatusAnalyticsEngine getStatusAnalyticsEngine(FlowManager flowManager, ComponentStatusRepository statusRepository) {
-        return new ConnectionStatusAnalyticsEngine(flowManager, statusRepository);
+    public StatusAnalyticsEngine getStatusAnalyticsEngine(FlowManager flowManager, FlowFileEventRepository flowFileEventRepository, ComponentStatusRepository statusRepository) {
+        return new ConnectionStatusAnalyticsEngine(flowManager, statusRepository, flowFileEventRepository);
     }
 
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestStatusAnalyticsEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestStatusAnalyticsEngine.java
index 183cd27..ee3d4e5 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestStatusAnalyticsEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestStatusAnalyticsEngine.java
@@ -25,6 +25,7 @@ import static org.mockito.Mockito.when;
 import java.util.Collections;
 
 import org.apache.nifi.controller.flow.FlowManager;
+import org.apache.nifi.controller.repository.FlowFileEventRepository;
 import org.apache.nifi.controller.status.history.ComponentStatusRepository;
 import org.apache.nifi.controller.status.history.StatusHistory;
 import org.apache.nifi.controller.status.history.StatusSnapshot;
@@ -37,6 +38,7 @@ public abstract class TestStatusAnalyticsEngine {
 
     protected ComponentStatusRepository statusRepository;
     protected FlowManager flowManager;
+    protected FlowFileEventRepository flowFileEventRepository;
 
     @Before
     public void setup() {
@@ -53,11 +55,11 @@ public abstract class TestStatusAnalyticsEngine {
 
     @Test
     public void testGetStatusAnalytics() {
-        StatusAnalyticsEngine statusAnalyticsEngine = getStatusAnalyticsEngine(flowManager, statusRepository);
+        StatusAnalyticsEngine statusAnalyticsEngine = getStatusAnalyticsEngine(flowManager,flowFileEventRepository, statusRepository);
         StatusAnalytics statusAnalytics = statusAnalyticsEngine.getStatusAnalytics("1");
         assertNotNull(statusAnalytics);
     }
 
-    public abstract StatusAnalyticsEngine getStatusAnalyticsEngine(FlowManager flowManager, ComponentStatusRepository componentStatusRepository);
+    public abstract StatusAnalyticsEngine getStatusAnalyticsEngine(FlowManager flowManager, FlowFileEventRepository flowFileEventRepository, ComponentStatusRepository componentStatusRepository);
 
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/models/TestOrdinaryLeastSqauresMSAM.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/models/TestOrdinaryLeastSqauresMSAM.java
new file mode 100644
index 0000000..d4775c2
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/models/TestOrdinaryLeastSqauresMSAM.java
@@ -0,0 +1,159 @@
+/*
+ * 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.analytics.models;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Stream;
+
+import org.apache.commons.lang3.time.DateUtils;
+import org.apache.commons.math3.linear.SingularMatrixException;
+import org.apache.nifi.util.Tuple;
+import org.junit.Test;
+
+public class TestOrdinaryLeastSqauresMSAM {
+
+
+    @Test
+    public void testConstantPrediction(){
+
+        Double timestamp = 1565444720000.0;
+        Double inputCount = 1000.0;
+        Double outputCount = 1000.0;
+        Double queueCount = 50.0;
+
+        Double[] feature0 = {timestamp - 1000,outputCount/inputCount};
+        Double[] feature1 = {timestamp,outputCount/inputCount};
+        Double[] feature2 = {timestamp + 1000,outputCount/inputCount};
+        Double[] feature3 = {timestamp + 2000,outputCount/inputCount};
+
+        Double[][] features = {feature0, feature1,feature2,feature3};
+        Double[] labels = {queueCount,queueCount,queueCount, queueCount};
+
+        OrdinaryLeastSquaresMSAM model = new OrdinaryLeastSquaresMSAM();
+        boolean exOccurred = false;
+        try {
+            model.learn(Stream.of(features), Stream.of(labels));
+        } catch (SingularMatrixException sme){
+            exOccurred = true;
+        }
+        assertTrue(exOccurred);
+
+    }
+
+    @Test
+    public void testVaryingPredictionOfVariable(){
+
+        Double timestamp = 1565444720000.0;
+        Double inputCount = 1000.0;
+        Double outputCount = 50.0;
+        Double queueCount = 950.0;
+
+        Double[] feature0 = {timestamp,outputCount/inputCount};
+        Double[] feature1 = {timestamp + 1000,outputCount/(inputCount + 50)};
+        Double[] feature2 = {timestamp + 2000,(outputCount + 50)/(inputCount)};
+        Double[] feature3 = {timestamp + 3000,(outputCount + 100)/(inputCount - 100)};
+
+        Double[][] features = {feature0, feature1,feature2,feature3};
+        Double[] labels = {queueCount,queueCount + 50, queueCount - 50, queueCount - 100};
+
+        OrdinaryLeastSquaresMSAM model = new OrdinaryLeastSquaresMSAM();
+
+        model.learn(Stream.of(features), Stream.of(labels));
+
+        Tuple<Integer,Double> ratioPredictor = new Tuple<>(1,200/800.0);
+        List<Tuple<Integer,Double>> predictorVars = new ArrayList<>();
+        predictorVars.add(ratioPredictor);
+        Double target = model.predictVariable(0,predictorVars, 750.0);
+        Double rSquared = model.getRSquared();
+        assert(rSquared > .90);
+        Date targetDate = new Date(target.longValue());
+        Date testDate = new Date(timestamp.longValue());
+        assert(DateUtils.isSameDay(targetDate,testDate) && targetDate.after(testDate));
+
+    }
+
+    @Test
+    public void testVaryingPrediction(){
+
+        Double timestamp = 1565444720000.0;
+        Double inputCount = 1000.0;
+        Double outputCount = 50.0;
+        Double queueCount = 950.0;
+
+        Double[] feature0 = {timestamp,outputCount/inputCount};
+        Double[] feature1 = {timestamp + 1000,outputCount/(inputCount + 50)};
+        Double[] feature2 = {timestamp + 2000,(outputCount + 50)/(inputCount)};
+        Double[] feature3 = {timestamp + 3000,(outputCount + 100)/(inputCount - 100)};
+
+        Double[][] features = {feature0, feature1,feature2,feature3};
+        Double[] labels = {queueCount,queueCount + 50, queueCount - 50, queueCount - 100};
+
+
+        OrdinaryLeastSquaresMSAM model = new OrdinaryLeastSquaresMSAM();
+
+        Double[] predictor = {timestamp + 5000, outputCount/inputCount};
+
+        model.learn(Stream.of(features), Stream.of(labels));
+        Double target = model.predict(predictor);
+        Double rSquared = model.getRSquared();
+        assert(rSquared > .90);
+        assert(target >= 950);
+
+    }
+
+    @Test
+    public void comparePredictions(){
+
+        Double timestamp = 1565444720000.0;
+        Double inputCount = 1000.0;
+        Double outputCount = 50.0;
+        Double queueCount = 950.0;
+
+        Double[] feature0 = {timestamp,outputCount/inputCount};
+        Double[] feature1 = {timestamp + 1000,outputCount/(inputCount + 50)};
+        Double[] feature2 = {timestamp + 2000,(outputCount + 50)/(inputCount)};
+        Double[] feature3 = {timestamp + 3000,(outputCount + 100)/(inputCount - 100)};
+
+        Double[][] features = {feature0, feature1,feature2,feature3};
+        Double[] labels = {queueCount,queueCount + 50, queueCount - 50, queueCount - 100};
+
+        OrdinaryLeastSquaresMSAM ordinaryLeastSquaresMSAM = new OrdinaryLeastSquaresMSAM();
+        SimpleRegressionBSAM simpleRegressionBSAM = new SimpleRegressionBSAM(false);
+
+        ordinaryLeastSquaresMSAM.learn(Stream.of(features), Stream.of(labels));
+        simpleRegressionBSAM.learn(Stream.of(features), Stream.of(labels));
+        double olsR2 = ordinaryLeastSquaresMSAM.getRSquared();
+        double srR2 = simpleRegressionBSAM.getRSquared();
+        assert(!Double.isNaN(olsR2));
+        assert(!Double.isNaN(srR2));
+        Map<String,Double> olsScores = ordinaryLeastSquaresMSAM.getScores();
+        Map<String,Double> srScores = simpleRegressionBSAM.getScores();
+        System.out.print(olsScores.toString());
+        System.out.print(srScores.toString());
+        assert(olsR2 > srR2);
+
+    }
+
+
+
+
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/models/TestSimpleRegressionBSAM.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/models/TestSimpleRegressionBSAM.java
new file mode 100644
index 0000000..71da432
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/models/TestSimpleRegressionBSAM.java
@@ -0,0 +1,102 @@
+/*
+ * 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.analytics.models;
+
+import static org.junit.Assert.assertNotNull;
+
+import java.util.stream.Stream;
+import org.junit.Test;
+
+public class TestSimpleRegressionBSAM {
+
+    @Test
+    public void testConstantPrediction(){
+
+        Double timestamp = 1565444720000.0;
+        Double queueCount = 50.0;
+
+        Double[] feature0 = {timestamp - 1000};
+        Double[] feature1 = {timestamp};
+        Double[] feature2 = {timestamp + 1000};
+        Double[] feature3 = {timestamp + 2000};
+
+        Double[][] features = {feature0, feature1,feature2,feature3};
+        Double[] labels = {queueCount,queueCount,queueCount, queueCount};
+
+        SimpleRegressionBSAM model = new SimpleRegressionBSAM(false);
+
+        model.learn(Stream.of(features), Stream.of(labels));
+
+        Double[] predictor = {timestamp + 5000};
+        Double target = model.predict(predictor);
+        assertNotNull(target);
+        assert(target  == 50);
+
+    }
+
+    @Test
+    public void testVaryingPredictX(){
+
+        Double timestamp = 1565444720000.0;
+        Double queueCount = 950.0;
+
+        Double[] feature0 = {timestamp};
+        Double[] feature1 = {timestamp + 1000};
+        Double[] feature2 = {timestamp + 2000};
+        Double[] feature3 = {timestamp + 3000 };
+
+        Double[][] features = {feature0, feature1,feature2,feature3};
+        Double[] labels = {queueCount,queueCount + 50, queueCount - 50, queueCount - 100};
+
+        SimpleRegressionBSAM model = new SimpleRegressionBSAM(false);
+
+        model.learn(Stream.of(features), Stream.of(labels));
+
+        Double target = model.predictX(1000.0);
+        Double minTimeMillis = 1565343920000.0;
+        Double maxTimeMillis = 1565516720000.0;
+        assert(target >= minTimeMillis && target <= maxTimeMillis);
+
+    }
+
+    @Test
+    public void testVaryingPredictY(){
+
+        Double timestamp = 1565444720000.0;
+        Double queueCount = 950.0;
+
+        Double[] feature0 = {timestamp};
+        Double[] feature1 = {timestamp + 1000};
+        Double[] feature2 = {timestamp + 2000};
+        Double[] feature3 = {timestamp + 3000};
+
+        Double[][] features = {feature0, feature1,feature2,feature3};
+        Double[] labels = {queueCount,queueCount + 50, queueCount - 50, queueCount - 100};
+
+        SimpleRegressionBSAM model = new SimpleRegressionBSAM(false);
+
+        Double[] predictor = {timestamp + 5000};
+
+        model.learn(Stream.of(features), Stream.of(labels));
+        Double target = model.predict(predictor);
+        Double rSquared = model.getRSquared();
+        Double minCount = -1265.0;
+        Double maxCount = 3235.0;
+        assert(rSquared > .60);
+        assert(target >= minCount && target <= maxCount);
+    }
+}


[nifi] 08/21: NIFI-6510 Remove redundant connection prediction interfaces as we can just use ConnectionStatusAnalytics directly

Posted by ai...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

aichrist pushed a commit to branch analytics-framework
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit af7becb125bf62099de35f6731c5735f9601298a
Author: Andrew I. Christianson <an...@andyic.org>
AuthorDate: Wed Jul 24 12:21:35 2019 -0400

    NIFI-6510 Remove redundant connection prediction interfaces as we can just use ConnectionStatusAnalytics directly
---
 .../status/analytics/StatusAnalytics.java          | 28 ----------------------
 .../analytics/CachingStatusAnalyticEngine.java     | 20 ----------------
 .../status/analytics/StatusAnalyticEngine.java     | 22 -----------------
 .../apache/nifi/reporting/StandardEventAccess.java | 10 ++++----
 4 files changed, 6 insertions(+), 74 deletions(-)

diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java
index 45e1c12..131531f 100644
--- a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java
+++ b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java
@@ -28,32 +28,4 @@ public interface StatusAnalytics {
      * @return A ConnectionStatusAnalytics object
      */
     ConnectionStatusAnalytics getConnectionStatusAnalytics(String connectionId);
-
-    /**
-     * Returns the predicted time (in milliseconds) when backpressure is expected to be applied to this connection, based on the total number of bytes in the queue.
-     * @return milliseconds until backpressure is predicted to occur, based on the total number of bytes in the queue.
-     * @param connectionId
-     */
-    long getTimeToBytesBackpressureMillis(String connectionId);
-
-    /**
-     * Returns the predicted time (in milliseconds) when backpressure is expected to be applied to this connection, based on the number of objects in the queue.
-     * @return milliseconds until backpressure is predicted to occur, based on the number of objects in the queue.
-     * @param connectionId
-     */
-    long getTimeToCountBackpressureMillis(String connectionId);
-
-    /**
-     * Returns the predicted total number of bytes in the queue to occur at the next configured interval (5 mins in the future, e.g.).
-     * @return milliseconds until backpressure is predicted to occur, based on the total number of bytes in the queue.
-     * @param connectionId
-     */
-    long getNextIntervalBytes(String connectionId);
-
-    /**
-     * Returns the predicted number of objects in the queue to occur at the next configured interval (5 mins in the future, e.g.).
-     * @return milliseconds until backpressure is predicted to occur, based on the number of bytes in the queue.
-     * @param connectionId
-     */
-    int getNextIntervalCount(String connectionId);
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingStatusAnalyticEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingStatusAnalyticEngine.java
index 864a5d4..015d6f8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingStatusAnalyticEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingStatusAnalyticEngine.java
@@ -51,26 +51,6 @@ public class CachingStatusAnalyticEngine implements StatusAnalytics {
         return cachedResult;
     }
 
-    @Override
-    public long getTimeToBytesBackpressureMillis(String connectionId) {
-        return 0;
-    }
-
-    @Override
-    public long getTimeToCountBackpressureMillis(String connectionId) {
-        return getConnectionStatusAnalytics(connectionId).getTimeToCountBackpressureMillis();
-    }
-
-    @Override
-    public long getNextIntervalBytes(String connectionId) {
-        return 0;
-    }
-
-    @Override
-    public int getNextIntervalCount(String connectionId) {
-        return 0;
-    }
-
     protected ConnectionStatusAnalytics calculate(SimpleRegression regression, Connection conn){
         long backPressureObjectThreshold = conn.getFlowFileQueue().getBackPressureObjectThreshold();
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java
index 5a873d5..56c263e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java
@@ -158,7 +158,6 @@ public class StatusAnalyticEngine implements StatusAnalytics {
     public long getMinTimeToBackpressureMillis() {
         ProcessGroup rootGroup = controller.getFlowManager().getRootGroup();
         List<Connection> allConnections = rootGroup.findAllConnections();
-        rootGroup.findConnection("asdf");
         long minTimeToBackpressure = Long.MAX_VALUE;
 
         for (Connection conn : allConnections) {
@@ -169,25 +168,4 @@ public class StatusAnalyticEngine implements StatusAnalytics {
         LOG.info("Min time to backpressure is: " + Long.toString(minTimeToBackpressure));
         return minTimeToBackpressure;
     }
-
-    // TODO - populate the prediction fields. Do we need to pass in connection ID?
-    @Override
-    public long getTimeToCountBackpressureMillis(String connectionId) {
-        return 0;
-    }
-
-    @Override
-    public long getTimeToBytesBackpressureMillis(String connectionId) {
-        return 0;
-    }
-
-    @Override
-    public long getNextIntervalBytes(String connectionId) {
-        return 0;
-    }
-
-    @Override
-    public int getNextIntervalCount(String connectionId) {
-        return 0;
-    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java
index 87fcd4d..aeb9559 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java
@@ -51,6 +51,7 @@ import org.apache.nifi.controller.status.ProcessorStatus;
 import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
 import org.apache.nifi.controller.status.RunStatus;
 import org.apache.nifi.controller.status.TransmissionStatus;
+import org.apache.nifi.controller.status.analytics.ConnectionStatusAnalytics;
 import org.apache.nifi.controller.status.analytics.StatusAnalytics;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
@@ -342,10 +343,11 @@ public class StandardEventAccess implements UserAwareEventAccess {
             }
 
             if (statusAnalytics != null) {
-                connStatus.setPredictedTimeToBytesBackpressureMillis(statusAnalytics.getTimeToBytesBackpressureMillis(conn.getIdentifier()));
-                connStatus.setPredictedTimeToCountBackpressureMillis(statusAnalytics.getTimeToCountBackpressureMillis(conn.getIdentifier()));
-                connStatus.setNextPredictedQueuedBytes(statusAnalytics.getNextIntervalBytes(conn.getIdentifier()));
-                connStatus.setNextPredictedQueuedCount(statusAnalytics.getNextIntervalCount(conn.getIdentifier()));
+                ConnectionStatusAnalytics connectionStatusAnalytics = statusAnalytics.getConnectionStatusAnalytics(conn.getIdentifier());
+                connStatus.setPredictedTimeToBytesBackpressureMillis(connectionStatusAnalytics.getTimeToBytesBackpressureMillis());
+                connStatus.setPredictedTimeToCountBackpressureMillis(connectionStatusAnalytics.getTimeToCountBackpressureMillis());
+                connStatus.setNextPredictedQueuedBytes(connectionStatusAnalytics.getNextIntervalBytes());
+                connStatus.setNextPredictedQueuedCount(connectionStatusAnalytics.getNextIntervalCount());
             }
 
             if (isConnectionAuthorized) {


[nifi] 03/21: NIFI-6510 Initial analytics REST endpoint and supporting objects

Posted by ai...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

aichrist pushed a commit to branch analytics-framework
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 42b50e81c8e7b1251c401baabad1c6ae41bdb111
Author: Matthew Burgess <ma...@apache.org>
AuthorDate: Mon Jul 15 11:56:55 2019 -0400

    NIFI-6510 Initial analytics REST endpoint and supporting objects
---
 .../status/analytics/StatusAnalytics.java          |  21 +++
 .../api/dto/status/ConnectionStatisticsDTO.java    | 161 +++++++++++++++++++++
 .../status/ConnectionStatisticsSnapshotDTO.java    | 149 +++++++++++++++++++
 .../NodeConnectionStatisticsSnapshotDTO.java       |  78 ++++++++++
 .../web/api/entity/ConnectionStatisticsEntity.java |  55 +++++++
 .../entity/ConnectionStatisticsSnapshotEntity.java |  76 ++++++++++
 .../org/apache/nifi/controller/FlowController.java |   3 +-
 .../status/analytics/StatusAnalyticEngine.java     |   3 +-
 .../org/apache/nifi/web/NiFiServiceFacade.java     |   9 ++
 .../apache/nifi/web/StandardNiFiServiceFacade.java |  10 ++
 .../java/org/apache/nifi/web/api/FlowResource.java |  74 ++++++++++
 .../org/apache/nifi/web/api/dto/DtoFactory.java    |  28 ++++
 .../org/apache/nifi/web/api/dto/EntityFactory.java |  19 +++
 .../nifi/web/controller/ControllerFacade.java      |  32 ++++
 14 files changed, 716 insertions(+), 2 deletions(-)

diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java
new file mode 100644
index 0000000..d6ad3bc
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java
@@ -0,0 +1,21 @@
+/*
+ * 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.analytics;
+
+public interface StatusAnalytics {
+    long getMinTimeToBackpressureMillis();
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatisticsDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatisticsDTO.java
new file mode 100644
index 0000000..79ae947
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatisticsDTO.java
@@ -0,0 +1,161 @@
+/*
+ * 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.web.api.dto.status;
+
+import io.swagger.annotations.ApiModelProperty;
+import org.apache.nifi.web.api.dto.util.TimeAdapter;
+
+import javax.xml.bind.annotation.XmlType;
+import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+
+@XmlType(name = "connectionStatistics")
+public class ConnectionStatisticsDTO implements Cloneable {
+    private String id;
+    private String groupId;
+    private String name;
+    private Date statsLastRefreshed;
+
+    private String sourceId;
+    private String sourceName;
+    private String destinationId;
+
+    private String destinationName;
+    private ConnectionStatisticsSnapshotDTO aggregateSnapshot;
+
+    private List<NodeConnectionStatisticsSnapshotDTO> nodeSnapshots;
+
+    @ApiModelProperty("The ID of the connection")
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    @ApiModelProperty("The ID of the Process Group that the connection belongs to")
+    public String getGroupId() {
+        return groupId;
+    }
+
+    public void setGroupId(String groupId) {
+        this.groupId = groupId;
+    }
+
+    @ApiModelProperty("The name of the connection")
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    @ApiModelProperty("The ID of the source component")
+    public String getSourceId() {
+        return sourceId;
+    }
+
+    public void setSourceId(String sourceId) {
+        this.sourceId = sourceId;
+    }
+
+    @ApiModelProperty("The name of the source component")
+    public String getSourceName() {
+        return sourceName;
+    }
+
+    public void setSourceName(String sourceName) {
+        this.sourceName = sourceName;
+    }
+
+    @ApiModelProperty("The ID of the destination component")
+    public String getDestinationId() {
+        return destinationId;
+    }
+
+    public void setDestinationId(String destinationId) {
+        this.destinationId = destinationId;
+    }
+
+    @ApiModelProperty("The name of the destination component")
+    public String getDestinationName() {
+        return destinationName;
+    }
+
+    public void setDestinationName(String destinationName) {
+        this.destinationName = destinationName;
+    }
+
+    @ApiModelProperty("The status snapshot that represents the aggregate stats of the cluster")
+    public ConnectionStatisticsSnapshotDTO getAggregateSnapshot() {
+        return aggregateSnapshot;
+    }
+
+    public void setAggregateSnapshot(ConnectionStatisticsSnapshotDTO aggregateSnapshot) {
+        this.aggregateSnapshot = aggregateSnapshot;
+    }
+
+    @ApiModelProperty("A list of status snapshots for each node")
+    public List<NodeConnectionStatisticsSnapshotDTO> getNodeSnapshots() {
+        return nodeSnapshots;
+    }
+
+    public void setNodeSnapshots(List<NodeConnectionStatisticsSnapshotDTO> nodeSnapshots) {
+        this.nodeSnapshots = nodeSnapshots;
+    }
+
+    @XmlJavaTypeAdapter(TimeAdapter.class)
+    @ApiModelProperty(
+            value = "The timestamp of when the stats were last refreshed",
+            dataType = "string"
+    )
+    public Date getStatsLastRefreshed() {
+        return statsLastRefreshed;
+    }
+
+    public void setStatsLastRefreshed(Date statsLastRefreshed) {
+        this.statsLastRefreshed = statsLastRefreshed;
+    }
+
+    @Override
+    public ConnectionStatisticsDTO clone() {
+        final ConnectionStatisticsDTO other = new ConnectionStatisticsDTO();
+        other.setDestinationId(getDestinationId());
+        other.setDestinationName(getDestinationName());
+        other.setGroupId(getGroupId());
+        other.setId(getId());
+        other.setName(getName());
+        other.setSourceId(getSourceId());
+        other.setSourceName(getSourceName());
+        other.setAggregateSnapshot(getAggregateSnapshot().clone());
+
+
+        final List<NodeConnectionStatisticsSnapshotDTO> nodeStatuses = getNodeSnapshots();
+        final List<NodeConnectionStatisticsSnapshotDTO> nodeStatusClones = new ArrayList<>(nodeStatuses.size());
+        for (final NodeConnectionStatisticsSnapshotDTO nodeStatus : nodeStatuses) {
+            nodeStatusClones.add(nodeStatus.clone());
+        }
+        other.setNodeSnapshots(nodeStatusClones);
+
+        return other;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatisticsSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatisticsSnapshotDTO.java
new file mode 100644
index 0000000..e914f74
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatisticsSnapshotDTO.java
@@ -0,0 +1,149 @@
+/*
+ * 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.web.api.dto.status;
+
+import io.swagger.annotations.ApiModelProperty;
+
+import javax.xml.bind.annotation.XmlType;
+
+/**
+ * DTO for serializing the statistics of a connection.
+ */
+@XmlType(name = "connectionStatisticsSnapshot")
+public class ConnectionStatisticsSnapshotDTO implements Cloneable {
+
+    private String id;
+    private String groupId;
+    private String name;
+
+    private String sourceId;
+    private String sourceName;
+    private String destinationId;
+    private String destinationName;
+
+    private Long predictedMillisUntilBackpressure = 0L;
+
+    /* getters / setters */
+    /**
+     * @return The connection id
+     */
+    @ApiModelProperty("The id of the connection.")
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    /**
+     * @return the ID of the Process Group to which this connection belongs.
+     */
+    @ApiModelProperty("The id of the process group the connection belongs to.")
+    public String getGroupId() {
+        return groupId;
+    }
+
+    public void setGroupId(final String groupId) {
+        this.groupId = groupId;
+    }
+
+    /**
+     * @return name of this connection
+     */
+    @ApiModelProperty("The name of the connection.")
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    /**
+     * @return id of the source of this connection
+     */
+    @ApiModelProperty("The id of the source of the connection.")
+    public String getSourceId() {
+        return sourceId;
+    }
+
+    public void setSourceId(String sourceId) {
+        this.sourceId = sourceId;
+    }
+
+    /**
+     * @return name of the source of this connection
+     */
+    @ApiModelProperty("The name of the source of the connection.")
+    public String getSourceName() {
+        return sourceName;
+    }
+
+    public void setSourceName(String sourceName) {
+        this.sourceName = sourceName;
+    }
+
+    /**
+     * @return id of the destination of this connection
+     */
+    @ApiModelProperty("The id of the destination of the connection.")
+    public String getDestinationId() {
+        return destinationId;
+    }
+
+    public void setDestinationId(String destinationId) {
+        this.destinationId = destinationId;
+    }
+
+    /**
+     * @return name of the destination of this connection
+     */
+    @ApiModelProperty("The name of the destination of the connection.")
+    public String getDestinationName() {
+        return destinationName;
+    }
+
+    public void setDestinationName(String destinationName) {
+        this.destinationName = destinationName;
+    }
+
+    @ApiModelProperty("The predicted number of milliseconds before the connection will have backpressure applied.")
+    public Long getPredictedMillisUntilBackpressure() {
+        return predictedMillisUntilBackpressure;
+    }
+
+    public void setPredictedMillisUntilBackpressure(Long predictedMillisUntilBackpressure) {
+        this.predictedMillisUntilBackpressure = predictedMillisUntilBackpressure;
+    }
+
+    @Override
+    public ConnectionStatisticsSnapshotDTO clone() {
+        final ConnectionStatisticsSnapshotDTO other = new ConnectionStatisticsSnapshotDTO();
+        other.setDestinationId(getDestinationId());
+        other.setDestinationName(getDestinationName());
+        other.setGroupId(getGroupId());
+        other.setId(getId());
+        other.setName(getName());
+        other.setSourceId(getSourceId());
+        other.setSourceName(getSourceName());
+
+        other.setPredictedMillisUntilBackpressure(getPredictedMillisUntilBackpressure());
+
+        return other;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectionStatisticsSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectionStatisticsSnapshotDTO.java
new file mode 100644
index 0000000..76f94ec
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectionStatisticsSnapshotDTO.java
@@ -0,0 +1,78 @@
+/*
+ * 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.web.api.dto.status;
+
+import io.swagger.annotations.ApiModelProperty;
+
+import javax.xml.bind.annotation.XmlType;
+
+@XmlType(name = "nodeConnectionStatisticsSnapshot")
+public class NodeConnectionStatisticsSnapshotDTO implements Cloneable {
+    private String nodeId;
+    private String address;
+    private Integer apiPort;
+
+    private ConnectionStatisticsSnapshotDTO statisticsSnapshot;
+
+    @ApiModelProperty("The unique ID that identifies the node")
+    public String getNodeId() {
+        return nodeId;
+    }
+
+    public void setNodeId(String nodeId) {
+        this.nodeId = nodeId;
+    }
+
+    @ApiModelProperty("The API address of the node")
+    public String getAddress() {
+        return address;
+    }
+
+    public void setAddress(String address) {
+        this.address = address;
+    }
+
+    @ApiModelProperty("The API port used to communicate with the node")
+    public Integer getApiPort() {
+        return apiPort;
+    }
+
+    public void setApiPort(Integer apiPort) {
+        this.apiPort = apiPort;
+    }
+
+    @ApiModelProperty("The connection status snapshot from the node.")
+    public ConnectionStatisticsSnapshotDTO getStatisticsSnapshot() {
+        return statisticsSnapshot;
+    }
+
+    public void setStatisticsSnapshot(ConnectionStatisticsSnapshotDTO statisticsSnapshot) {
+        this.statisticsSnapshot = statisticsSnapshot;
+    }
+
+    @Override
+    public NodeConnectionStatisticsSnapshotDTO clone() {
+        final NodeConnectionStatisticsSnapshotDTO other = new NodeConnectionStatisticsSnapshotDTO();
+        other.setNodeId(getNodeId());
+        other.setAddress(getAddress());
+        other.setApiPort(getApiPort());
+        other.setStatisticsSnapshot(getStatisticsSnapshot().clone());
+        return other;
+    }
+
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionStatisticsEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionStatisticsEntity.java
new file mode 100644
index 0000000..781cff6
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionStatisticsEntity.java
@@ -0,0 +1,55 @@
+/*
+ * 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.web.api.entity;
+
+import org.apache.nifi.web.api.dto.ReadablePermission;
+import org.apache.nifi.web.api.dto.status.ConnectionStatisticsDTO;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+/**
+ * A serialized representation of this class can be placed in the entity body of a request or response to or from the API. This particular entity holds a reference to a ConnectionStatisticsDTO.
+ */
+@XmlRootElement(name = "connectionStatisticsEntity")
+public class ConnectionStatisticsEntity extends Entity implements ReadablePermission {
+
+    private ConnectionStatisticsDTO connectionStatistics;
+    private Boolean canRead;
+
+    /**
+     * The ConnectionStatisticsDTO that is being serialized.
+     *
+     * @return The ConnectionStatisticsDTO object
+     */
+    public ConnectionStatisticsDTO getConnectionStatistics() {
+        return connectionStatistics;
+    }
+
+    public void setConnectionStatistics(ConnectionStatisticsDTO connectionStatistics) {
+        this.connectionStatistics = connectionStatistics;
+    }
+
+    @Override
+    public Boolean getCanRead() {
+        return canRead;
+    }
+
+    @Override
+    public void setCanRead(Boolean canRead) {
+        this.canRead = canRead;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionStatisticsSnapshotEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionStatisticsSnapshotEntity.java
new file mode 100644
index 0000000..da7e5ca
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionStatisticsSnapshotEntity.java
@@ -0,0 +1,76 @@
+/*
+ * 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.web.api.entity;
+
+import io.swagger.annotations.ApiModelProperty;
+import org.apache.nifi.web.api.dto.ReadablePermission;
+import org.apache.nifi.web.api.dto.status.ConnectionStatisticsSnapshotDTO;
+import org.apache.nifi.web.api.dto.status.ConnectionStatusSnapshotDTO;
+
+/**
+ * A serialized representation of this class can be placed in the entity body of a request or response to or from the API.
+ * This particular entity holds a reference to a ConnectionStatisticsSnapshotDTO.
+ */
+public class ConnectionStatisticsSnapshotEntity extends Entity implements ReadablePermission, Cloneable {
+    private String id;
+    private ConnectionStatisticsSnapshotDTO connectionStatisticsSnapshot;
+    private Boolean canRead;
+
+    /**
+     * @return The connection id
+     */
+    @ApiModelProperty("The id of the connection.")
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    /**
+     * The ConnectionStatisticsSnapshotDTO that is being serialized.
+     *
+     * @return The ConnectionStatisticsSnapshotDTO object
+     */
+    public ConnectionStatisticsSnapshotDTO getConnectionStatisticsSnapshot() {
+        return connectionStatisticsSnapshot;
+    }
+
+    public void setConnectionStatisticsSnapshot(ConnectionStatisticsSnapshotDTO connectionStatusSnapshot) {
+        this.connectionStatisticsSnapshot = connectionStatusSnapshot;
+    }
+
+    @Override
+    public Boolean getCanRead() {
+        return canRead;
+    }
+
+    @Override
+    public void setCanRead(Boolean canRead) {
+        this.canRead = canRead;
+    }
+
+    @Override
+    public ConnectionStatisticsSnapshotEntity clone() {
+        final ConnectionStatisticsSnapshotEntity other = new ConnectionStatisticsSnapshotEntity();
+        other.setCanRead(this.getCanRead());
+        other.setConnectionStatisticsSnapshot(this.getConnectionStatisticsSnapshot().clone());
+
+        return other;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index f7ed734..0c422b4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -118,6 +118,7 @@ import org.apache.nifi.controller.service.StandardControllerServiceProvider;
 import org.apache.nifi.controller.state.manager.StandardStateManagerProvider;
 import org.apache.nifi.controller.state.server.ZooKeeperStateServer;
 import org.apache.nifi.controller.status.analytics.StatusAnalyticEngine;
+import org.apache.nifi.controller.status.analytics.StatusAnalytics;
 import org.apache.nifi.controller.status.history.ComponentStatusRepository;
 import org.apache.nifi.controller.status.history.GarbageCollectionHistory;
 import org.apache.nifi.controller.status.history.GarbageCollectionStatus;
@@ -602,7 +603,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
             }
         }, snapshotMillis, snapshotMillis, TimeUnit.MILLISECONDS);
 
-        StatusAnalyticEngine analyticsEngine = new StatusAnalyticEngine(this, componentStatusRepository);
+        StatusAnalytics analyticsEngine = new StatusAnalyticEngine(this, componentStatusRepository);
 
         timerDrivenEngineRef.get().scheduleWithFixedDelay(new Runnable() {
             @Override
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java
index 0602a93..9231707 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java
@@ -31,7 +31,7 @@ import org.apache.nifi.web.api.dto.status.StatusSnapshotDTO;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class StatusAnalyticEngine {
+public class StatusAnalyticEngine implements StatusAnalytics {
     private ComponentStatusRepository statusRepository;
     private FlowController controller;
 
@@ -42,6 +42,7 @@ public class StatusAnalyticEngine {
         this.statusRepository = statusRepository;
     }
 
+    @Override
     public long getMinTimeToBackpressureMillis() {
         ProcessGroup rootGroup = controller.getFlowManager().getRootGroup();
         List<Connection> allConnections = rootGroup.findAllConnections();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
index f62bec2..9ed8808 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
@@ -87,6 +87,7 @@ import org.apache.nifi.web.api.entity.BucketEntity;
 import org.apache.nifi.web.api.entity.BulletinEntity;
 import org.apache.nifi.web.api.entity.ComponentValidationResultEntity;
 import org.apache.nifi.web.api.entity.ConnectionEntity;
+import org.apache.nifi.web.api.entity.ConnectionStatisticsEntity;
 import org.apache.nifi.web.api.entity.ConnectionStatusEntity;
 import org.apache.nifi.web.api.entity.ControllerBulletinsEntity;
 import org.apache.nifi.web.api.entity.ControllerConfigurationEntity;
@@ -662,6 +663,14 @@ public interface NiFiServiceFacade {
     StatusHistoryEntity getConnectionStatusHistory(String connectionId);
 
     /**
+     * Gets analytical statistics for the specified connection.
+     *
+     * @param connectionId connection
+     * @return statistics
+     */
+    ConnectionStatisticsEntity getConnectionStatistics(String connectionId);
+
+    /**
      * Creates a new Relationship target.
      *
      * @param revision revision
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
index 9f4744f..73115b5 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
@@ -216,6 +216,7 @@ import org.apache.nifi.web.api.dto.provenance.ProvenanceEventDTO;
 import org.apache.nifi.web.api.dto.provenance.ProvenanceOptionsDTO;
 import org.apache.nifi.web.api.dto.provenance.lineage.LineageDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.api.dto.status.ConnectionStatisticsDTO;
 import org.apache.nifi.web.api.dto.status.ConnectionStatusDTO;
 import org.apache.nifi.web.api.dto.status.ControllerStatusDTO;
 import org.apache.nifi.web.api.dto.status.NodeProcessGroupStatusSnapshotDTO;
@@ -235,6 +236,7 @@ import org.apache.nifi.web.api.entity.BulletinEntity;
 import org.apache.nifi.web.api.entity.ComponentReferenceEntity;
 import org.apache.nifi.web.api.entity.ComponentValidationResultEntity;
 import org.apache.nifi.web.api.entity.ConnectionEntity;
+import org.apache.nifi.web.api.entity.ConnectionStatisticsEntity;
 import org.apache.nifi.web.api.entity.ConnectionStatusEntity;
 import org.apache.nifi.web.api.entity.ControllerBulletinsEntity;
 import org.apache.nifi.web.api.entity.ControllerConfigurationEntity;
@@ -3190,6 +3192,14 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
         return entityFactory.createStatusHistoryEntity(dto, permissions);
     }
 
+    @Override
+    public ConnectionStatisticsEntity getConnectionStatistics(final String connectionId) {
+        final Connection connection = connectionDAO.getConnection(connectionId);
+        final PermissionsDTO permissions = dtoFactory.createPermissionsDto(connection);
+        final ConnectionStatisticsDTO dto = dtoFactory.createConnectionStatisticsDto(controllerFacade.getConnectionStatistics(connectionId));
+        return entityFactory.createConnectionStatisticsEntity(dto, permissions);
+    }
+
     private ProcessorEntity createProcessorEntity(final ProcessorNode processor, final NiFiUser user) {
         final RevisionDTO revision = dtoFactory.createRevisionDTO(revisionManager.getRevision(processor.getIdentifier()));
         final PermissionsDTO permissions = dtoFactory.createPermissionsDto(processor, user);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
index 4b34f39..6f28d44 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
@@ -75,6 +75,7 @@ import org.apache.nifi.web.api.entity.BulletinBoardEntity;
 import org.apache.nifi.web.api.entity.ClusteSummaryEntity;
 import org.apache.nifi.web.api.entity.ClusterSearchResultsEntity;
 import org.apache.nifi.web.api.entity.ComponentHistoryEntity;
+import org.apache.nifi.web.api.entity.ConnectionStatisticsEntity;
 import org.apache.nifi.web.api.entity.ConnectionStatusEntity;
 import org.apache.nifi.web.api.entity.ControllerBulletinsEntity;
 import org.apache.nifi.web.api.entity.ControllerServiceEntity;
@@ -2074,6 +2075,79 @@ public class FlowResource extends ApplicationResource {
         return generateOkResponse(entity).build();
     }
 
+    /**
+     * Retrieves the specified connection statistics.
+     *
+     * @param id The id of the connection statistics to retrieve.
+     * @return A ConnectionStatisticsEntity.
+     * @throws InterruptedException if interrupted
+     */
+    @GET
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.APPLICATION_JSON)
+    @Path("connections/{id}/statistics")
+    @ApiOperation(
+            value = "Gets statistics for a connection",
+            response = ConnectionStatisticsEntity.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 getConnectionStatistics(
+            @ApiParam(
+                    value = "Whether or not to include the breakdown per node. Optional, defaults to false",
+                    required = false
+            )
+            @QueryParam("nodewise") @DefaultValue(NODEWISE) Boolean nodewise,
+            @ApiParam(
+                    value = "The id of the node where to get the statistics.",
+                    required = false
+            )
+            @QueryParam("clusterNodeId") String clusterNodeId,
+            @ApiParam(
+                    value = "The connection id.",
+                    required = true
+            )
+            @PathParam("id") String id) throws InterruptedException {
+
+        authorizeFlow();
+
+        // ensure a valid request
+        if (Boolean.TRUE.equals(nodewise) && clusterNodeId != null) {
+            throw new IllegalArgumentException("Nodewise requests cannot be directed at a specific node.");
+        }
+
+        if (isReplicateRequest()) {
+            // determine where this request should be sent
+            if (clusterNodeId == null) {
+                final NodeResponse nodeResponse = replicateNodeResponse(HttpMethod.GET);
+                final ConnectionStatisticsEntity entity = (ConnectionStatisticsEntity) nodeResponse.getUpdatedEntity();
+
+                // ensure there is an updated entity (result of merging) and prune the response as necessary
+                if (entity != null && !nodewise) {
+                    entity.getConnectionStatistics().setNodeSnapshots(null);
+                }
+
+                return nodeResponse.getResponse();
+            } else {
+                return replicate(HttpMethod.GET, clusterNodeId);
+            }
+        }
+
+        // get the specified connection status
+        final ConnectionStatisticsEntity entity = serviceFacade.getConnectionStatistics(id);
+        return generateOkResponse(entity).build();
+    }
+
     // --------------
     // status history
     // --------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
index 97b8c5e..6903e44 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
@@ -105,6 +105,7 @@ import org.apache.nifi.controller.status.PortStatus;
 import org.apache.nifi.controller.status.ProcessGroupStatus;
 import org.apache.nifi.controller.status.ProcessorStatus;
 import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
+import org.apache.nifi.controller.status.analytics.StatusAnalytics;
 import org.apache.nifi.controller.status.history.GarbageCollectionHistory;
 import org.apache.nifi.controller.status.history.GarbageCollectionStatus;
 import org.apache.nifi.diagnostics.GarbageCollection;
@@ -198,6 +199,8 @@ import org.apache.nifi.web.api.dto.provenance.lineage.LineageRequestDTO.LineageR
 import org.apache.nifi.web.api.dto.provenance.lineage.LineageResultsDTO;
 import org.apache.nifi.web.api.dto.provenance.lineage.ProvenanceLinkDTO;
 import org.apache.nifi.web.api.dto.provenance.lineage.ProvenanceNodeDTO;
+import org.apache.nifi.web.api.dto.status.ConnectionStatisticsDTO;
+import org.apache.nifi.web.api.dto.status.ConnectionStatisticsSnapshotDTO;
 import org.apache.nifi.web.api.dto.status.ConnectionStatusDTO;
 import org.apache.nifi.web.api.dto.status.ConnectionStatusSnapshotDTO;
 import org.apache.nifi.web.api.dto.status.PortStatusDTO;
@@ -1186,6 +1189,31 @@ public final class DtoFactory {
         return connectionStatusDto;
     }
 
+    public ConnectionStatisticsDTO createConnectionStatisticsDto(final StatusAnalytics connectionStatistics) {
+        final ConnectionStatisticsDTO connectionStatisticsDTO = new ConnectionStatisticsDTO();
+        connectionStatisticsDTO.setGroupId(connectionStatistics.getGroupId());
+        connectionStatisticsDTO.setId(connectionStatistics.getId());
+        connectionStatisticsDTO.setName(connectionStatistics.getName());
+        connectionStatisticsDTO.setSourceId(connectionStatistics.getSourceId());
+        connectionStatisticsDTO.setSourceName(connectionStatistics.getSourceName());
+        connectionStatisticsDTO.setDestinationId(connectionStatistics.getDestinationId());
+        connectionStatisticsDTO.setDestinationName(connectionStatistics.getDestinationName());
+        connectionStatisticsDTO.setStatsLastRefreshed(new Date());
+
+        final ConnectionStatisticsSnapshotDTO snapshot = new ConnectionStatisticsSnapshotDTO();
+        connectionStatisticsDTO.setAggregateSnapshot(snapshot);
+
+        snapshot.setId(connectionStatistics.getId());
+        snapshot.setGroupId(connectionStatistics.getGroupId());
+        snapshot.setName(connectionStatistics.getName());
+        snapshot.setSourceName(connectionStatistics.getSourceName());
+        snapshot.setDestinationName(connectionStatistics.getDestinationName());
+
+        snapshot.setPredictedMillisUntilBackpressure(connectionStatistics.getMinTimeToBackpressureMillis());
+
+        return connectionStatisticsDTO;
+    }
+
     public ProcessorStatusDTO createProcessorStatusDto(final ProcessorStatus procStatus) {
         final ProcessorStatusDTO dto = new ProcessorStatusDTO();
         dto.setId(procStatus.getId());
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java
index 915ad2c..1db0adc 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java
@@ -20,6 +20,8 @@ import org.apache.nifi.web.api.dto.action.ActionDTO;
 import org.apache.nifi.web.api.dto.diagnostics.ProcessorDiagnosticsDTO;
 import org.apache.nifi.web.api.dto.flow.FlowBreadcrumbDTO;
 import org.apache.nifi.web.api.dto.flow.ProcessGroupFlowDTO;
+import org.apache.nifi.web.api.dto.status.ConnectionStatisticsDTO;
+import org.apache.nifi.web.api.dto.status.ConnectionStatisticsSnapshotDTO;
 import org.apache.nifi.web.api.dto.status.ConnectionStatusDTO;
 import org.apache.nifi.web.api.dto.status.ConnectionStatusSnapshotDTO;
 import org.apache.nifi.web.api.dto.status.ControllerServiceStatusDTO;
@@ -43,6 +45,8 @@ import org.apache.nifi.web.api.entity.BulletinEntity;
 import org.apache.nifi.web.api.entity.ComponentReferenceEntity;
 import org.apache.nifi.web.api.entity.ComponentValidationResultEntity;
 import org.apache.nifi.web.api.entity.ConnectionEntity;
+import org.apache.nifi.web.api.entity.ConnectionStatisticsEntity;
+import org.apache.nifi.web.api.entity.ConnectionStatisticsSnapshotEntity;
 import org.apache.nifi.web.api.entity.ConnectionStatusEntity;
 import org.apache.nifi.web.api.entity.ConnectionStatusSnapshotEntity;
 import org.apache.nifi.web.api.entity.ControllerConfigurationEntity;
@@ -137,6 +141,21 @@ public final class EntityFactory {
         return entity;
     }
 
+    public ConnectionStatisticsEntity createConnectionStatisticsEntity(final ConnectionStatisticsDTO statistics, final PermissionsDTO permissions) {
+        final ConnectionStatisticsEntity entity = new ConnectionStatisticsEntity();
+        entity.setCanRead(permissions.getCanRead());
+        entity.setConnectionStatistics(statistics); // always set the statistics, as it's always allowed... just need to provide permission context for merging responses
+        return entity;
+    }
+
+    public ConnectionStatisticsSnapshotEntity createConnectionStatisticsSnapshotEntity(final ConnectionStatisticsSnapshotDTO statistics, final PermissionsDTO permissions) {
+        final ConnectionStatisticsSnapshotEntity entity = new ConnectionStatisticsSnapshotEntity();
+        entity.setId(statistics.getId());
+        entity.setCanRead(permissions.getCanRead());
+        entity.setConnectionStatisticsSnapshot(statistics); // always set the statistics, as it's always allowed... just need to provide permission context for merging responses
+        return entity;
+    }
+
     public ProcessGroupStatusEntity createProcessGroupStatusEntity(final ProcessGroupStatusDTO status, final PermissionsDTO permissions) {
         final ProcessGroupStatusEntity entity = new ProcessGroupStatusEntity();
         entity.setCanRead(permissions.getCanRead());
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
index e560516..c1b6754 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
@@ -56,6 +56,7 @@ import org.apache.nifi.controller.status.PortStatus;
 import org.apache.nifi.controller.status.ProcessGroupStatus;
 import org.apache.nifi.controller.status.ProcessorStatus;
 import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
+import org.apache.nifi.controller.status.analytics.StatusAnalytics;
 import org.apache.nifi.controller.status.history.ComponentStatusRepository;
 import org.apache.nifi.diagnostics.SystemDiagnostics;
 import org.apache.nifi.flowfile.FlowFilePrioritizer;
@@ -681,6 +682,37 @@ public class ControllerFacade implements Authorizable {
     }
 
     /**
+     * Gets analytical statistics for the specified connection.
+     *
+     * @param connectionId connection id
+     * @return the statistics for the specified connection
+     */
+    public StatusAnalytics getConnectionStatistics(final String connectionId) {
+        final ProcessGroup root = getRootGroup();
+        final Connection connection = root.findConnection(connectionId);
+
+        // ensure the connection was found
+        if (connection == null) {
+            throw new ResourceNotFoundException(String.format("Unable to locate connection with id '%s'.", connectionId));
+        }
+
+        // calculate the process group status
+        final String groupId = connection.getProcessGroup().getIdentifier();
+        final ProcessGroupStatus processGroupStatus = flowController.getEventAccess().getGroupStatus(groupId, NiFiUserUtils.getNiFiUser(), 1);
+        if (processGroupStatus == null) {
+            throw new ResourceNotFoundException(String.format("Unable to locate group with id '%s'.", groupId));
+        }
+
+        // TODO get from flow controller
+        final StatusAnalytics status;
+        if (status == null) {
+            throw new ResourceNotFoundException(String.format("Unable to locate connection with id '%s'.", connectionId));
+        }
+
+        return status;
+    }
+
+    /**
      * Gets the status for the specified input port.
      *
      * @param portId input port id


[nifi] 10/21: NIFI-6510 Split StatusAnalytics interface into Engine and per-Connection versions

Posted by ai...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

aichrist pushed a commit to branch analytics-framework
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 7adaa2460fce47c7851dcada7e54a670e8fe0bad
Author: Matthew Burgess <ma...@apache.org>
AuthorDate: Tue Jul 23 16:47:29 2019 -0400

    NIFI-6510 Split StatusAnalytics interface into Engine and per-Connection versions
---
 .../analytics/ConnectionStatusAnalytics.java       | 28 ++--------
 .../status/analytics/StatusAnalytics.java          | 22 ++------
 .../status/analytics/StatusAnalyticsEngine.java    | 22 ++++++++
 .../org/apache/nifi/controller/FlowController.java | 10 ++--
 ...=> CachingConnectionStatusAnalyticsEngine.java} | 60 ++++++++++------------
 ...e.java => ConnectionStatusAnalyticsEngine.java} | 47 ++---------------
 .../apache/nifi/reporting/StandardEventAccess.java | 18 ++++---
 .../nifi/web/controller/ControllerFacade.java      |  4 +-
 8 files changed, 81 insertions(+), 130 deletions(-)

diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
index 2380d55..9792ae4 100644
--- a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
+++ b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
@@ -16,31 +16,11 @@
  */
 package org.apache.nifi.controller.status.analytics;
 
-public interface ConnectionStatusAnalytics {
 
-    /**
-     * Returns the predicted time (in milliseconds) when backpressure is expected to be applied to this connection, based on the total number of bytes in the queue.
-     * @return milliseconds until backpressure is predicted to occur, based on the total number of bytes in the queue.
-     */
-    long getTimeToBytesBackpressureMillis();
-
-    /**
-     * Returns the predicted time (in milliseconds) when backpressure is expected to be applied to this connection, based on the number of objects in the queue.
-     * @return milliseconds until backpressure is predicted to occur, based on the number of objects in the queue.
-     */
-    long getTimeToCountBackpressureMillis();
-
-    /**
-     * Returns the predicted total number of bytes in the queue to occur at the next configured interval (5 mins in the future, e.g.).
-     * @return milliseconds until backpressure is predicted to occur, based on the total number of bytes in the queue.
-     */
-    long getNextIntervalBytes();
-
-    /**
-     * Returns the predicted number of objects in the queue to occur at the next configured interval (5 mins in the future, e.g.).
-     * @return milliseconds until backpressure is predicted to occur, based on the number of bytes in the queue.
-     */
-    int getNextIntervalCount();
+/**
+ * The ConnectionStatusAnalytics interface offers additional methods to the StatusAnalytics interface related to the supporting connection information (group ID, e.g.)
+ */
+public interface ConnectionStatusAnalytics extends StatusAnalytics{
 
     String getGroupId();
     String getId();
diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java
index 45e1c12..564f1c9 100644
--- a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java
+++ b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java
@@ -17,43 +17,31 @@
 package org.apache.nifi.controller.status.analytics;
 
 /**
- * StatusAnalytics
+ * The StatusAnalytics interface offers methods for accessing predicted and other values for a single component (Connection instance, e.g.)
  */
 public interface StatusAnalytics {
 
     /**
-     * Returns a ConnectionStatusAnalytics object containing all relevant metrics and analytical & statistical objects, as well as identity information for the connection.
-     *
-     * @param connectionId The unique ID of the connection
-     * @return A ConnectionStatusAnalytics object
-     */
-    ConnectionStatusAnalytics getConnectionStatusAnalytics(String connectionId);
-
-    /**
      * Returns the predicted time (in milliseconds) when backpressure is expected to be applied to this connection, based on the total number of bytes in the queue.
      * @return milliseconds until backpressure is predicted to occur, based on the total number of bytes in the queue.
-     * @param connectionId
      */
-    long getTimeToBytesBackpressureMillis(String connectionId);
+    long getTimeToBytesBackpressureMillis();
 
     /**
      * Returns the predicted time (in milliseconds) when backpressure is expected to be applied to this connection, based on the number of objects in the queue.
      * @return milliseconds until backpressure is predicted to occur, based on the number of objects in the queue.
-     * @param connectionId
      */
-    long getTimeToCountBackpressureMillis(String connectionId);
+    long getTimeToCountBackpressureMillis();
 
     /**
      * Returns the predicted total number of bytes in the queue to occur at the next configured interval (5 mins in the future, e.g.).
      * @return milliseconds until backpressure is predicted to occur, based on the total number of bytes in the queue.
-     * @param connectionId
      */
-    long getNextIntervalBytes(String connectionId);
+    long getNextIntervalBytes();
 
     /**
      * Returns the predicted number of objects in the queue to occur at the next configured interval (5 mins in the future, e.g.).
      * @return milliseconds until backpressure is predicted to occur, based on the number of bytes in the queue.
-     * @param connectionId
      */
-    int getNextIntervalCount(String connectionId);
+    int getNextIntervalCount();
 }
diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticsEngine.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticsEngine.java
new file mode 100644
index 0000000..5cbc333
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticsEngine.java
@@ -0,0 +1,22 @@
+/*
+ * 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.analytics;
+
+public interface StatusAnalyticsEngine {
+
+    ConnectionStatusAnalytics getConnectionStatusAnalytics(String connectionId);
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index 450d944..b5de777 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -150,8 +150,8 @@ import org.apache.nifi.controller.service.StandardConfigurationContext;
 import org.apache.nifi.controller.service.StandardControllerServiceProvider;
 import org.apache.nifi.controller.state.manager.StandardStateManagerProvider;
 import org.apache.nifi.controller.state.server.ZooKeeperStateServer;
-import org.apache.nifi.controller.status.analytics.CachingStatusAnalyticEngine;
-import org.apache.nifi.controller.status.analytics.StatusAnalytics;
+import org.apache.nifi.controller.status.analytics.CachingConnectionStatusAnalyticsEngine;
+import org.apache.nifi.controller.status.analytics.StatusAnalyticsEngine;
 import org.apache.nifi.controller.status.history.ComponentStatusRepository;
 import org.apache.nifi.controller.status.history.GarbageCollectionHistory;
 import org.apache.nifi.controller.status.history.GarbageCollectionStatus;
@@ -351,7 +351,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
     // guarded by rwLock
     private NodeConnectionStatus connectionStatus;
 
-    private StatusAnalytics analyticsEngine;
+    private StatusAnalyticsEngine analyticsEngine;
 
     // guarded by rwLock
     private String instanceId;
@@ -594,7 +594,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
         }
 
         componentStatusRepository = createComponentStatusRepository();
-        analyticsEngine = new CachingStatusAnalyticEngine(this, componentStatusRepository);
+        analyticsEngine = new CachingConnectionStatusAnalyticsEngine(this, componentStatusRepository);
         eventAccess = new StandardEventAccess(this, flowFileEventRepository);
 
         timerDrivenEngineRef.get().scheduleWithFixedDelay(new Runnable() {
@@ -1379,7 +1379,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
         return eventAccess;
     }
 
-    public StatusAnalytics getStatusAnalytics() {
+    public StatusAnalyticsEngine getStatusAnalyticsEngine() {
         return analyticsEngine;
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingStatusAnalyticEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingConnectionStatusAnalyticsEngine.java
similarity index 79%
rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingStatusAnalyticEngine.java
rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingConnectionStatusAnalyticsEngine.java
index 864a5d4..c12dbae 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingStatusAnalyticEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingConnectionStatusAnalyticsEngine.java
@@ -1,3 +1,19 @@
+/*
+ * 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.analytics;
 
 import java.util.Date;
@@ -19,17 +35,17 @@ import org.slf4j.LoggerFactory;
 import com.github.benmanes.caffeine.cache.Cache;
 import com.github.benmanes.caffeine.cache.Caffeine;
 
-public class CachingStatusAnalyticEngine implements StatusAnalytics {
+public class CachingConnectionStatusAnalyticsEngine implements StatusAnalyticsEngine {
     private ComponentStatusRepository statusRepository;
     private FlowController controller;
     private volatile Cache<String, SimpleRegression> cache;
-    private static final Logger LOG = LoggerFactory.getLogger(StatusAnalyticEngine.class);
+    private static final Logger LOG = LoggerFactory.getLogger(ConnectionStatusAnalyticsEngine.class);
 
-    public CachingStatusAnalyticEngine(FlowController controller, ComponentStatusRepository statusRepository) {
+    public CachingConnectionStatusAnalyticsEngine(FlowController controller, ComponentStatusRepository statusRepository) {
         this.controller = controller;
         this.statusRepository = statusRepository;
         this.cache = Caffeine.newBuilder()
-                .expireAfterWrite(1,TimeUnit.MINUTES)
+                .expireAfterWrite(1, TimeUnit.MINUTES)
                 .build();
     }
 
@@ -40,45 +56,25 @@ public class CachingStatusAnalyticEngine implements StatusAnalytics {
         Connection connection = rootGroup.findConnection(connectionId);
         SimpleRegression cachedRegression = cache.getIfPresent(connection.getIdentifier());
 
-        if(cachedRegression == null) {
+        if (cachedRegression == null) {
             cachedRegression = getBackPressureRegressionModel(connection);
-            if(cachedRegression != null)
+            if (cachedRegression != null)
                 cache.put(connection.getIdentifier(), cachedRegression);
         }
 
-        ConnectionStatusAnalytics cachedResult = calculate(cachedRegression,connection);
+        ConnectionStatusAnalytics cachedResult = calculate(cachedRegression, connection);
         LOG.info("Connection: " + connectionId + " Cached backpressure Time: " + cachedResult.getTimeToCountBackpressureMillis());
         return cachedResult;
     }
 
-    @Override
-    public long getTimeToBytesBackpressureMillis(String connectionId) {
-        return 0;
-    }
-
-    @Override
-    public long getTimeToCountBackpressureMillis(String connectionId) {
-        return getConnectionStatusAnalytics(connectionId).getTimeToCountBackpressureMillis();
-    }
-
-    @Override
-    public long getNextIntervalBytes(String connectionId) {
-        return 0;
-    }
-
-    @Override
-    public int getNextIntervalCount(String connectionId) {
-        return 0;
-    }
-
-    protected ConnectionStatusAnalytics calculate(SimpleRegression regression, Connection conn){
+    protected ConnectionStatusAnalytics calculate(SimpleRegression regression, Connection conn) {
         long backPressureObjectThreshold = conn.getFlowFileQueue().getBackPressureObjectThreshold();
 
         final long connTimeToBackpressure;
 
-        if(regression == null){
+        if (regression == null) {
             connTimeToBackpressure = Long.MAX_VALUE;
-        }else{
+        } else {
             //If calculation returns as negative only 0 will return
             connTimeToBackpressure = Math.max(0, Math.round((backPressureObjectThreshold - regression.getIntercept()) / regression.getSlope())
                     - System.currentTimeMillis());
@@ -147,6 +143,7 @@ public class CachingStatusAnalyticEngine implements StatusAnalytics {
 
     /**
      * Get backpressure model based on current data
+     *
      * @param conn the connection to run the analytic on
      * @return
      */
@@ -177,8 +174,5 @@ public class CachingStatusAnalyticEngine implements StatusAnalytics {
                 return regression;
             }
         }
-
     }
-
-
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalyticsEngine.java
similarity index 77%
rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java
rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalyticsEngine.java
index 5a873d5..11862c8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalyticsEngine.java
@@ -31,13 +31,13 @@ import org.apache.nifi.web.api.dto.status.StatusSnapshotDTO;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class StatusAnalyticEngine implements StatusAnalytics {
+public class ConnectionStatusAnalyticsEngine implements StatusAnalyticsEngine {
     private ComponentStatusRepository statusRepository;
     private FlowController controller;
 
-    private static final Logger LOG = LoggerFactory.getLogger(StatusAnalyticEngine.class);
+    private static final Logger LOG = LoggerFactory.getLogger(ConnectionStatusAnalyticsEngine.class);
 
-    public StatusAnalyticEngine(FlowController controller, ComponentStatusRepository statusRepository) {
+    public ConnectionStatusAnalyticsEngine(FlowController controller, ComponentStatusRepository statusRepository) {
         this.controller = controller;
         this.statusRepository = statusRepository;
     }
@@ -54,7 +54,7 @@ public class StatusAnalyticEngine implements StatusAnalytics {
      * @return
      */
     public ConnectionStatusAnalytics getConnectionStatusAnalytics(Connection conn) {
-        LOG.info("Getting connection history for: " + conn.getIdentifier());
+        LOG.debug("Getting connection history for: " + conn.getIdentifier());
         long connTimeToBackpressure;
         Date minDate = new Date(System.currentTimeMillis() - (5 * 60 * 1000));
         StatusHistoryDTO connHistory = StatusHistoryUtil.createStatusHistoryDTO(
@@ -67,8 +67,7 @@ public class StatusAnalyticEngine implements StatusAnalytics {
         } else {
 
             long backPressureObjectThreshold = conn.getFlowFileQueue().getBackPressureObjectThreshold();
-            LOG.info("Connection " + conn.getIdentifier() + " backpressure object threshold is "
-                    + Long.toString(backPressureObjectThreshold));
+            LOG.info("Connection " + conn.getIdentifier() + " backpressure object threshold is " + backPressureObjectThreshold);
 
             ConnectionStatusDescriptor.QUEUED_COUNT.getField();
 
@@ -154,40 +153,4 @@ public class StatusAnalyticEngine implements StatusAnalytics {
             }
         };
     }
-
-    public long getMinTimeToBackpressureMillis() {
-        ProcessGroup rootGroup = controller.getFlowManager().getRootGroup();
-        List<Connection> allConnections = rootGroup.findAllConnections();
-        rootGroup.findConnection("asdf");
-        long minTimeToBackpressure = Long.MAX_VALUE;
-
-        for (Connection conn : allConnections) {
-            ConnectionStatusAnalytics connAnalytics = getConnectionStatusAnalytics(conn);
-            minTimeToBackpressure = Math.min(minTimeToBackpressure, connAnalytics.getTimeToCountBackpressureMillis());
-        }
-
-        LOG.info("Min time to backpressure is: " + Long.toString(minTimeToBackpressure));
-        return minTimeToBackpressure;
-    }
-
-    // TODO - populate the prediction fields. Do we need to pass in connection ID?
-    @Override
-    public long getTimeToCountBackpressureMillis(String connectionId) {
-        return 0;
-    }
-
-    @Override
-    public long getTimeToBytesBackpressureMillis(String connectionId) {
-        return 0;
-    }
-
-    @Override
-    public long getNextIntervalBytes(String connectionId) {
-        return 0;
-    }
-
-    @Override
-    public int getNextIntervalCount(String connectionId) {
-        return 0;
-    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java
index 87fcd4d..33e650b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java
@@ -52,6 +52,7 @@ import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
 import org.apache.nifi.controller.status.RunStatus;
 import org.apache.nifi.controller.status.TransmissionStatus;
 import org.apache.nifi.controller.status.analytics.StatusAnalytics;
+import org.apache.nifi.controller.status.analytics.StatusAnalyticsEngine;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.history.History;
@@ -65,12 +66,12 @@ import org.apache.nifi.remote.RemoteGroupPort;
 public class StandardEventAccess implements UserAwareEventAccess {
     private final FlowFileEventRepository flowFileEventRepository;
     private final FlowController flowController;
-    private final StatusAnalytics statusAnalytics;
+    private final StatusAnalyticsEngine statusAnalyticsEngine;
 
     public StandardEventAccess(final FlowController flowController, final FlowFileEventRepository flowFileEventRepository) {
         this.flowController = flowController;
         this.flowFileEventRepository = flowFileEventRepository;
-        this.statusAnalytics = flowController.getStatusAnalytics();
+        this.statusAnalyticsEngine = flowController.getStatusAnalyticsEngine();
     }
 
     /**
@@ -341,11 +342,14 @@ public class StandardEventAccess implements UserAwareEventAccess {
                 bytesTransferred += connectionStatusReport.getContentSizeIn() + connectionStatusReport.getContentSizeOut();
             }
 
-            if (statusAnalytics != null) {
-                connStatus.setPredictedTimeToBytesBackpressureMillis(statusAnalytics.getTimeToBytesBackpressureMillis(conn.getIdentifier()));
-                connStatus.setPredictedTimeToCountBackpressureMillis(statusAnalytics.getTimeToCountBackpressureMillis(conn.getIdentifier()));
-                connStatus.setNextPredictedQueuedBytes(statusAnalytics.getNextIntervalBytes(conn.getIdentifier()));
-                connStatus.setNextPredictedQueuedCount(statusAnalytics.getNextIntervalCount(conn.getIdentifier()));
+            if (statusAnalyticsEngine != null) {
+                StatusAnalytics statusAnalytics = statusAnalyticsEngine.getConnectionStatusAnalytics(conn.getIdentifier());
+                if (statusAnalytics != null) {
+                    connStatus.setPredictedTimeToBytesBackpressureMillis(statusAnalytics.getTimeToBytesBackpressureMillis());
+                    connStatus.setPredictedTimeToCountBackpressureMillis(statusAnalytics.getTimeToCountBackpressureMillis());
+                    connStatus.setNextPredictedQueuedBytes(statusAnalytics.getNextIntervalBytes());
+                    connStatus.setNextPredictedQueuedCount(statusAnalytics.getNextIntervalCount());
+                }
             }
 
             if (isConnectionAuthorized) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
index 367ea51..2fcef27 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
@@ -57,7 +57,7 @@ import org.apache.nifi.controller.status.ProcessGroupStatus;
 import org.apache.nifi.controller.status.ProcessorStatus;
 import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
 import org.apache.nifi.controller.status.analytics.ConnectionStatusAnalytics;
-import org.apache.nifi.controller.status.analytics.StatusAnalytics;
+import org.apache.nifi.controller.status.analytics.StatusAnalyticsEngine;
 import org.apache.nifi.controller.status.history.ComponentStatusRepository;
 import org.apache.nifi.diagnostics.SystemDiagnostics;
 import org.apache.nifi.flowfile.FlowFilePrioritizer;
@@ -705,7 +705,7 @@ public class ControllerFacade implements Authorizable {
         }
 
         // get from flow controller
-        final StatusAnalytics status = flowController.getStatusAnalytics();
+        final StatusAnalyticsEngine status = flowController.getStatusAnalyticsEngine();
         if (status == null) {
             throw new ResourceNotFoundException(String.format("Unable to locate connection with id '%s'.", connectionId));
         }


[nifi] 19/21: NIFI-6510 Fixed checkstyle issue in TestConnectionStatusAnalytics

Posted by ai...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

aichrist pushed a commit to branch analytics-framework
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 0c9e8edffbd1e4270502be9bee9570047f314321
Author: Matthew Burgess <ma...@apache.org>
AuthorDate: Fri Aug 2 13:30:58 2019 -0400

    NIFI-6510 Fixed checkstyle issue in TestConnectionStatusAnalytics
---
 .../controller/status/analytics/TestConnectionStatusAnalytics.java     | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalytics.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalytics.java
index 548c97e..1f5d520 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalytics.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalytics.java
@@ -48,7 +48,8 @@ public class TestConnectionStatusAnalytics {
             .map(ConnectionStatusDescriptor::getDescriptor)
             .collect(Collectors.toSet());
 
-    protected ConnectionStatusAnalytics getConnectionStatusAnalytics(Long queuedBytes, Long queuedCount, String backPressureDataSizeThreshhold, Long backPressureObjectThreshold, Boolean isConstantStatus) {
+    protected ConnectionStatusAnalytics getConnectionStatusAnalytics(Long queuedBytes, Long queuedCount, String backPressureDataSizeThreshhold,
+                                                                     Long backPressureObjectThreshold, Boolean isConstantStatus) {
         ComponentStatusRepository statusRepository = Mockito.mock(ComponentStatusRepository.class);
         FlowManager flowManager;
         flowManager = Mockito.mock(FlowManager.class);


[nifi] 17/21: NIFI-6150 Make checkstyle happy

Posted by ai...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

aichrist pushed a commit to branch analytics-framework
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 34e6191543df04cdb65179558fd06ca177c65064
Author: Yolanda Davis <yo...@gmail.com>
AuthorDate: Wed Jul 31 13:39:20 2019 -0400

    NIFI-6150 Make checkstyle happy
    
    (cherry picked from commit b6e35ac)
---
 ...TestCachingConnectionStatusAnalyticsEngine.java | 10 +--
 .../analytics/TestConnectionStatusAnalytics.java   | 75 +++++++++++++---------
 .../analytics/TestStatusAnalyticsEngine.java       |  6 +-
 3 files changed, 54 insertions(+), 37 deletions(-)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestCachingConnectionStatusAnalyticsEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestCachingConnectionStatusAnalyticsEngine.java
index abffafd..9fb67db 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestCachingConnectionStatusAnalyticsEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestCachingConnectionStatusAnalyticsEngine.java
@@ -20,20 +20,20 @@ import org.apache.nifi.controller.flow.FlowManager;
 import org.apache.nifi.controller.status.history.ComponentStatusRepository;
 import org.junit.Test;
 
-public class TestCachingConnectionStatusAnalyticsEngine extends TestStatusAnalyticsEngine{
+public class TestCachingConnectionStatusAnalyticsEngine extends TestStatusAnalyticsEngine {
 
     @Override
     public StatusAnalyticsEngine getStatusAnalyticsEngine(FlowManager flowManager, ComponentStatusRepository componentStatusRepository) {
-        return new CachingConnectionStatusAnalyticsEngine(flowManager,componentStatusRepository);
+        return new CachingConnectionStatusAnalyticsEngine(flowManager, componentStatusRepository);
     }
 
     @Test
-    public void testCachedStatusAnalytics(){
-        StatusAnalyticsEngine statusAnalyticsEngine = new CachingConnectionStatusAnalyticsEngine(flowManager,statusRepository);
+    public void testCachedStatusAnalytics() {
+        StatusAnalyticsEngine statusAnalyticsEngine = new CachingConnectionStatusAnalyticsEngine(flowManager, statusRepository);
         StatusAnalytics statusAnalyticsA = statusAnalyticsEngine.getStatusAnalytics("A");
         StatusAnalytics statusAnalyticsB = statusAnalyticsEngine.getStatusAnalytics("B");
         StatusAnalytics statusAnalyticsTest = statusAnalyticsEngine.getStatusAnalytics("A");
-        assert(statusAnalyticsA.equals(statusAnalyticsTest));
+        assert (statusAnalyticsA.equals(statusAnalyticsTest));
     }
 
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalytics.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalytics.java
index e17ae9a..548c97e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalytics.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalytics.java
@@ -1,3 +1,19 @@
+/*
+ * 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.analytics;
 
 import static org.junit.Assert.assertNotNull;
@@ -32,9 +48,10 @@ public class TestConnectionStatusAnalytics {
             .map(ConnectionStatusDescriptor::getDescriptor)
             .collect(Collectors.toSet());
 
-    protected ConnectionStatusAnalytics getConnectionStatusAnalytics(Long queuedBytes, Long queuedCount,String backPressureDataSizeThreshhold, Long backPressureObjectThreshold, Boolean isConstantStatus){
+    protected ConnectionStatusAnalytics getConnectionStatusAnalytics(Long queuedBytes, Long queuedCount, String backPressureDataSizeThreshhold, Long backPressureObjectThreshold, Boolean isConstantStatus) {
         ComponentStatusRepository statusRepository = Mockito.mock(ComponentStatusRepository.class);
-        FlowManager flowManager;flowManager = Mockito.mock(FlowManager.class);
+        FlowManager flowManager;
+        flowManager = Mockito.mock(FlowManager.class);
         final ProcessGroup processGroup = Mockito.mock(ProcessGroup.class);
         final StatusHistory statusHistory = Mockito.mock(StatusHistory.class);
         final Connection connection = Mockito.mock(Connection.class);
@@ -47,11 +64,11 @@ public class TestConnectionStatusAnalytics {
         final long startTime = System.currentTimeMillis();
         int iterations = 10;
 
-        for (int i=0; i < iterations; i++) {
+        for (int i = 0; i < iterations; i++) {
             final StandardStatusSnapshot snapshot = new StandardStatusSnapshot(CONNECTION_METRICS);
             snapshot.setTimestamp(new Date(startTime + i * 1000));
-            snapshot.addStatusMetric(ConnectionStatusDescriptor.QUEUED_BYTES.getDescriptor(), (isConstantStatus || i < 5) ? queuedBytes: queuedBytes * 2);
-            snapshot.addStatusMetric(ConnectionStatusDescriptor.QUEUED_COUNT.getDescriptor(), (isConstantStatus || i < 5) ? queuedCount: queuedCount * 2);
+            snapshot.addStatusMetric(ConnectionStatusDescriptor.QUEUED_BYTES.getDescriptor(), (isConstantStatus || i < 5) ? queuedBytes : queuedBytes * 2);
+            snapshot.addStatusMetric(ConnectionStatusDescriptor.QUEUED_COUNT.getDescriptor(), (isConstantStatus || i < 5) ? queuedCount : queuedCount * 2);
             snapshotList.add(snapshot);
         }
 
@@ -62,98 +79,98 @@ public class TestConnectionStatusAnalytics {
         when(processGroup.findAllConnections()).thenReturn(connections);
         when(statusHistory.getStatusSnapshots()).thenReturn(snapshotList);
         when(flowManager.getRootGroup()).thenReturn(processGroup);
-        when(statusRepository.getConnectionStatusHistory(anyString(),any(),any(),anyInt())).thenReturn(statusHistory);
-        ConnectionStatusAnalytics connectionStatusAnalytics = new ConnectionStatusAnalytics(statusRepository,flowManager,connectionIdentifier,false);
+        when(statusRepository.getConnectionStatusHistory(anyString(), any(), any(), anyInt())).thenReturn(statusHistory);
+        ConnectionStatusAnalytics connectionStatusAnalytics = new ConnectionStatusAnalytics(statusRepository, flowManager, connectionIdentifier, false);
         connectionStatusAnalytics.init();
         return connectionStatusAnalytics;
     }
 
     @Test
-    public void testGetIntervalTimeMillis(){
+    public void testGetIntervalTimeMillis() {
         ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, true);
         Long interval = connectionStatusAnalytics.getIntervalTimeMillis();
         assertNotNull(interval);
-        assert(interval == 300000);
+        assert (interval == 300000);
     }
 
     @Test
-    public void testGetTimeToCountBackpressureMillisConstantStatus(){
+    public void testGetTimeToCountBackpressureMillisConstantStatus() {
         ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, true);
         Long countTime = connectionStatusAnalytics.getTimeToCountBackpressureMillis();
         assertNotNull(countTime);
-        assert(countTime == -1L);
+        assert (countTime == -1L);
     }
 
     @Test
-    public void testGetTimeToCountBackpressureMillisVaryingStatus(){
+    public void testGetTimeToCountBackpressureMillisVaryingStatus() {
         ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, false);
         Long countTime = connectionStatusAnalytics.getTimeToCountBackpressureMillis();
         assertNotNull(countTime);
-        assert(countTime > -1L);
+        assert (countTime > -1L);
     }
 
     @Test
-    public void testGetTimeToBytesBackpressureMillisConstantStatus(){
+    public void testGetTimeToBytesBackpressureMillisConstantStatus() {
         ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, true);
         Long bytesTime = connectionStatusAnalytics.getTimeToBytesBackpressureMillis();
         assertNotNull(bytesTime);
-        assert(bytesTime == -1L);
+        assert (bytesTime == -1L);
     }
 
     @Test
-    public void testGetTimeToBytesBackpressureMillisVaryingStatus(){
+    public void testGetTimeToBytesBackpressureMillisVaryingStatus() {
         ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, false);
         Long bytesTime = connectionStatusAnalytics.getTimeToBytesBackpressureMillis();
         assertNotNull(bytesTime);
-        assert(bytesTime > -1L);
+        assert (bytesTime > -1L);
     }
 
     @Test
-    public void testGetNextIntervalBytesConstantStatus(){
+    public void testGetNextIntervalBytesConstantStatus() {
         ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, true);
         Long nextBytes = connectionStatusAnalytics.getNextIntervalBytes();
         assertNotNull(nextBytes);
-        assert(nextBytes == 5000L);
+        assert (nextBytes == 5000L);
     }
 
     @Test
-    public void testGetNextIntervalBytesVaryingStatus(){
+    public void testGetNextIntervalBytesVaryingStatus() {
         ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, false);
         Long nextBytes = connectionStatusAnalytics.getNextIntervalBytes();
         assertNotNull(nextBytes);
-        assert(nextBytes > -1L);
+        assert (nextBytes > -1L);
     }
 
     @Test
-    public void testGetNextIntervalCountConstantStatus(){
+    public void testGetNextIntervalCountConstantStatus() {
         ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, true);
         Long nextCount = connectionStatusAnalytics.getNextIntervalCount();
         assertNotNull(nextCount);
-        assert(nextCount == 50L);
+        assert (nextCount == 50L);
     }
 
     @Test
-    public void testGetNextIntervalCountVaryingStatus(){
+    public void testGetNextIntervalCountVaryingStatus() {
         ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, true);
         Long nextCount = connectionStatusAnalytics.getNextIntervalCount();
         assertNotNull(nextCount);
-        assert(nextCount == 50L);
+        assert (nextCount == 50L);
     }
 
     @Test
-    public void testGetNextIntervalPercentageUseBytesConstantStatus(){
+    public void testGetNextIntervalPercentageUseBytesConstantStatus() {
         ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(50000L, 50L, "1MB", 100L, true);
         Long nextBytesPercentage = connectionStatusAnalytics.getNextIntervalPercentageUseBytes();
         assertNotNull(nextBytesPercentage);
-        assert(nextBytesPercentage == 5);
+        assert (nextBytesPercentage == 5);
     }
 
     @Test
-    public void testGetNextIntervalPercentageUseCountConstantStatus(){
+    public void testGetNextIntervalPercentageUseCountConstantStatus() {
         ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(5000L, 50L, "100MB", 100L, true);
         Long nextCountPercentage = connectionStatusAnalytics.getNextIntervalPercentageUseCount();
         assertNotNull(nextCountPercentage);
-        assert(nextCountPercentage == 50);
+        assert (nextCountPercentage == 50);
     }
 
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestStatusAnalyticsEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestStatusAnalyticsEngine.java
index dfaaa02..183cd27 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestStatusAnalyticsEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestStatusAnalyticsEngine.java
@@ -48,12 +48,12 @@ public abstract class TestStatusAnalyticsEngine {
         StatusSnapshot statusSnapshot = Mockito.mock(StatusSnapshot.class);
         when(statusSnapshot.getMetricDescriptors()).thenReturn(Collections.emptySet());
         when(flowManager.getRootGroup()).thenReturn(processGroup);
-        when(statusRepository.getConnectionStatusHistory(anyString(),any(),any(),anyInt())).thenReturn(statusHistory);
+        when(statusRepository.getConnectionStatusHistory(anyString(), any(), any(), anyInt())).thenReturn(statusHistory);
     }
 
     @Test
-    public void testGetStatusAnalytics(){
-        StatusAnalyticsEngine statusAnalyticsEngine = getStatusAnalyticsEngine(flowManager,statusRepository);
+    public void testGetStatusAnalytics() {
+        StatusAnalyticsEngine statusAnalyticsEngine = getStatusAnalyticsEngine(flowManager, statusRepository);
         StatusAnalytics statusAnalytics = statusAnalyticsEngine.getStatusAnalytics("1");
         assertNotNull(statusAnalytics);
     }


[nifi] 14/21: NIFI-6510 Changes to inject flowManager instead of flow controller, also changes to properly reflect when predictions can be made vs not.

Posted by ai...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

aichrist pushed a commit to branch analytics-framework
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 6c5eda124b2b77c62ca455d66230a08f4befec5e
Author: Yolanda Davis <yo...@gmail.com>
AuthorDate: Wed Jul 31 08:57:15 2019 -0400

    NIFI-6510 Changes to inject flowManager instead of flow controller, also changes to properly reflect when predictions can be made vs not.
    
    (cherry picked from commit 6fae058)
---
 .../org/apache/nifi/controller/FlowController.java |  2 +-
 .../CachingConnectionStatusAnalyticsEngine.java    | 16 ++--
 .../analytics/ConnectionStatusAnalytics.java       | 89 +++++++++++++---------
 .../analytics/ConnectionStatusAnalyticsEngine.java | 11 ++-
 .../status/analytics/SimpleRegressionBSAM.java     | 18 ++++-
 5 files changed, 81 insertions(+), 55 deletions(-)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index b5de777..141a6a4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -594,7 +594,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
         }
 
         componentStatusRepository = createComponentStatusRepository();
-        analyticsEngine = new CachingConnectionStatusAnalyticsEngine(this, componentStatusRepository);
+        analyticsEngine = new CachingConnectionStatusAnalyticsEngine(flowManager, componentStatusRepository);
         eventAccess = new StandardEventAccess(this, flowFileEventRepository);
 
         timerDrivenEngineRef.get().scheduleWithFixedDelay(new Runnable() {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingConnectionStatusAnalyticsEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingConnectionStatusAnalyticsEngine.java
index c4836c6..f69ed33 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingConnectionStatusAnalyticsEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingConnectionStatusAnalyticsEngine.java
@@ -18,7 +18,7 @@ package org.apache.nifi.controller.status.analytics;
 
 import java.util.concurrent.TimeUnit;
 
-import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.controller.flow.FlowManager;
 import org.apache.nifi.controller.status.history.ComponentStatusRepository;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -28,15 +28,15 @@ import com.github.benmanes.caffeine.cache.Caffeine;
 
 public class CachingConnectionStatusAnalyticsEngine implements StatusAnalyticsEngine {
     private ComponentStatusRepository statusRepository;
-    private FlowController controller;
+    private FlowManager flowManager;
     private volatile Cache<String, ConnectionStatusAnalytics> cache;
     private static final Logger LOG = LoggerFactory.getLogger(CachingConnectionStatusAnalyticsEngine.class);
 
-    public CachingConnectionStatusAnalyticsEngine(FlowController controller, ComponentStatusRepository statusRepository) {
-        this.controller = controller;
+    public CachingConnectionStatusAnalyticsEngine(FlowManager flowManager, ComponentStatusRepository statusRepository) {
+        this.flowManager = flowManager;
         this.statusRepository = statusRepository;
         this.cache = Caffeine.newBuilder()
-                .expireAfterWrite(5, TimeUnit.MINUTES)
+                .expireAfterWrite(30, TimeUnit.MINUTES)
                 .build();
     }
 
@@ -45,12 +45,12 @@ public class CachingConnectionStatusAnalyticsEngine implements StatusAnalyticsEn
 
         ConnectionStatusAnalytics connectionStatusAnalytics = cache.getIfPresent(identifier);
         if(connectionStatusAnalytics == null){
-            LOG.info("Creating new analytics for connection id: {0}", identifier);
-            connectionStatusAnalytics = new ConnectionStatusAnalytics(statusRepository,controller,identifier);
+            LOG.debug("Creating new status analytics object for connection id: {}", identifier);
+            connectionStatusAnalytics = new ConnectionStatusAnalytics(statusRepository,flowManager,identifier,true);
             connectionStatusAnalytics.init();
             cache.put(identifier,connectionStatusAnalytics);
         }else{
-            LOG.info("Pulled existing analytics from cache for connection id: {}", identifier);
+            LOG.debug("Pulled existing analytics from cache for connection id: {}", identifier);
             connectionStatusAnalytics.refresh();
         }
         return connectionStatusAnalytics;
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
index bad2ff1..8b7964e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
@@ -25,7 +25,7 @@ import java.util.Optional;
 import java.util.stream.Stream;
 
 import org.apache.nifi.connectable.Connection;
-import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.controller.flow.FlowManager;
 import org.apache.nifi.controller.status.history.ComponentStatusRepository;
 import org.apache.nifi.controller.status.history.ConnectionStatusDescriptor;
 import org.apache.nifi.controller.status.history.StatusHistory;
@@ -45,24 +45,27 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
     private QueryWindow queryWindow;
     private final ComponentStatusRepository componentStatusRepository;
     private final String connectionIdentifier;
-    private final FlowController flowController;
+    private final FlowManager flowManager;
+    private final Boolean supportOnlineLearning;
 
-    public ConnectionStatusAnalytics(ComponentStatusRepository componentStatusRepository, FlowController flowController, String connectionIdentifier) {
+    public ConnectionStatusAnalytics(ComponentStatusRepository componentStatusRepository, FlowManager flowManager, String connectionIdentifier, Boolean supportOnlineLearning) {
         this.componentStatusRepository = componentStatusRepository;
-        this.flowController = flowController;
+        this.flowManager = flowManager;
         this.connectionIdentifier = connectionIdentifier;
+        this.supportOnlineLearning = supportOnlineLearning;
     }
 
     public void init() {
 
+        LOG.debug("Initialize analytics connection id: {} ", connectionIdentifier);
+
         if (this.modelMap == null || this.modelMap.isEmpty()) {
-            Tuple<StatusAnalyticsModel, ExtractFunction> countModelFunction = new Tuple<>(new SimpleRegressionBSAM(), extract);
-            Tuple<StatusAnalyticsModel, ExtractFunction> byteModelFunction = new Tuple<>(new SimpleRegressionBSAM(), extract);
+            Tuple<StatusAnalyticsModel, ExtractFunction> countModelFunction = new Tuple<>(new SimpleRegressionBSAM(!supportsOnlineLearning()), extract);
+            Tuple<StatusAnalyticsModel, ExtractFunction> byteModelFunction = new Tuple<>(new SimpleRegressionBSAM(!supportsOnlineLearning()), extract);
             this.modelMap = new HashMap<>();
             //TODO: Should change keys used here
             this.modelMap.put(ConnectionStatusDescriptor.QUEUED_COUNT.getField(), countModelFunction);
             this.modelMap.put(ConnectionStatusDescriptor.QUEUED_BYTES.getField(), byteModelFunction);
-            this.queryWindow = new QueryWindow(System.currentTimeMillis() - (5 * 60 * 1000), System.currentTimeMillis());
         }
 
         refresh();
@@ -70,17 +73,20 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
 
     public void refresh() {
 
+
+        LOG.debug("Refreshing model with new data for connection id: {} ", connectionIdentifier);
+
+        this.queryWindow = new QueryWindow(System.currentTimeMillis() - getIntervalTimeMillis(), System.currentTimeMillis());
         modelMap.forEach((metric, modelFunction) -> {
 
             StatusAnalyticsModel model = modelFunction.getKey();
             ExtractFunction extract = modelFunction.getValue();
             StatusHistory statusHistory = componentStatusRepository.getConnectionStatusHistory(connectionIdentifier, queryWindow.getStartDateTime(), queryWindow.getEndDateTime(), Integer.MAX_VALUE);
             Tuple<Stream<Double>, Stream<Double>> modelData = extract.extractMetric(metric, statusHistory);
-            LOG.info("Refreshing model for connection id: {} ", connectionIdentifier);
             Stream<Double> times = modelData.getKey();
-            Stream<Double> counts = modelData.getValue();
-            //times is the X axis and counts is on the y axis
-            model.learn(times, counts);
+            Stream<Double> values = modelData.getValue();
+            //times are the X axis and values are on the y axis
+            model.learn(times, values);
 
         });
     }
@@ -90,7 +96,7 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
      *
      * @return milliseconds until backpressure is predicted to occur, based on the total number of bytes in the queue.
      */
-    public long getTimeToBytesBackpressureMillis() {
+    public Long getTimeToBytesBackpressureMillis() {
 
         final BivariateStatusAnalyticsModel bytesModel = (BivariateStatusAnalyticsModel) modelMap.get(ConnectionStatusDescriptor.QUEUED_BYTES.getField()).getKey();
         final Connection connection = getConnection();
@@ -100,10 +106,10 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
         final String backPressureDataSize = connection.getFlowFileQueue().getBackPressureDataSizeThreshold();
         final double backPressureBytes = DataUnit.parseDataSize(backPressureDataSize, DataUnit.B);
         final double prediction = bytesModel.predictX(backPressureBytes);
-        if (prediction != Double.NaN) {
+        if (prediction != Double.NaN && prediction >= System.currentTimeMillis() && !Double.isInfinite(prediction)) {
             return Math.max(0, Math.round(prediction) - System.currentTimeMillis());
         } else {
-            return Long.MAX_VALUE;
+            return -1L;
         }
 
     }
@@ -113,7 +119,7 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
      *
      * @return milliseconds until backpressure is predicted to occur, based on the number of objects in the queue.
      */
-    public long getTimeToCountBackpressureMillis() {
+    public Long getTimeToCountBackpressureMillis() {
 
         final BivariateStatusAnalyticsModel countModel = (BivariateStatusAnalyticsModel) modelMap.get(ConnectionStatusDescriptor.QUEUED_COUNT.getField()).getKey();
         final Connection connection = getConnection();
@@ -123,10 +129,10 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
         final double backPressureCountThreshold = connection.getFlowFileQueue().getBackPressureObjectThreshold();
         final Double prediction = countModel.predictX(backPressureCountThreshold);
 
-        if (prediction != Double.NaN) {
+        if (prediction != Double.NaN && prediction >= System.currentTimeMillis() && !Double.isInfinite(prediction)) {
             return Math.max(0, Math.round(prediction) - System.currentTimeMillis());
         } else {
-            return Long.MAX_VALUE;
+            return -1L;
         }
     }
 
@@ -136,13 +142,13 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
      * @return milliseconds until backpressure is predicted to occur, based on the total number of bytes in the queue.
      */
 
-    public long getNextIntervalBytes() {
+    public Long getNextIntervalBytes() {
         final BivariateStatusAnalyticsModel bytesModel = (BivariateStatusAnalyticsModel) modelMap.get(ConnectionStatusDescriptor.QUEUED_BYTES.getField()).getKey();
         final Double prediction = bytesModel.predictY((double) System.currentTimeMillis() + getIntervalTimeMillis());
-        if (prediction != Double.NaN) {
+        if (prediction != Double.NaN && prediction >= 0) {
             return Math.round(prediction);
         } else {
-            return 0;
+            return -1L;
         }
     }
 
@@ -152,29 +158,34 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
      * @return milliseconds until backpressure is predicted to occur, based on the number of bytes in the queue.
      */
 
-    public int getNextIntervalCount() {
+    public Long getNextIntervalCount() {
         final BivariateStatusAnalyticsModel countModel = (BivariateStatusAnalyticsModel) modelMap.get(ConnectionStatusDescriptor.QUEUED_COUNT.getField()).getKey();
         final Double prediction = countModel.predictY((double) System.currentTimeMillis() + getIntervalTimeMillis());
-        if (prediction != Double.NaN) {
-            return ((Long) Math.round(prediction)).intValue();
+        if (prediction != Double.NaN && prediction >= 0) {
+            return Math.round(prediction);
         } else {
-            return 0;
+            return -1L;
         }
     }
 
-    public int getNextIntervalPercentageUseCount(){
+    public Long getNextIntervalPercentageUseCount(){
 
         final Connection connection = getConnection();
         if (connection == null) {
             throw new NoSuchElementException("Connection with the following id cannot be found:" + connectionIdentifier + ". Model should be invalidated!");
         }
         final double backPressureCountThreshold = connection.getFlowFileQueue().getBackPressureObjectThreshold();
+        final long nextIntervalCount = getNextIntervalCount();
 
-        return ((Long)Math.round((getNextIntervalCount()/backPressureCountThreshold) * 100)).intValue();
+        if(nextIntervalCount > -1L) {
+            return Math.round((getNextIntervalCount() / backPressureCountThreshold) * 100);
+        }else{
+            return -1L;
+        }
 
     }
 
-    public int getNextIntervalPercentageUseBytes(){
+    public Long getNextIntervalPercentageUseBytes(){
 
         final Connection connection = getConnection();
         if (connection == null) {
@@ -182,13 +193,17 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
         }
         final String backPressureDataSize = connection.getFlowFileQueue().getBackPressureDataSizeThreshold();
         final double backPressureBytes = DataUnit.parseDataSize(backPressureDataSize, DataUnit.B);
+        final long nextIntervalBytes = getNextIntervalBytes();
 
-        return ((Long)Math.round((getNextIntervalBytes()/ backPressureBytes) * 100)).intValue();
-
+        if(nextIntervalBytes > -1L) {
+            return Math.round((getNextIntervalBytes() / backPressureBytes) * 100);
+        }else{
+            return -1L;
+        }
     }
 
-    public long getIntervalTimeMillis(){
-        return getQueryWindow().getTimeDifferenceMillis();
+    public Long getIntervalTimeMillis(){
+        return (5L * 60 * 1000);
     }
 
     @Override
@@ -206,13 +221,13 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
         predictions.put("timeToBytesBackpressureMillis", getTimeToBytesBackpressureMillis());
         predictions.put("timeToCountBackpressureMillis", getTimeToCountBackpressureMillis());
         predictions.put("nextIntervalBytes", getNextIntervalBytes());
-        predictions.put("nextIntervalCount", (long) getNextIntervalCount());
-        predictions.put("nextIntervalPercentageUseCount", (long)getNextIntervalPercentageUseCount());
-        predictions.put("nextIntervalPercentageUseBytes", (long)getNextIntervalPercentageUseBytes());
+        predictions.put("nextIntervalCount", getNextIntervalCount());
+        predictions.put("nextIntervalPercentageUseCount", getNextIntervalPercentageUseCount());
+        predictions.put("nextIntervalPercentageUseBytes", getNextIntervalPercentageUseBytes());
         predictions.put("intervalTimeMillis", getIntervalTimeMillis());
 
         predictions.forEach((key,value) -> {
-            LOG.info("Prediction model for connection id {}: {}={} ", connectionIdentifier,key,value);
+            LOG.debug("Prediction model for connection id {}: {}={} ", connectionIdentifier,key,value);
         });
 
         return predictions;
@@ -220,11 +235,11 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
 
     @Override
     public boolean supportsOnlineLearning() {
-        return true;
+        return supportOnlineLearning;
     }
 
     private Connection getConnection() {
-        final ProcessGroup rootGroup = flowController.getFlowManager().getRootGroup();
+        final ProcessGroup rootGroup = flowManager.getRootGroup();
         Optional<Connection> connection = rootGroup.findAllConnections().stream().filter(c -> c.getIdentifier().equals(this.connectionIdentifier)).findFirst();
         return connection.orElse(null);
     }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalyticsEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalyticsEngine.java
index 7f9db25..8f1222c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalyticsEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalyticsEngine.java
@@ -16,28 +16,27 @@
  */
 package org.apache.nifi.controller.status.analytics;
 
-import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.controller.flow.FlowManager;
 import org.apache.nifi.controller.status.history.ComponentStatusRepository;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class ConnectionStatusAnalyticsEngine implements StatusAnalyticsEngine {
     private ComponentStatusRepository statusRepository;
-    private FlowController controller;
+    private FlowManager flowManager;
 
     private static final Logger LOG = LoggerFactory.getLogger(ConnectionStatusAnalyticsEngine.class);
 
-    public ConnectionStatusAnalyticsEngine(FlowController controller, ComponentStatusRepository statusRepository) {
-        this.controller = controller;
+    public ConnectionStatusAnalyticsEngine(FlowManager flowManager, ComponentStatusRepository statusRepository) {
+        this.flowManager = flowManager;
         this.statusRepository = statusRepository;
     }
 
     @Override
     public StatusAnalytics getStatusAnalytics(String identifier) {
-        ConnectionStatusAnalytics connectionStatusAnalytics = new ConnectionStatusAnalytics(statusRepository,controller,identifier);
+        ConnectionStatusAnalytics connectionStatusAnalytics = new ConnectionStatusAnalytics(statusRepository,flowManager,identifier,false);
         connectionStatusAnalytics.init();
         return connectionStatusAnalytics;
     }
 
-
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/SimpleRegressionBSAM.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/SimpleRegressionBSAM.java
index 8aa4a45..21c0370 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/SimpleRegressionBSAM.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/SimpleRegressionBSAM.java
@@ -21,21 +21,33 @@ import java.util.stream.Stream;
 
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.commons.math3.stat.regression.SimpleRegression;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class SimpleRegressionBSAM extends BivariateStatusAnalyticsModel {
 
-    private SimpleRegression regression;
+    private static final Logger LOG = LoggerFactory.getLogger(SimpleRegressionBSAM.class);
+    private final SimpleRegression regression;
+    private final Boolean clearObservationsOnLearn;
+
+    public SimpleRegressionBSAM(Boolean clearObservationsOnLearn) {
 
-    public SimpleRegressionBSAM() {
         this.regression = new SimpleRegression();
+        this.clearObservationsOnLearn = clearObservationsOnLearn;
     }
 
     @Override
     public void learn(Stream<Double> features, Stream<Double> labels) {
         double[] labelArray = ArrayUtils.toPrimitive(labels.toArray(Double[]::new));
         double[][] featuresMatrix = features.map(feature -> new double[]{feature}).toArray(double[][]::new);
-        regression.clear();
+
+        if(clearObservationsOnLearn) {
+            regression.clear();
+        }
+
         regression.addObservations(featuresMatrix, labelArray);
+        LOG.debug("Model is using equation: y = {}x + {}", regression.getSlope(), regression.getIntercept());
+
     }
 
     @Override


[nifi] 12/21: NIFI-6510 Analytics Framework Introduction (#10)

Posted by ai...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

aichrist pushed a commit to branch analytics-framework
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit f3824a0661ae7c32a3b5601ba75e4ec5a11e8f4a
Author: Yolanda Davis <yo...@gmail.com>
AuthorDate: Mon Jul 29 07:16:21 2019 -0400

    NIFI-6510 Analytics Framework Introduction (#10)
    
    * DFA-9 - Initial refactor for Status Analytics - created additional interfaces for models, refactored callers to use StatusAnalytics objects with connection context. Implemented SimpleRegression model.
    
    DFA-9 - added logging
    
    * DFA-9 - relocated query window to CSA from model, adding the prediction percentages and time interval
    
    * DFA-9 - checkstyle fixes
---
 .../nifi/controller/status/ConnectionStatus.java   |  19 ++
 .../controller/status/analytics/QueryWindow.java   |  59 +++++
 .../status/analytics/StatusAnalytics.java          |  26 +--
 .../status/analytics/StatusAnalyticsEngine.java    |   3 +-
 ...tusAnalytics.java => StatusAnalyticsModel.java} |  17 +-
 .../status/ConnectionStatisticsSnapshotDTO.java    |  37 ++-
 .../analytics/BivariateStatusAnalyticsModel.java   |  30 ++-
 .../CachingConnectionStatusAnalyticsEngine.java    | 147 ++----------
 .../analytics/ConnectionStatusAnalytics.java       | 254 +++++++++++++++++++++
 .../analytics/ConnectionStatusAnalyticsEngine.java | 121 +---------
 .../status/analytics/SimpleRegressionBSAM.java     |  55 +++++
 .../apache/nifi/reporting/StandardEventAccess.java |  15 +-
 .../apache/nifi/web/StandardNiFiServiceFacade.java |  60 ++---
 .../org/apache/nifi/web/api/dto/DtoFactory.java    |  98 ++++----
 .../nifi/web/controller/ControllerFacade.java      |  59 ++---
 15 files changed, 595 insertions(+), 405 deletions(-)

diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/ConnectionStatus.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/ConnectionStatus.java
index ee7dd45..783677a 100644
--- a/nifi-api/src/main/java/org/apache/nifi/controller/status/ConnectionStatus.java
+++ b/nifi-api/src/main/java/org/apache/nifi/controller/status/ConnectionStatus.java
@@ -45,6 +45,8 @@ public class ConnectionStatus implements Cloneable {
     private long nextPredictedQueuedBytes;
     private long predictedTimeToCountBackpressureMillis;
     private long predictedTimeToBytesBackpressureMillis;
+    private int predictedPercentCount = 0;
+    private int predictedPercentBytes = 0;
 
     public String getId() {
         return id;
@@ -231,6 +233,23 @@ public class ConnectionStatus implements Cloneable {
         this.predictedTimeToBytesBackpressureMillis = predictedTimeToBytesBackpressureMillis;
     }
 
+    public int getPredictedPercentCount() {
+        return predictedPercentCount;
+    }
+
+    public void setPredictedPercentCount(int predictedPercentCount) {
+        this.predictedPercentCount = predictedPercentCount;
+    }
+
+    public int getPredictedPercentBytes() {
+        return predictedPercentBytes;
+    }
+
+    public void setPredictedPercentBytes(int predictedPercentBytes) {
+        this.predictedPercentBytes = predictedPercentBytes;
+    }
+
+
     @Override
     public ConnectionStatus clone() {
         final ConnectionStatus clonedObj = new ConnectionStatus();
diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/QueryWindow.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/QueryWindow.java
new file mode 100644
index 0000000..c477872
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/QueryWindow.java
@@ -0,0 +1,59 @@
+/*
+ * 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.analytics;
+
+import java.util.Date;
+
+public class QueryWindow {
+
+    private long startTimeMillis;
+    private long endTimeMillis;
+
+    public QueryWindow(long startTimeMillis, long endTimeMillis) {
+        this.startTimeMillis = startTimeMillis;
+        this.endTimeMillis = endTimeMillis;
+    }
+
+    public long getStartTimeMillis() {
+        return startTimeMillis;
+    }
+
+    public void setStartTimeMillis(long startTimeMillis) {
+        this.startTimeMillis = startTimeMillis;
+    }
+
+    public long getEndTimeMillis() {
+        return endTimeMillis;
+    }
+
+    public void setEndTimeMillis(long endTimeMillis) {
+        this.endTimeMillis = endTimeMillis;
+    }
+
+    public Date getStartDateTime() {
+        return new Date(startTimeMillis);
+    }
+
+    public Date getEndDateTime() {
+        return new Date(endTimeMillis);
+    }
+
+    public long getTimeDifferenceMillis(){
+        return endTimeMillis - startTimeMillis;
+    }
+
+}
diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java
index 564f1c9..a65629f 100644
--- a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java
+++ b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java
@@ -16,32 +16,16 @@
  */
 package org.apache.nifi.controller.status.analytics;
 
+import java.util.Map;
+
 /**
  * The StatusAnalytics interface offers methods for accessing predicted and other values for a single component (Connection instance, e.g.)
  */
 public interface StatusAnalytics {
 
-    /**
-     * Returns the predicted time (in milliseconds) when backpressure is expected to be applied to this connection, based on the total number of bytes in the queue.
-     * @return milliseconds until backpressure is predicted to occur, based on the total number of bytes in the queue.
-     */
-    long getTimeToBytesBackpressureMillis();
-
-    /**
-     * Returns the predicted time (in milliseconds) when backpressure is expected to be applied to this connection, based on the number of objects in the queue.
-     * @return milliseconds until backpressure is predicted to occur, based on the number of objects in the queue.
-     */
-    long getTimeToCountBackpressureMillis();
+    QueryWindow getQueryWindow();
+    Map<String,Long> getPredictions();
+    boolean supportsOnlineLearning();
 
-    /**
-     * Returns the predicted total number of bytes in the queue to occur at the next configured interval (5 mins in the future, e.g.).
-     * @return milliseconds until backpressure is predicted to occur, based on the total number of bytes in the queue.
-     */
-    long getNextIntervalBytes();
 
-    /**
-     * Returns the predicted number of objects in the queue to occur at the next configured interval (5 mins in the future, e.g.).
-     * @return milliseconds until backpressure is predicted to occur, based on the number of bytes in the queue.
-     */
-    int getNextIntervalCount();
 }
diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticsEngine.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticsEngine.java
index 5cbc333..01021c2 100644
--- a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticsEngine.java
+++ b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticsEngine.java
@@ -18,5 +18,6 @@ package org.apache.nifi.controller.status.analytics;
 
 public interface StatusAnalyticsEngine {
 
-    ConnectionStatusAnalytics getConnectionStatusAnalytics(String connectionId);
+    StatusAnalytics getStatusAnalytics(String componentId);
+
 }
diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticsModel.java
similarity index 66%
copy from nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
copy to nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticsModel.java
index 9792ae4..72c81b1 100644
--- a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
+++ b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticsModel.java
@@ -16,17 +16,12 @@
  */
 package org.apache.nifi.controller.status.analytics;
 
+import java.util.stream.Stream;
 
-/**
- * The ConnectionStatusAnalytics interface offers additional methods to the StatusAnalytics interface related to the supporting connection information (group ID, e.g.)
- */
-public interface ConnectionStatusAnalytics extends StatusAnalytics{
+public interface StatusAnalyticsModel {
+
+    void learn(Stream<Double> features, Stream<Double> labels);
+    Double predict(Double feature);
+    Boolean supportsOnlineLearning();
 
-    String getGroupId();
-    String getId();
-    String getName();
-    String getSourceId();
-    String getSourceName();
-    String getDestinationId();
-    String getDestinationName();
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatisticsSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatisticsSnapshotDTO.java
index 526bdcf..a521db4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatisticsSnapshotDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatisticsSnapshotDTO.java
@@ -16,10 +16,10 @@
  */
 package org.apache.nifi.web.api.dto.status;
 
-import io.swagger.annotations.ApiModelProperty;
-
 import javax.xml.bind.annotation.XmlType;
 
+import io.swagger.annotations.ApiModelProperty;
+
 /**
  * DTO for serializing the statistics of a connection.
  */
@@ -39,6 +39,9 @@ public class ConnectionStatisticsSnapshotDTO implements Cloneable {
     private Long predictedMillisUntilBytesBackpressure = 0L;
     private Integer predictedCountAtNextInterval = 0;
     private Long predictedBytesAtNextInterval = 0L;
+    private Integer predictedPercentCount = 0;
+    private Integer predictedPercentBytes = 0;
+    private Long predictionIntervalMillis = 0L;
 
     /* getters / setters */
     /**
@@ -161,6 +164,33 @@ public class ConnectionStatisticsSnapshotDTO implements Cloneable {
         this.predictedBytesAtNextInterval = predictedBytesAtNextInterval;
     }
 
+    @ApiModelProperty("The predicted percentage of queued objects at the next configured interval.")
+    public Integer getPredictedPercentCount() {
+        return predictedPercentCount;
+    }
+
+    public void setPredictedPercentCount(Integer predictedPercentCount) {
+        this.predictedPercentCount = predictedPercentCount;
+    }
+
+    @ApiModelProperty("The predicted percentage of bytes in the queue against current threshold at the next configured interval.")
+    public Integer getPredictedPercentBytes() {
+        return predictedPercentBytes;
+    }
+
+    public void setPredictedPercentBytes(Integer predictedPercentBytes) {
+        this.predictedPercentBytes = predictedPercentBytes;
+    }
+
+    @ApiModelProperty("The prediction interval in seconds")
+    public Long getPredictionIntervalMillis() {
+        return predictionIntervalMillis;
+    }
+
+    public void setPredictionIntervalMillis(Long predictionIntervalMillis) {
+        this.predictionIntervalMillis = predictionIntervalMillis;
+    }
+
     @Override
     public ConnectionStatisticsSnapshotDTO clone() {
         final ConnectionStatisticsSnapshotDTO other = new ConnectionStatisticsSnapshotDTO();
@@ -176,6 +206,9 @@ public class ConnectionStatisticsSnapshotDTO implements Cloneable {
         other.setPredictedMillisUntilBytesBackpressure(getPredictedMillisUntilBytesBackpressure());
         other.setPredictedCountAtNextInterval(getPredictedCountAtNextInterval());
         other.setPredictedBytesAtNextInterval(getPredictedBytesAtNextInterval());
+        other.setPredictedPercentCount(getPredictedPercentCount());
+        other.setPredictedPercentBytes(getPredictedPercentBytes());
+        other.setPredictionIntervalMillis(getPredictionIntervalMillis());
 
         return other;
     }
diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/BivariateStatusAnalyticsModel.java
similarity index 66%
rename from nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/BivariateStatusAnalyticsModel.java
index 9792ae4..eff661b 100644
--- a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/BivariateStatusAnalyticsModel.java
@@ -16,17 +16,23 @@
  */
 package org.apache.nifi.controller.status.analytics;
 
+import java.util.stream.Stream;
+
+public abstract class BivariateStatusAnalyticsModel implements StatusAnalyticsModel {
+
+
+    public abstract void learn(Stream<Double> features, Stream<Double> labels);
+
+    public abstract Double predict(Double feature);
+
+    public abstract Double predictX(Double y);
+
+    public abstract Double predictY(Double x);
+
+    @Override
+    public Boolean supportsOnlineLearning() {
+        return false;
+    }
+
 
-/**
- * The ConnectionStatusAnalytics interface offers additional methods to the StatusAnalytics interface related to the supporting connection information (group ID, e.g.)
- */
-public interface ConnectionStatusAnalytics extends StatusAnalytics{
-
-    String getGroupId();
-    String getId();
-    String getName();
-    String getSourceId();
-    String getSourceName();
-    String getDestinationId();
-    String getDestinationName();
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingConnectionStatusAnalyticsEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingConnectionStatusAnalyticsEngine.java
index c12dbae..c4836c6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingConnectionStatusAnalyticsEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingConnectionStatusAnalyticsEngine.java
@@ -16,19 +16,10 @@
  */
 package org.apache.nifi.controller.status.analytics;
 
-import java.util.Date;
-import java.util.List;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.commons.math3.stat.regression.SimpleRegression;
-import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.status.history.ComponentStatusRepository;
-import org.apache.nifi.controller.status.history.ConnectionStatusDescriptor;
-import org.apache.nifi.controller.status.history.StatusHistoryUtil;
-import org.apache.nifi.groups.ProcessGroup;
-import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
-import org.apache.nifi.web.api.dto.status.StatusSnapshotDTO;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -38,141 +29,33 @@ import com.github.benmanes.caffeine.cache.Caffeine;
 public class CachingConnectionStatusAnalyticsEngine implements StatusAnalyticsEngine {
     private ComponentStatusRepository statusRepository;
     private FlowController controller;
-    private volatile Cache<String, SimpleRegression> cache;
-    private static final Logger LOG = LoggerFactory.getLogger(ConnectionStatusAnalyticsEngine.class);
+    private volatile Cache<String, ConnectionStatusAnalytics> cache;
+    private static final Logger LOG = LoggerFactory.getLogger(CachingConnectionStatusAnalyticsEngine.class);
 
     public CachingConnectionStatusAnalyticsEngine(FlowController controller, ComponentStatusRepository statusRepository) {
         this.controller = controller;
         this.statusRepository = statusRepository;
         this.cache = Caffeine.newBuilder()
-                .expireAfterWrite(1, TimeUnit.MINUTES)
+                .expireAfterWrite(5, TimeUnit.MINUTES)
                 .build();
     }
 
     @Override
-    public ConnectionStatusAnalytics getConnectionStatusAnalytics(String connectionId) {
-
-        ProcessGroup rootGroup = controller.getFlowManager().getRootGroup();
-        Connection connection = rootGroup.findConnection(connectionId);
-        SimpleRegression cachedRegression = cache.getIfPresent(connection.getIdentifier());
-
-        if (cachedRegression == null) {
-            cachedRegression = getBackPressureRegressionModel(connection);
-            if (cachedRegression != null)
-                cache.put(connection.getIdentifier(), cachedRegression);
+    public StatusAnalytics getStatusAnalytics(String identifier) {
+
+        ConnectionStatusAnalytics connectionStatusAnalytics = cache.getIfPresent(identifier);
+        if(connectionStatusAnalytics == null){
+            LOG.info("Creating new analytics for connection id: {0}", identifier);
+            connectionStatusAnalytics = new ConnectionStatusAnalytics(statusRepository,controller,identifier);
+            connectionStatusAnalytics.init();
+            cache.put(identifier,connectionStatusAnalytics);
+        }else{
+            LOG.info("Pulled existing analytics from cache for connection id: {}", identifier);
+            connectionStatusAnalytics.refresh();
         }
-
-        ConnectionStatusAnalytics cachedResult = calculate(cachedRegression, connection);
-        LOG.info("Connection: " + connectionId + " Cached backpressure Time: " + cachedResult.getTimeToCountBackpressureMillis());
-        return cachedResult;
-    }
-
-    protected ConnectionStatusAnalytics calculate(SimpleRegression regression, Connection conn) {
-        long backPressureObjectThreshold = conn.getFlowFileQueue().getBackPressureObjectThreshold();
-
-        final long connTimeToBackpressure;
-
-        if (regression == null) {
-            connTimeToBackpressure = Long.MAX_VALUE;
-        } else {
-            //If calculation returns as negative only 0 will return
-            connTimeToBackpressure = Math.max(0, Math.round((backPressureObjectThreshold - regression.getIntercept()) / regression.getSlope())
-                    - System.currentTimeMillis());
-        }
-
-        return new ConnectionStatusAnalytics() {
-
-            @Override
-            public String getSourceName() {
-                return conn.getSource().getName();
-            }
-
-            @Override
-            public String getSourceId() {
-                return conn.getSource().getIdentifier();
-            }
-
-            @Override
-            public String getName() {
-                return conn.getName();
-            }
-
-
-            @Override
-            public String getId() {
-                return conn.getIdentifier();
-            }
-
-            @Override
-            public long getTimeToBytesBackpressureMillis() {
-                return 0;
-            }
-
-            @Override
-            public long getTimeToCountBackpressureMillis() {
-                return connTimeToBackpressure;
-            }
-
-            @Override
-            public long getNextIntervalBytes() {
-                return 0;
-            }
-
-            @Override
-            public int getNextIntervalCount() {
-                return 0;
-            }
-
-            @Override
-            public String getGroupId() {
-                return conn.getProcessGroup().getIdentifier();
-            }
-
-            @Override
-            public String getDestinationName() {
-                return conn.getDestination().getName();
-            }
-
-            @Override
-            public String getDestinationId() {
-                return conn.getDestination().getIdentifier();
-            }
-        };
+        return connectionStatusAnalytics;
 
     }
 
-    /**
-     * Get backpressure model based on current data
-     *
-     * @param conn the connection to run the analytic on
-     * @return
-     */
-    protected SimpleRegression getBackPressureRegressionModel(Connection conn) {
-        Date minDate = new Date(System.currentTimeMillis() - (60 * 1000));
-        StatusHistoryDTO connHistory = StatusHistoryUtil.createStatusHistoryDTO(
-                statusRepository.getConnectionStatusHistory(conn.getIdentifier(), minDate, null, Integer.MAX_VALUE));
-        List<StatusSnapshotDTO> aggregateSnapshots = connHistory.getAggregateSnapshots();
-
-        if (aggregateSnapshots.size() < 2) {
-            LOG.info("Not enough data to model time to backpressure.");
-            return null;
-        } else {
-
-            ConnectionStatusDescriptor.QUEUED_COUNT.getField();
-            SimpleRegression regression = new SimpleRegression();
 
-            for (StatusSnapshotDTO snap : aggregateSnapshots) {
-                Long snapQueuedCount = snap.getStatusMetrics().get(ConnectionStatusDescriptor.QUEUED_COUNT.getField());
-                long snapTime = snap.getTimestamp().getTime();
-                regression.addData(snapTime, snapQueuedCount);
-            }
-
-            if (regression.getSlope() <= 0 && !conn.getFlowFileQueue().isFull()) {
-                LOG.info("Connection " + conn.getIdentifier() + " is not experiencing backpressure.");
-                return null;
-            } else {
-                return regression;
-            }
-        }
-    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
new file mode 100644
index 0000000..bad2ff1
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
@@ -0,0 +1,254 @@
+/*
+ * 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.analytics;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.stream.Stream;
+
+import org.apache.nifi.connectable.Connection;
+import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.controller.status.history.ComponentStatusRepository;
+import org.apache.nifi.controller.status.history.ConnectionStatusDescriptor;
+import org.apache.nifi.controller.status.history.StatusHistory;
+import org.apache.nifi.controller.status.history.StatusHistoryUtil;
+import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.processor.DataUnit;
+import org.apache.nifi.util.Tuple;
+import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
+import org.apache.nifi.web.api.dto.status.StatusSnapshotDTO;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ConnectionStatusAnalytics implements StatusAnalytics {
+
+    private static final Logger LOG = LoggerFactory.getLogger(ConnectionStatusAnalytics.class);
+    private Map<String, Tuple<StatusAnalyticsModel, ExtractFunction>> modelMap;
+    private QueryWindow queryWindow;
+    private final ComponentStatusRepository componentStatusRepository;
+    private final String connectionIdentifier;
+    private final FlowController flowController;
+
+    public ConnectionStatusAnalytics(ComponentStatusRepository componentStatusRepository, FlowController flowController, String connectionIdentifier) {
+        this.componentStatusRepository = componentStatusRepository;
+        this.flowController = flowController;
+        this.connectionIdentifier = connectionIdentifier;
+    }
+
+    public void init() {
+
+        if (this.modelMap == null || this.modelMap.isEmpty()) {
+            Tuple<StatusAnalyticsModel, ExtractFunction> countModelFunction = new Tuple<>(new SimpleRegressionBSAM(), extract);
+            Tuple<StatusAnalyticsModel, ExtractFunction> byteModelFunction = new Tuple<>(new SimpleRegressionBSAM(), extract);
+            this.modelMap = new HashMap<>();
+            //TODO: Should change keys used here
+            this.modelMap.put(ConnectionStatusDescriptor.QUEUED_COUNT.getField(), countModelFunction);
+            this.modelMap.put(ConnectionStatusDescriptor.QUEUED_BYTES.getField(), byteModelFunction);
+            this.queryWindow = new QueryWindow(System.currentTimeMillis() - (5 * 60 * 1000), System.currentTimeMillis());
+        }
+
+        refresh();
+    }
+
+    public void refresh() {
+
+        modelMap.forEach((metric, modelFunction) -> {
+
+            StatusAnalyticsModel model = modelFunction.getKey();
+            ExtractFunction extract = modelFunction.getValue();
+            StatusHistory statusHistory = componentStatusRepository.getConnectionStatusHistory(connectionIdentifier, queryWindow.getStartDateTime(), queryWindow.getEndDateTime(), Integer.MAX_VALUE);
+            Tuple<Stream<Double>, Stream<Double>> modelData = extract.extractMetric(metric, statusHistory);
+            LOG.info("Refreshing model for connection id: {} ", connectionIdentifier);
+            Stream<Double> times = modelData.getKey();
+            Stream<Double> counts = modelData.getValue();
+            //times is the X axis and counts is on the y axis
+            model.learn(times, counts);
+
+        });
+    }
+
+    /**
+     * Returns the predicted time (in milliseconds) when backpressure is expected to be applied to this connection, based on the total number of bytes in the queue.
+     *
+     * @return milliseconds until backpressure is predicted to occur, based on the total number of bytes in the queue.
+     */
+    public long getTimeToBytesBackpressureMillis() {
+
+        final BivariateStatusAnalyticsModel bytesModel = (BivariateStatusAnalyticsModel) modelMap.get(ConnectionStatusDescriptor.QUEUED_BYTES.getField()).getKey();
+        final Connection connection = getConnection();
+        if (connection == null) {
+            throw new NoSuchElementException("Connection with the following id cannot be found:" + connectionIdentifier + ". Model should be invalidated!");
+        }
+        final String backPressureDataSize = connection.getFlowFileQueue().getBackPressureDataSizeThreshold();
+        final double backPressureBytes = DataUnit.parseDataSize(backPressureDataSize, DataUnit.B);
+        final double prediction = bytesModel.predictX(backPressureBytes);
+        if (prediction != Double.NaN) {
+            return Math.max(0, Math.round(prediction) - System.currentTimeMillis());
+        } else {
+            return Long.MAX_VALUE;
+        }
+
+    }
+
+    /**
+     * Returns the predicted time (in milliseconds) when backpressure is expected to be applied to this connection, based on the number of objects in the queue.
+     *
+     * @return milliseconds until backpressure is predicted to occur, based on the number of objects in the queue.
+     */
+    public long getTimeToCountBackpressureMillis() {
+
+        final BivariateStatusAnalyticsModel countModel = (BivariateStatusAnalyticsModel) modelMap.get(ConnectionStatusDescriptor.QUEUED_COUNT.getField()).getKey();
+        final Connection connection = getConnection();
+        if (connection == null) {
+            throw new NoSuchElementException("Connection with the following id cannot be found:" + connectionIdentifier + ". Model should be invalidated!");
+        }
+        final double backPressureCountThreshold = connection.getFlowFileQueue().getBackPressureObjectThreshold();
+        final Double prediction = countModel.predictX(backPressureCountThreshold);
+
+        if (prediction != Double.NaN) {
+            return Math.max(0, Math.round(prediction) - System.currentTimeMillis());
+        } else {
+            return Long.MAX_VALUE;
+        }
+    }
+
+    /**
+     * Returns the predicted total number of bytes in the queue to occur at the next configured interval (5 mins in the future, e.g.).
+     *
+     * @return milliseconds until backpressure is predicted to occur, based on the total number of bytes in the queue.
+     */
+
+    public long getNextIntervalBytes() {
+        final BivariateStatusAnalyticsModel bytesModel = (BivariateStatusAnalyticsModel) modelMap.get(ConnectionStatusDescriptor.QUEUED_BYTES.getField()).getKey();
+        final Double prediction = bytesModel.predictY((double) System.currentTimeMillis() + getIntervalTimeMillis());
+        if (prediction != Double.NaN) {
+            return Math.round(prediction);
+        } else {
+            return 0;
+        }
+    }
+
+    /**
+     * Returns the predicted number of objects in the queue to occur at the next configured interval (5 mins in the future, e.g.).
+     *
+     * @return milliseconds until backpressure is predicted to occur, based on the number of bytes in the queue.
+     */
+
+    public int getNextIntervalCount() {
+        final BivariateStatusAnalyticsModel countModel = (BivariateStatusAnalyticsModel) modelMap.get(ConnectionStatusDescriptor.QUEUED_COUNT.getField()).getKey();
+        final Double prediction = countModel.predictY((double) System.currentTimeMillis() + getIntervalTimeMillis());
+        if (prediction != Double.NaN) {
+            return ((Long) Math.round(prediction)).intValue();
+        } else {
+            return 0;
+        }
+    }
+
+    public int getNextIntervalPercentageUseCount(){
+
+        final Connection connection = getConnection();
+        if (connection == null) {
+            throw new NoSuchElementException("Connection with the following id cannot be found:" + connectionIdentifier + ". Model should be invalidated!");
+        }
+        final double backPressureCountThreshold = connection.getFlowFileQueue().getBackPressureObjectThreshold();
+
+        return ((Long)Math.round((getNextIntervalCount()/backPressureCountThreshold) * 100)).intValue();
+
+    }
+
+    public int getNextIntervalPercentageUseBytes(){
+
+        final Connection connection = getConnection();
+        if (connection == null) {
+            throw new NoSuchElementException("Connection with the following id cannot be found:" + connectionIdentifier + ". Model should be invalidated!");
+        }
+        final String backPressureDataSize = connection.getFlowFileQueue().getBackPressureDataSizeThreshold();
+        final double backPressureBytes = DataUnit.parseDataSize(backPressureDataSize, DataUnit.B);
+
+        return ((Long)Math.round((getNextIntervalBytes()/ backPressureBytes) * 100)).intValue();
+
+    }
+
+    public long getIntervalTimeMillis(){
+        return getQueryWindow().getTimeDifferenceMillis();
+    }
+
+    @Override
+    public QueryWindow getQueryWindow() {
+        return queryWindow;
+    }
+
+    /**
+     * Returns all available predictions
+     */
+    @Override
+    public Map<String, Long> getPredictions() {
+
+        Map<String, Long> predictions = new HashMap<>();
+        predictions.put("timeToBytesBackpressureMillis", getTimeToBytesBackpressureMillis());
+        predictions.put("timeToCountBackpressureMillis", getTimeToCountBackpressureMillis());
+        predictions.put("nextIntervalBytes", getNextIntervalBytes());
+        predictions.put("nextIntervalCount", (long) getNextIntervalCount());
+        predictions.put("nextIntervalPercentageUseCount", (long)getNextIntervalPercentageUseCount());
+        predictions.put("nextIntervalPercentageUseBytes", (long)getNextIntervalPercentageUseBytes());
+        predictions.put("intervalTimeMillis", getIntervalTimeMillis());
+
+        predictions.forEach((key,value) -> {
+            LOG.info("Prediction model for connection id {}: {}={} ", connectionIdentifier,key,value);
+        });
+
+        return predictions;
+    }
+
+    @Override
+    public boolean supportsOnlineLearning() {
+        return true;
+    }
+
+    private Connection getConnection() {
+        final ProcessGroup rootGroup = flowController.getFlowManager().getRootGroup();
+        Optional<Connection> connection = rootGroup.findAllConnections().stream().filter(c -> c.getIdentifier().equals(this.connectionIdentifier)).findFirst();
+        return connection.orElse(null);
+    }
+
+    private interface ExtractFunction {
+        Tuple<Stream<Double>, Stream<Double>> extractMetric(String metric, StatusHistory statusHistory);
+    }
+
+    private final ExtractFunction extract = (metric, statusHistory) -> {
+
+        List<Double> counts = new ArrayList<>();
+        List<Double> times = new ArrayList<>();
+
+        StatusHistoryDTO statusHistoryDTO = StatusHistoryUtil.createStatusHistoryDTO(statusHistory);
+
+        for (StatusSnapshotDTO snap : statusHistoryDTO.getAggregateSnapshots()) {
+            Long snapValue = snap.getStatusMetrics().get(metric);
+            long snapTime = snap.getTimestamp().getTime();
+            counts.add((double) snapValue);
+            times.add((double) snapTime);
+        }
+        return new Tuple<>(times.stream(), counts.stream());
+
+    };
+
+
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalyticsEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalyticsEngine.java
index 11862c8..7f9db25 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalyticsEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalyticsEngine.java
@@ -16,18 +16,8 @@
  */
 package org.apache.nifi.controller.status.analytics;
 
-import java.util.Date;
-import java.util.List;
-
-import org.apache.commons.math3.stat.regression.SimpleRegression;
-import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.status.history.ComponentStatusRepository;
-import org.apache.nifi.controller.status.history.ConnectionStatusDescriptor;
-import org.apache.nifi.controller.status.history.StatusHistoryUtil;
-import org.apache.nifi.groups.ProcessGroup;
-import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
-import org.apache.nifi.web.api.dto.status.StatusSnapshotDTO;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -43,114 +33,11 @@ public class ConnectionStatusAnalyticsEngine implements StatusAnalyticsEngine {
     }
 
     @Override
-    public ConnectionStatusAnalytics getConnectionStatusAnalytics(String connectionId) {
-        ProcessGroup rootGroup = controller.getFlowManager().getRootGroup();
-        return getConnectionStatusAnalytics(rootGroup.findConnection(connectionId));
+    public StatusAnalytics getStatusAnalytics(String identifier) {
+        ConnectionStatusAnalytics connectionStatusAnalytics = new ConnectionStatusAnalytics(statusRepository,controller,identifier);
+        connectionStatusAnalytics.init();
+        return connectionStatusAnalytics;
     }
 
-    /**
-     * Finds the number of millis until the given connection will experience backpressure.
-     * @param conn the connection to run the analytic on
-     * @return
-     */
-    public ConnectionStatusAnalytics getConnectionStatusAnalytics(Connection conn) {
-        LOG.debug("Getting connection history for: " + conn.getIdentifier());
-        long connTimeToBackpressure;
-        Date minDate = new Date(System.currentTimeMillis() - (5 * 60 * 1000));
-        StatusHistoryDTO connHistory = StatusHistoryUtil.createStatusHistoryDTO(
-                statusRepository.getConnectionStatusHistory(conn.getIdentifier(), minDate, null, Integer.MAX_VALUE));
-        List<StatusSnapshotDTO> aggregateSnapshots = connHistory.getAggregateSnapshots();
-
-        if (aggregateSnapshots.size() < 2) {
-            LOG.info("Not enough data to model time to backpressure.");
-            connTimeToBackpressure = Long.MAX_VALUE;
-        } else {
-
-            long backPressureObjectThreshold = conn.getFlowFileQueue().getBackPressureObjectThreshold();
-            LOG.info("Connection " + conn.getIdentifier() + " backpressure object threshold is " + backPressureObjectThreshold);
-
-            ConnectionStatusDescriptor.QUEUED_COUNT.getField();
-
-            SimpleRegression regression = new SimpleRegression();
-
-            for (StatusSnapshotDTO snap : aggregateSnapshots) {
-                Long snapQueuedCount = snap.getStatusMetrics().get(ConnectionStatusDescriptor.QUEUED_COUNT.getField());
-                long snapTime = snap.getTimestamp().getTime();
-                regression.addData(snapTime, snapQueuedCount);
-            }
-
-            // Skip this connection if its queue is declining.
-            if (regression.getSlope() <= 0) {
-                LOG.info("Connection " + conn.getIdentifier() + " is not experiencing backpressure.");
-                connTimeToBackpressure = Long.MAX_VALUE;
-            } else {
-
-                // Compute time-to backpressure for this connection; Reduce total result iff
-                // this connection is lower.
-                connTimeToBackpressure = Math
-                        .round((backPressureObjectThreshold - regression.getIntercept()) / regression.getSlope())
-                        - System.currentTimeMillis();
-                LOG.info("Connection " + conn.getIdentifier() + " time to backpressure is " + connTimeToBackpressure);
-            }
-        }
-
-        return new ConnectionStatusAnalytics() {
-
-            @Override
-            public String getSourceName() {
-                return conn.getSource().getName();
-            }
 
-            @Override
-            public String getSourceId() {
-                return conn.getSource().getIdentifier();
-            }
-
-            @Override
-            public String getName() {
-                return conn.getName();
-            }
-
-            @Override
-            public long getTimeToCountBackpressureMillis() {
-                return connTimeToBackpressure;
-            }
-
-            // TODO - populate the other prediction fields
-            @Override
-            public long getTimeToBytesBackpressureMillis() {
-                return 0;
-            }
-
-            @Override
-            public long getNextIntervalBytes() {
-                return 0;
-            }
-
-            @Override
-            public int getNextIntervalCount() {
-                return 0;
-            }
-
-            @Override
-            public String getId() {
-                return conn.getIdentifier();
-            }
-
-            @Override
-            public String getGroupId() {
-                return conn.getProcessGroup().getIdentifier();
-            }
-
-            @Override
-            public String getDestinationName() {
-                return conn.getDestination().getName();
-            }
-
-            @Override
-            public String getDestinationId() {
-                return conn.getDestination().getIdentifier();
-            }
-        };
-    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/SimpleRegressionBSAM.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/SimpleRegressionBSAM.java
new file mode 100644
index 0000000..8aa4a45
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/SimpleRegressionBSAM.java
@@ -0,0 +1,55 @@
+/*
+ * 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.analytics;
+
+
+import java.util.stream.Stream;
+
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.commons.math3.stat.regression.SimpleRegression;
+
+public class SimpleRegressionBSAM extends BivariateStatusAnalyticsModel {
+
+    private SimpleRegression regression;
+
+    public SimpleRegressionBSAM() {
+        this.regression = new SimpleRegression();
+    }
+
+    @Override
+    public void learn(Stream<Double> features, Stream<Double> labels) {
+        double[] labelArray = ArrayUtils.toPrimitive(labels.toArray(Double[]::new));
+        double[][] featuresMatrix = features.map(feature -> new double[]{feature}).toArray(double[][]::new);
+        regression.clear();
+        regression.addObservations(featuresMatrix, labelArray);
+    }
+
+    @Override
+    public Double predict(Double feature) {
+        return predictY(feature);
+    }
+
+    @Override
+    public Double predictX(Double y) {
+        return (y - regression.getIntercept()) / regression.getSlope();
+    }
+
+    @Override
+    public Double predictY(Double x) {
+        return regression.getSlope() * x + regression.getIntercept();
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java
index 33e650b..1e36975 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java
@@ -20,6 +20,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
@@ -343,12 +344,16 @@ public class StandardEventAccess implements UserAwareEventAccess {
             }
 
             if (statusAnalyticsEngine != null) {
-                StatusAnalytics statusAnalytics = statusAnalyticsEngine.getConnectionStatusAnalytics(conn.getIdentifier());
+                StatusAnalytics statusAnalytics =  statusAnalyticsEngine.getStatusAnalytics(conn.getIdentifier());
                 if (statusAnalytics != null) {
-                    connStatus.setPredictedTimeToBytesBackpressureMillis(statusAnalytics.getTimeToBytesBackpressureMillis());
-                    connStatus.setPredictedTimeToCountBackpressureMillis(statusAnalytics.getTimeToCountBackpressureMillis());
-                    connStatus.setNextPredictedQueuedBytes(statusAnalytics.getNextIntervalBytes());
-                    connStatus.setNextPredictedQueuedCount(statusAnalytics.getNextIntervalCount());
+                    Map<String,Long> predictions = statusAnalytics.getPredictions();
+                    connStatus.setPredictedTimeToBytesBackpressureMillis(predictions.get("timeToBytesBackpressureMillis"));
+                    connStatus.setPredictedTimeToCountBackpressureMillis(predictions.get("timeToCountBackpressureMillis"));
+                    connStatus.setNextPredictedQueuedBytes(predictions.get("nextIntervalBytes"));
+                    connStatus.setNextPredictedQueuedCount(predictions.get("nextIntervalCount").intValue());
+                    connStatus.setPredictedPercentCount(predictions.get("nextIntervalPercentageUseCount").intValue());
+                    connStatus.setPredictedPercentBytes(predictions.get("nextIntervalPercentageUseBytes").intValue());
+                    connStatus.setPredictionIntervalMillis(predictions.get("intervalTimeMillis"));
                 }
             }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
index 73115b5..331847b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
@@ -16,7 +16,35 @@
  */
 package org.apache.nifi.web;
 
-import com.google.common.collect.Sets;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.Response;
+
 import org.apache.commons.collections4.CollectionUtils;
 import org.apache.nifi.action.Action;
 import org.apache.nifi.action.Component;
@@ -309,33 +337,7 @@ import org.apache.nifi.web.util.SnippetUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.ws.rs.WebApplicationException;
-import javax.ws.rs.core.Response;
-import java.io.IOException;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.ListIterator;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-import java.util.function.Function;
-import java.util.function.Predicate;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
+import com.google.common.collect.Sets;
 
 /**
  * Implementation of NiFiServiceFacade that performs revision checking.
@@ -3196,7 +3198,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
     public ConnectionStatisticsEntity getConnectionStatistics(final String connectionId) {
         final Connection connection = connectionDAO.getConnection(connectionId);
         final PermissionsDTO permissions = dtoFactory.createPermissionsDto(connection);
-        final ConnectionStatisticsDTO dto = dtoFactory.createConnectionStatisticsDto(controllerFacade.getConnectionStatistics(connectionId));
+        final ConnectionStatisticsDTO dto = dtoFactory.createConnectionStatisticsDto(connection, controllerFacade.getConnectionStatusAnalytics(connectionId));
         return entityFactory.createConnectionStatisticsEntity(dto, permissions);
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
index 88cc299..e56e99a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
@@ -16,6 +16,34 @@
  */
 package org.apache.nifi.web.api.dto;
 
+import java.text.Collator;
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TimeZone;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+import javax.ws.rs.WebApplicationException;
+
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.lang3.ClassUtils;
 import org.apache.commons.lang3.StringUtils;
@@ -105,7 +133,7 @@ import org.apache.nifi.controller.status.PortStatus;
 import org.apache.nifi.controller.status.ProcessGroupStatus;
 import org.apache.nifi.controller.status.ProcessorStatus;
 import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
-import org.apache.nifi.controller.status.analytics.ConnectionStatusAnalytics;
+import org.apache.nifi.controller.status.analytics.StatusAnalytics;
 import org.apache.nifi.controller.status.history.GarbageCollectionHistory;
 import org.apache.nifi.controller.status.history.GarbageCollectionStatus;
 import org.apache.nifi.diagnostics.GarbageCollection;
@@ -234,33 +262,6 @@ import org.apache.nifi.web.api.entity.VariableEntity;
 import org.apache.nifi.web.controller.ControllerFacade;
 import org.apache.nifi.web.revision.RevisionManager;
 
-import javax.ws.rs.WebApplicationException;
-import java.text.Collator;
-import java.text.NumberFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.TimeZone;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
-import java.util.function.Function;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
 public final class DtoFactory {
 
     @SuppressWarnings("rawtypes")
@@ -1197,30 +1198,35 @@ public final class DtoFactory {
         return connectionStatusDto;
     }
 
-    public ConnectionStatisticsDTO createConnectionStatisticsDto(final ConnectionStatusAnalytics connectionStatistics) {
+    public ConnectionStatisticsDTO createConnectionStatisticsDto(final Connection connection, final StatusAnalytics statusAnalytics) {
         final ConnectionStatisticsDTO connectionStatisticsDTO = new ConnectionStatisticsDTO();
-        connectionStatisticsDTO.setGroupId(connectionStatistics.getGroupId());
-        connectionStatisticsDTO.setId(connectionStatistics.getId());
-        connectionStatisticsDTO.setName(connectionStatistics.getName());
-        connectionStatisticsDTO.setSourceId(connectionStatistics.getSourceId());
-        connectionStatisticsDTO.setSourceName(connectionStatistics.getSourceName());
-        connectionStatisticsDTO.setDestinationId(connectionStatistics.getDestinationId());
-        connectionStatisticsDTO.setDestinationName(connectionStatistics.getDestinationName());
+
+        connectionStatisticsDTO.setGroupId(connection.getProcessGroup().getIdentifier());
+        connectionStatisticsDTO.setId(connection.getIdentifier());
+        connectionStatisticsDTO.setName(connection.getName());
+        connectionStatisticsDTO.setSourceId(connection.getSource().getIdentifier());
+        connectionStatisticsDTO.setSourceName(connection.getSource().getName());
+        connectionStatisticsDTO.setDestinationId(connection.getDestination().getIdentifier());
+        connectionStatisticsDTO.setDestinationName(connection.getDestination().getName());
         connectionStatisticsDTO.setStatsLastRefreshed(new Date());
 
         final ConnectionStatisticsSnapshotDTO snapshot = new ConnectionStatisticsSnapshotDTO();
         connectionStatisticsDTO.setAggregateSnapshot(snapshot);
 
-        snapshot.setId(connectionStatistics.getId());
-        snapshot.setGroupId(connectionStatistics.getGroupId());
-        snapshot.setName(connectionStatistics.getName());
-        snapshot.setSourceName(connectionStatistics.getSourceName());
-        snapshot.setDestinationName(connectionStatistics.getDestinationName());
-
-        snapshot.setPredictedMillisUntilBytesBackpressure(connectionStatistics.getTimeToBytesBackpressureMillis());
-        snapshot.setPredictedMillisUntilCountBackpressure(connectionStatistics.getTimeToCountBackpressureMillis());
-        snapshot.setPredictedBytesAtNextInterval(connectionStatistics.getNextIntervalBytes());
-        snapshot.setPredictedCountAtNextInterval(connectionStatistics.getNextIntervalCount());
+        snapshot.setId(connection.getIdentifier());
+        snapshot.setGroupId(connection.getProcessGroup().getIdentifier());
+        snapshot.setName(connection.getName());
+        snapshot.setSourceName(connection.getSource().getName());
+        snapshot.setDestinationName(connection.getDestination().getName());
+
+        Map<String,Long> predictions = statusAnalytics.getPredictions();
+        snapshot.setPredictedMillisUntilBytesBackpressure(predictions.get("timeToBytesBackpressureMillis"));
+        snapshot.setPredictedMillisUntilCountBackpressure(predictions.get("timeToCountBackpressureMillis"));
+        snapshot.setPredictedBytesAtNextInterval(predictions.get("nextIntervalBytes"));
+        snapshot.setPredictedCountAtNextInterval(predictions.get("nextIntervalCount").intValue());
+        snapshot.setPredictedPercentBytes(predictions.get("nextIntervalPercentageUseBytes").intValue());
+        snapshot.setPredictedPercentCount(predictions.get("nextIntervalPercentageUseCount").intValue());
+        snapshot.setPredictionIntervalMillis(predictions.get("intervalTimeMillis"));
 
         return connectionStatisticsDTO;
     }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
index 2fcef27..48abda9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
@@ -16,6 +16,30 @@
  */
 package org.apache.nifi.web.controller;
 
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.Collator;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TimeZone;
+import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import javax.ws.rs.WebApplicationException;
+
 import org.apache.commons.collections4.CollectionUtils;
 import org.apache.commons.lang3.ClassUtils;
 import org.apache.commons.lang3.StringUtils;
@@ -56,7 +80,7 @@ import org.apache.nifi.controller.status.PortStatus;
 import org.apache.nifi.controller.status.ProcessGroupStatus;
 import org.apache.nifi.controller.status.ProcessorStatus;
 import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
-import org.apache.nifi.controller.status.analytics.ConnectionStatusAnalytics;
+import org.apache.nifi.controller.status.analytics.StatusAnalytics;
 import org.apache.nifi.controller.status.analytics.StatusAnalyticsEngine;
 import org.apache.nifi.controller.status.history.ComponentStatusRepository;
 import org.apache.nifi.diagnostics.SystemDiagnostics;
@@ -112,29 +136,6 @@ import org.apache.nifi.web.api.entity.ControllerServiceEntity;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.ws.rs.WebApplicationException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.text.Collator;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TimeZone;
-import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
-import java.util.function.Consumer;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-
 public class ControllerFacade implements Authorizable {
 
     private static final Logger logger = LoggerFactory.getLogger(ControllerFacade.class);
@@ -683,12 +684,12 @@ public class ControllerFacade implements Authorizable {
     }
 
     /**
-     * Gets analytical statistics for the specified connection.
+     * Gets status analytics for the specified connection.
      *
      * @param connectionId connection id
      * @return the statistics for the specified connection
      */
-    public ConnectionStatusAnalytics getConnectionStatistics(final String connectionId) {
+    public StatusAnalytics getConnectionStatusAnalytics(final String connectionId) {
         final ProcessGroup root = getRootGroup();
         final Connection connection = root.findConnection(connectionId);
 
@@ -705,12 +706,12 @@ public class ControllerFacade implements Authorizable {
         }
 
         // get from flow controller
-        final StatusAnalyticsEngine status = flowController.getStatusAnalyticsEngine();
-        if (status == null) {
+        final StatusAnalyticsEngine statusAnalyticsEngine = flowController.getStatusAnalyticsEngine();
+        if (statusAnalyticsEngine == null) {
             throw new ResourceNotFoundException(String.format("Unable to locate connection with id '%s'.", connectionId));
         }
 
-        return status.getConnectionStatusAnalytics(connectionId);
+        return statusAnalyticsEngine.getStatusAnalytics(connectionId);
     }
 
     /**


[nifi] 21/21: NIFI-6510 Added property to nifi.properties - Prediction Interval for connection status analytics (#11)

Posted by ai...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

aichrist pushed a commit to branch analytics-framework
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 714b3f6a3fdafa4ddcf0740f102a9e87d45da041
Author: Matthew Burgess <ma...@apache.org>
AuthorDate: Fri Aug 16 15:44:44 2019 -0400

    NIFI-6510 Added property to nifi.properties - Prediction Interval for connection status analytics (#11)
---
 .../src/main/java/org/apache/nifi/util/NiFiProperties.java  |  6 ++++++
 .../src/test/resources/NiFiProperties/conf/nifi.properties  |  3 +++
 nifi-docs/src/main/asciidoc/administration-guide.adoc       | 10 ++++++++++
 .../src/test/resources/conf/nifi.properties                 |  3 +++
 .../java/org/apache/nifi/controller/FlowController.java     | 11 ++++++++++-
 .../analytics/CachingConnectionStatusAnalyticsEngine.java   |  6 +++++-
 .../status/analytics/ConnectionStatusAnalytics.java         |  7 ++++++-
 .../status/analytics/ConnectionStatusAnalyticsEngine.java   |  5 ++++-
 .../TestCachingConnectionStatusAnalyticsEngine.java         | 13 +++++++++----
 .../analytics/TestConnectionStatusAnalyticsEngine.java      |  5 +++--
 .../status/analytics/TestStatusAnalyticsEngine.java         |  7 +++++--
 .../src/test/resources/conf/nifi.properties                 |  3 +++
 .../src/test/resources/flowcontrollertest.nifi.properties   |  3 +++
 .../nifi-framework/nifi-resources/pom.xml                   |  3 +++
 .../nifi-resources/src/main/resources/conf/nifi.properties  |  5 ++++-
 15 files changed, 77 insertions(+), 13 deletions(-)

diff --git a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
index 31b1db5..aa00793 100644
--- a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
+++ b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
@@ -238,6 +238,9 @@ public abstract class NiFiProperties {
     // expression language properties
     public static final String VARIABLE_REGISTRY_PROPERTIES = "nifi.variable.registry.properties";
 
+    // analytics properties
+    public static final String ANALYTICS_PREDICTION_INTERVAL = "nifi.analytics.predict.interval";
+
     // defaults
     public static final Boolean DEFAULT_AUTO_RESUME_STATE = true;
     public static final String DEFAULT_AUTHORIZER_CONFIGURATION_FILE = "conf/authorizers.xml";
@@ -308,6 +311,9 @@ public abstract class NiFiProperties {
     // Kerberos defaults
     public static final String DEFAULT_KERBEROS_AUTHENTICATION_EXPIRATION = "12 hours";
 
+    // analytics defaults
+    public static final String DEFAULT_ANALYTICS_PREDICTION_INTERVAL = "3 mins";
+
 
     /**
      * Retrieves the property value for the given property key.
diff --git a/nifi-commons/nifi-properties/src/test/resources/NiFiProperties/conf/nifi.properties b/nifi-commons/nifi-properties/src/test/resources/NiFiProperties/conf/nifi.properties
index 2c58fa9..9ab3d99 100644
--- a/nifi-commons/nifi-properties/src/test/resources/NiFiProperties/conf/nifi.properties
+++ b/nifi-commons/nifi-properties/src/test/resources/NiFiProperties/conf/nifi.properties
@@ -120,3 +120,6 @@ nifi.cluster.manager.node.api.request.threads=10
 nifi.cluster.manager.flow.retrieval.delay=5 sec
 nifi.cluster.manager.protocol.threads=10
 nifi.cluster.manager.safemode.duration=0 sec
+
+# analytics properties #
+nifi.analytics.predict.interval=3 mins
\ No newline at end of file
diff --git a/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi-docs/src/main/asciidoc/administration-guide.adoc
index 90b7101..699010e 100644
--- a/nifi-docs/src/main/asciidoc/administration-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/administration-guide.adoc
@@ -3313,6 +3313,16 @@ that is specified.
 |`nifi.kerberos.spengo.authentication.expiration`*|The expiration duration of a successful Kerberos user authentication, if used. The default value is `12 hours`.
 |====
 
+[[analytics_properties]]
+=== Analytics Properties
+
+These properties determine the behavior of the internal NiFi Analytics capability, such as backpressure prediction, and should be configured the same way on all nodes.
+
+|====
+|*Property*|*Description*
+|`nifi.analytics.predict.interval`|This indicates a time interval for which analytical predictions (queue saturation, e.g.) should be made. The default value is `3 mins`.
+|====
+
 [[custom_properties]]
 === Custom Properties
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/resources/conf/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/resources/conf/nifi.properties
index a768adc..4d139d7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/resources/conf/nifi.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/resources/conf/nifi.properties
@@ -121,3 +121,6 @@ nifi.cluster.manager.node.api.request.threads=10
 nifi.cluster.manager.flow.retrieval.delay=5 sec
 nifi.cluster.manager.protocol.threads=10
 nifi.cluster.manager.safemode.duration=0 sec
+
+# analytics properties #
+nifi.analytics.predict.interval=3 mins
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index bf1d06c..55ff0bb 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -593,8 +593,17 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
             zooKeeperStateServer = null;
         }
 
+        // Determine interval for predicting future feature values
+        final String predictionInterval = nifiProperties.getProperty(NiFiProperties.ANALYTICS_PREDICTION_INTERVAL, NiFiProperties.DEFAULT_ANALYTICS_PREDICTION_INTERVAL);
+        long predictionIntervalMillis;
+        try {
+            predictionIntervalMillis = FormatUtils.getTimeDuration(predictionInterval, TimeUnit.MILLISECONDS);
+        } catch (final Exception e) {
+            predictionIntervalMillis = FormatUtils.getTimeDuration(NiFiProperties.DEFAULT_ANALYTICS_PREDICTION_INTERVAL, TimeUnit.MILLISECONDS);
+        }
+
         componentStatusRepository = createComponentStatusRepository();
-        analyticsEngine = new ConnectionStatusAnalyticsEngine(flowManager, componentStatusRepository,flowFileEventRepository);
+        analyticsEngine = new ConnectionStatusAnalyticsEngine(flowManager, componentStatusRepository, flowFileEventRepository, predictionIntervalMillis);
         eventAccess = new StandardEventAccess(this, flowFileEventRepository);
 
         timerDrivenEngineRef.get().scheduleWithFixedDelay(new Runnable() {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingConnectionStatusAnalyticsEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingConnectionStatusAnalyticsEngine.java
index 747c496..95f655e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingConnectionStatusAnalyticsEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingConnectionStatusAnalyticsEngine.java
@@ -32,15 +32,18 @@ public class CachingConnectionStatusAnalyticsEngine implements StatusAnalyticsEn
     private final FlowManager flowManager;
     private final FlowFileEventRepository flowFileEventRepository;
     private volatile Cache<String, ConnectionStatusAnalytics> cache;
+    private final long predictionIntervalMillis;
     private static final Logger LOG = LoggerFactory.getLogger(CachingConnectionStatusAnalyticsEngine.class);
 
-    public CachingConnectionStatusAnalyticsEngine(FlowManager flowManager, ComponentStatusRepository statusRepository, FlowFileEventRepository flowFileEventRepository) {
+    public CachingConnectionStatusAnalyticsEngine(FlowManager flowManager, ComponentStatusRepository statusRepository,
+                                                  FlowFileEventRepository flowFileEventRepository, long predictionIntervalMillis) {
         this.flowManager = flowManager;
         this.statusRepository = statusRepository;
         this.flowFileEventRepository = flowFileEventRepository;
         this.cache = Caffeine.newBuilder()
                 .expireAfterWrite(30, TimeUnit.MINUTES)
                 .build();
+        this.predictionIntervalMillis = predictionIntervalMillis;
     }
 
     @Override
@@ -50,6 +53,7 @@ public class CachingConnectionStatusAnalyticsEngine implements StatusAnalyticsEn
         if (connectionStatusAnalytics == null) {
             LOG.debug("Creating new status analytics object for connection id: {}", identifier);
             connectionStatusAnalytics = new ConnectionStatusAnalytics(statusRepository, flowManager,flowFileEventRepository, identifier, true);
+            connectionStatusAnalytics.setIntervalTimeMillis(predictionIntervalMillis);
             connectionStatusAnalytics.init();
             cache.put(identifier, connectionStatusAnalytics);
         } else {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
index aa19b1d..313e7ab 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java
@@ -59,6 +59,7 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
     private final FlowManager flowManager;
     private final Boolean supportOnlineLearning;
     private Boolean extendWindow = false;
+    private long intervalMillis = 3L * 60 * 1000; // Default is 3 minutes
     private static double SCORE_THRESHOLD = .90;
 
     public ConnectionStatusAnalytics(ComponentStatusRepository componentStatusRepository, FlowManager flowManager, FlowFileEventRepository flowFileEventRepository, String connectionIdentifier,
@@ -255,7 +256,11 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
     }
 
     public Long getIntervalTimeMillis() {
-        return 3L * 60 * 1000;
+        return intervalMillis;
+    }
+
+    public void setIntervalTimeMillis(long intervalTimeMillis) {
+        this.intervalMillis = intervalTimeMillis;
     }
 
     @Override
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalyticsEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalyticsEngine.java
index 6f261a0..a9ba4ea 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalyticsEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalyticsEngine.java
@@ -27,16 +27,19 @@ public class ConnectionStatusAnalyticsEngine implements StatusAnalyticsEngine {
     private final ComponentStatusRepository statusRepository;
     private final FlowManager flowManager;
     private final FlowFileEventRepository flowFileEventRepository;
+    private final long predictionIntervalMillis;
 
-    public ConnectionStatusAnalyticsEngine(FlowManager flowManager, ComponentStatusRepository statusRepository, FlowFileEventRepository flowFileEventRepository) {
+    public ConnectionStatusAnalyticsEngine(FlowManager flowManager, ComponentStatusRepository statusRepository, FlowFileEventRepository flowFileEventRepository, long predictionIntervalMillis) {
         this.flowManager = flowManager;
         this.statusRepository = statusRepository;
         this.flowFileEventRepository = flowFileEventRepository;
+        this.predictionIntervalMillis = predictionIntervalMillis;
     }
 
     @Override
     public StatusAnalytics getStatusAnalytics(String identifier) {
         ConnectionStatusAnalytics connectionStatusAnalytics = new ConnectionStatusAnalytics(statusRepository, flowManager, flowFileEventRepository, identifier, false);
+        connectionStatusAnalytics.setIntervalTimeMillis(predictionIntervalMillis);
         connectionStatusAnalytics.init();
         return connectionStatusAnalytics;
     }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestCachingConnectionStatusAnalyticsEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestCachingConnectionStatusAnalyticsEngine.java
index 77ffa9b..2c86bfc 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestCachingConnectionStatusAnalyticsEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestCachingConnectionStatusAnalyticsEngine.java
@@ -21,20 +21,25 @@ import org.apache.nifi.controller.repository.FlowFileEventRepository;
 import org.apache.nifi.controller.status.history.ComponentStatusRepository;
 import org.junit.Test;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+
 public class TestCachingConnectionStatusAnalyticsEngine extends TestStatusAnalyticsEngine {
 
     @Override
-    public StatusAnalyticsEngine getStatusAnalyticsEngine(FlowManager flowManager, FlowFileEventRepository flowFileEventRepository, ComponentStatusRepository componentStatusRepository) {
-        return new CachingConnectionStatusAnalyticsEngine(flowManager, componentStatusRepository, flowFileEventRepository);
+    public StatusAnalyticsEngine getStatusAnalyticsEngine(FlowManager flowManager, FlowFileEventRepository flowFileEventRepository,
+                                                          ComponentStatusRepository componentStatusRepository, long predictIntervalMillis) {
+        return new CachingConnectionStatusAnalyticsEngine(flowManager, componentStatusRepository, flowFileEventRepository, predictIntervalMillis);
     }
 
     @Test
     public void testCachedStatusAnalytics() {
-        StatusAnalyticsEngine statusAnalyticsEngine = new CachingConnectionStatusAnalyticsEngine(flowManager, statusRepository, flowFileEventRepository);
+        StatusAnalyticsEngine statusAnalyticsEngine = new CachingConnectionStatusAnalyticsEngine(flowManager, statusRepository, flowFileEventRepository, DEFAULT_PREDICT_INTERVAL_MILLIS);
         StatusAnalytics statusAnalyticsA = statusAnalyticsEngine.getStatusAnalytics("A");
         StatusAnalytics statusAnalyticsB = statusAnalyticsEngine.getStatusAnalytics("B");
         StatusAnalytics statusAnalyticsTest = statusAnalyticsEngine.getStatusAnalytics("A");
-        assert (statusAnalyticsA.equals(statusAnalyticsTest));
+        assertEquals(statusAnalyticsA, statusAnalyticsTest);
+        assertNotEquals(statusAnalyticsB, statusAnalyticsTest);
     }
 
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalyticsEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalyticsEngine.java
index 99c9fae..172c3b5 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalyticsEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalyticsEngine.java
@@ -23,8 +23,9 @@ import org.apache.nifi.controller.status.history.ComponentStatusRepository;
 public class TestConnectionStatusAnalyticsEngine extends TestStatusAnalyticsEngine {
 
     @Override
-    public StatusAnalyticsEngine getStatusAnalyticsEngine(FlowManager flowManager, FlowFileEventRepository flowFileEventRepository, ComponentStatusRepository statusRepository) {
-        return new ConnectionStatusAnalyticsEngine(flowManager, statusRepository, flowFileEventRepository);
+    public StatusAnalyticsEngine getStatusAnalyticsEngine(FlowManager flowManager, FlowFileEventRepository flowFileEventRepository,
+                                                          ComponentStatusRepository statusRepository, long predictIntervalMillis) {
+        return new ConnectionStatusAnalyticsEngine(flowManager, statusRepository, flowFileEventRepository, DEFAULT_PREDICT_INTERVAL_MILLIS);
     }
 
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestStatusAnalyticsEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestStatusAnalyticsEngine.java
index ee3d4e5..f666fdf 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestStatusAnalyticsEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestStatusAnalyticsEngine.java
@@ -36,6 +36,8 @@ import org.mockito.Mockito;
 
 public abstract class TestStatusAnalyticsEngine {
 
+    static final long DEFAULT_PREDICT_INTERVAL_MILLIS = 3L * 60 * 1000;
+
     protected ComponentStatusRepository statusRepository;
     protected FlowManager flowManager;
     protected FlowFileEventRepository flowFileEventRepository;
@@ -55,11 +57,12 @@ public abstract class TestStatusAnalyticsEngine {
 
     @Test
     public void testGetStatusAnalytics() {
-        StatusAnalyticsEngine statusAnalyticsEngine = getStatusAnalyticsEngine(flowManager,flowFileEventRepository, statusRepository);
+        StatusAnalyticsEngine statusAnalyticsEngine = getStatusAnalyticsEngine(flowManager,flowFileEventRepository, statusRepository, DEFAULT_PREDICT_INTERVAL_MILLIS);
         StatusAnalytics statusAnalytics = statusAnalyticsEngine.getStatusAnalytics("1");
         assertNotNull(statusAnalytics);
     }
 
-    public abstract StatusAnalyticsEngine getStatusAnalyticsEngine(FlowManager flowManager, FlowFileEventRepository flowFileEventRepository, ComponentStatusRepository componentStatusRepository);
+    public abstract StatusAnalyticsEngine getStatusAnalyticsEngine(FlowManager flowManager, FlowFileEventRepository flowFileEventRepository,
+                                                                   ComponentStatusRepository componentStatusRepository, long predictIntervalMillis);
 
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/conf/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/conf/nifi.properties
index cc8f098..3d9df16 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/conf/nifi.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/conf/nifi.properties
@@ -119,3 +119,6 @@ nifi.cluster.manager.node.api.request.threads=10
 nifi.cluster.manager.flow.retrieval.delay=5 sec
 nifi.cluster.manager.protocol.threads=10
 nifi.cluster.manager.safemode.duration=0 sec
+
+# analytics properties #
+nifi.analytics.predict.interval=3 mins
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/flowcontrollertest.nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/flowcontrollertest.nifi.properties
index a4c1a4a..9e6aecb 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/flowcontrollertest.nifi.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/flowcontrollertest.nifi.properties
@@ -127,3 +127,6 @@ nifi.cluster.manager.node.api.request.threads=10
 nifi.cluster.manager.flow.retrieval.delay=5 sec
 nifi.cluster.manager.protocol.threads=10
 nifi.cluster.manager.safemode.duration=0 sec
+
+# analytics properties #
+nifi.analytics.predict.interval=3 mins
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml
index 91ab8bd..15ac294 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml
@@ -208,6 +208,9 @@
         <nifi.kerberos.spnego.principal />
         <nifi.kerberos.spnego.keytab.location />
         <nifi.kerberos.spnego.authentication.expiration>12 hours</nifi.kerberos.spnego.authentication.expiration>
+
+        <!-- nifi.properties: analytics properties -->
+        <nifi.analytics.predict.interval>3 mins</nifi.analytics.predict.interval>
     </properties>
     <build>
         <plugins>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
index 556d783..e6140fa 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
@@ -255,4 +255,7 @@ nifi.kerberos.spnego.authentication.expiration=${nifi.kerberos.spnego.authentica
 
 # external properties files for variable registry
 # supports a comma delimited list of file locations
-nifi.variable.registry.properties=
\ No newline at end of file
+nifi.variable.registry.properties=
+
+# analytics properties #
+nifi.analytics.predict.interval=${nifi.analytics.predict.interval}
\ No newline at end of file


[nifi] 01/21: NIFI-6510 Implement initial analytic engine

Posted by ai...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

aichrist pushed a commit to branch analytics-framework
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit c4cc6c9a8a3d92777435d8559e5eb01e2d395daa
Author: Andrew I. Christianson <an...@andyic.org>
AuthorDate: Tue Jul 9 14:15:30 2019 -0400

    NIFI-6510 Implement initial analytic engine
---
 .../org/apache/nifi/controller/FlowController.java | 14 +++++
 .../status/analytics/StatusAnalyticEngine.java     | 60 ++++++++++++++++++++++
 2 files changed, 74 insertions(+)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index 331e73e..4c0288f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -117,6 +117,7 @@ import org.apache.nifi.controller.service.StandardConfigurationContext;
 import org.apache.nifi.controller.service.StandardControllerServiceProvider;
 import org.apache.nifi.controller.state.manager.StandardStateManagerProvider;
 import org.apache.nifi.controller.state.server.ZooKeeperStateServer;
+import org.apache.nifi.controller.status.analytics.StatusAnalyticEngine;
 import org.apache.nifi.controller.status.history.ComponentStatusRepository;
 import org.apache.nifi.controller.status.history.GarbageCollectionHistory;
 import org.apache.nifi.controller.status.history.GarbageCollectionStatus;
@@ -601,6 +602,19 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
             }
         }, snapshotMillis, snapshotMillis, TimeUnit.MILLISECONDS);
 
+        StatusAnalyticEngine analyticsEngine = new StatusAnalyticEngine(this, componentStatusRepository);
+
+        timerDrivenEngineRef.get().scheduleWithFixedDelay(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    analyticsEngine.getMinTimeToBackpressure();
+                } catch (final Exception e) {
+                    LOG.error("Failed to capture component stats for Stats History", e);
+                }
+            }
+        }, 1000, 1000, TimeUnit.MILLISECONDS); //FIXME use a real/configured interval
+
         this.connectionStatus = new NodeConnectionStatus(nodeId, DisconnectionCode.NOT_YET_CONNECTED);
         heartbeatBeanRef.set(new HeartbeatBean(rootGroup, false));
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java
new file mode 100644
index 0000000..8b69ebf
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java
@@ -0,0 +1,60 @@
+/*
+ * 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.analytics;
+
+import java.util.List;
+import java.util.Map.Entry;
+
+import org.apache.nifi.connectable.Connection;
+import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.controller.status.history.ComponentStatusRepository;
+import org.apache.nifi.controller.status.history.StatusHistoryUtil;
+import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
+import org.apache.nifi.web.api.dto.status.StatusSnapshotDTO;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class StatusAnalyticEngine {
+    private ComponentStatusRepository statusRepository;
+    private FlowController controller;
+
+    private static final Logger LOG = LoggerFactory.getLogger(StatusAnalyticEngine.class);
+
+    public StatusAnalyticEngine(FlowController controller, ComponentStatusRepository statusRepository) {
+        this.controller = controller;
+        this.statusRepository = statusRepository;
+    }
+
+    public long getMinTimeToBackpressure() {
+        ProcessGroup rootGroup = controller.getFlowManager().getRootGroup();
+        List<Connection> allConnections = rootGroup.findAllConnections();
+
+        for (Connection conn : allConnections) {
+            LOG.info("Getting connection history for: " + conn.getIdentifier());
+            StatusHistoryDTO connHistory = StatusHistoryUtil.createStatusHistoryDTO(
+                    statusRepository.getConnectionStatusHistory(conn.getIdentifier(), null, null, Integer.MAX_VALUE));
+            for (StatusSnapshotDTO snap : connHistory.getAggregateSnapshots()) {
+                for (Entry<String, Long> snapEntry : snap.getStatusMetrics().entrySet()) {
+                    LOG.info("Snap " + snapEntry.getKey() + ": " + snapEntry.getValue());
+                }
+            }
+        }
+
+        return 0;
+    }
+}


[nifi] 07/21: NIFI-6510 adjustments for interface updates, added call to StandardEventAccess, updated interface to use connection id

Posted by ai...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

aichrist pushed a commit to branch analytics-framework
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 2bcf3db5abb83acbe437dd88901d4578c674b771
Author: Yolanda Davis <yo...@gmail.com>
AuthorDate: Tue Jul 23 01:38:06 2019 -0400

    NIFI-6510 adjustments for interface updates, added call to StandardEventAccess, updated interface to use connection id
    
    (cherry picked from commit 14854ff)
    
    DFA-9 - reduced snapshot interval to 1 minute
    
    (cherry picked from commit 36abb0a)
---
 .../status/analytics/StatusAnalytics.java          | 12 ++--
 .../org/apache/nifi/controller/FlowController.java | 74 +++++++++++-----------
 .../analytics/CachingStatusAnalyticEngine.java     | 70 +++++++++++++-------
 .../status/analytics/StatusAnalyticEngine.java     |  8 +--
 .../apache/nifi/reporting/StandardEventAccess.java | 11 ++--
 5 files changed, 102 insertions(+), 73 deletions(-)

diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java
index 7d29314..45e1c12 100644
--- a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java
+++ b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java
@@ -32,24 +32,28 @@ public interface StatusAnalytics {
     /**
      * Returns the predicted time (in milliseconds) when backpressure is expected to be applied to this connection, based on the total number of bytes in the queue.
      * @return milliseconds until backpressure is predicted to occur, based on the total number of bytes in the queue.
+     * @param connectionId
      */
-    long getTimeToBytesBackpressureMillis();
+    long getTimeToBytesBackpressureMillis(String connectionId);
 
     /**
      * Returns the predicted time (in milliseconds) when backpressure is expected to be applied to this connection, based on the number of objects in the queue.
      * @return milliseconds until backpressure is predicted to occur, based on the number of objects in the queue.
+     * @param connectionId
      */
-    long getTimeToCountBackpressureMillis();
+    long getTimeToCountBackpressureMillis(String connectionId);
 
     /**
      * Returns the predicted total number of bytes in the queue to occur at the next configured interval (5 mins in the future, e.g.).
      * @return milliseconds until backpressure is predicted to occur, based on the total number of bytes in the queue.
+     * @param connectionId
      */
-    long getNextIntervalBytes();
+    long getNextIntervalBytes(String connectionId);
 
     /**
      * Returns the predicted number of objects in the queue to occur at the next configured interval (5 mins in the future, e.g.).
      * @return milliseconds until backpressure is predicted to occur, based on the number of bytes in the queue.
+     * @param connectionId
      */
-    int getNextIntervalCount();
+    int getNextIntervalCount(String connectionId);
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index 5f67b49..450d944 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -16,6 +16,39 @@
  */
 package org.apache.nifi.controller;
 
+import static java.util.Objects.requireNonNull;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.lang.management.GarbageCollectorMXBean;
+import java.lang.management.ManagementFactory;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import javax.net.ssl.SSLContext;
+
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.admin.service.AuditService;
 import org.apache.nifi.annotation.lifecycle.OnConfigurationRestored;
@@ -184,38 +217,6 @@ import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.net.ssl.SSLContext;
-import java.io.ByteArrayInputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.lang.management.GarbageCollectorMXBean;
-import java.lang.management.ManagementFactory;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.stream.Collectors;
-
-import static java.util.Objects.requireNonNull;
-
 public class FlowController implements ReportingTaskProvider, Authorizable, NodeTypeProvider {
 
     // default repository implementations
@@ -350,7 +351,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
     // guarded by rwLock
     private NodeConnectionStatus connectionStatus;
 
-    private CachingStatusAnalyticEngine analyticsEngine;
+    private StatusAnalytics analyticsEngine;
 
     // guarded by rwLock
     private String instanceId;
@@ -592,8 +593,10 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
             zooKeeperStateServer = null;
         }
 
-        eventAccess = new StandardEventAccess(this, flowFileEventRepository);
         componentStatusRepository = createComponentStatusRepository();
+        analyticsEngine = new CachingStatusAnalyticEngine(this, componentStatusRepository);
+        eventAccess = new StandardEventAccess(this, flowFileEventRepository);
+
         timerDrivenEngineRef.get().scheduleWithFixedDelay(new Runnable() {
             @Override
             public void run() {
@@ -605,9 +608,6 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
             }
         }, snapshotMillis, snapshotMillis, TimeUnit.MILLISECONDS);
 
-        analyticsEngine = new CachingStatusAnalyticEngine(this, componentStatusRepository);
-
-
         this.connectionStatus = new NodeConnectionStatus(nodeId, DisconnectionCode.NOT_YET_CONNECTED);
         heartbeatBeanRef.set(new HeartbeatBean(rootGroup, false));
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingStatusAnalyticEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingStatusAnalyticEngine.java
index 5241c4a..864a5d4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingStatusAnalyticEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingStatusAnalyticEngine.java
@@ -2,6 +2,7 @@ package org.apache.nifi.controller.status.analytics;
 
 import java.util.Date;
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.math3.stat.regression.SimpleRegression;
 import org.apache.nifi.connectable.Connection;
@@ -28,6 +29,7 @@ public class CachingStatusAnalyticEngine implements StatusAnalytics {
         this.controller = controller;
         this.statusRepository = statusRepository;
         this.cache = Caffeine.newBuilder()
+                .expireAfterWrite(1,TimeUnit.MINUTES)
                 .build();
     }
 
@@ -38,15 +40,37 @@ public class CachingStatusAnalyticEngine implements StatusAnalytics {
         Connection connection = rootGroup.findConnection(connectionId);
         SimpleRegression cachedRegression = cache.getIfPresent(connection.getIdentifier());
 
-        if(cachedRegression != null) {
-            cache.put(connection.getIdentifier(), cachedRegression);
+        if(cachedRegression == null) {
+            cachedRegression = getBackPressureRegressionModel(connection);
+            if(cachedRegression != null)
+                cache.put(connection.getIdentifier(), cachedRegression);
         }
 
         ConnectionStatusAnalytics cachedResult = calculate(cachedRegression,connection);
-        LOG.info("Connection: " + connectionId + " Cached backpressure Time: " + cachedResult.getMinTimeToBackpressureMillis() );
+        LOG.info("Connection: " + connectionId + " Cached backpressure Time: " + cachedResult.getTimeToCountBackpressureMillis());
         return cachedResult;
     }
 
+    @Override
+    public long getTimeToBytesBackpressureMillis(String connectionId) {
+        return 0;
+    }
+
+    @Override
+    public long getTimeToCountBackpressureMillis(String connectionId) {
+        return getConnectionStatusAnalytics(connectionId).getTimeToCountBackpressureMillis();
+    }
+
+    @Override
+    public long getNextIntervalBytes(String connectionId) {
+        return 0;
+    }
+
+    @Override
+    public int getNextIntervalCount(String connectionId) {
+        return 0;
+    }
+
     protected ConnectionStatusAnalytics calculate(SimpleRegression regression, Connection conn){
         long backPressureObjectThreshold = conn.getFlowFileQueue().getBackPressureObjectThreshold();
 
@@ -77,14 +101,30 @@ public class CachingStatusAnalyticEngine implements StatusAnalytics {
                 return conn.getName();
             }
 
+
+            @Override
+            public String getId() {
+                return conn.getIdentifier();
+            }
+
             @Override
-            public long getMinTimeToBackpressureMillis() {
+            public long getTimeToBytesBackpressureMillis() {
+                return 0;
+            }
+
+            @Override
+            public long getTimeToCountBackpressureMillis() {
                 return connTimeToBackpressure;
             }
 
             @Override
-            public String getId() {
-                return conn.getIdentifier();
+            public long getNextIntervalBytes() {
+                return 0;
+            }
+
+            @Override
+            public int getNextIntervalCount() {
+                return 0;
             }
 
             @Override
@@ -111,7 +151,7 @@ public class CachingStatusAnalyticEngine implements StatusAnalytics {
      * @return
      */
     protected SimpleRegression getBackPressureRegressionModel(Connection conn) {
-        Date minDate = new Date(System.currentTimeMillis() - (5 * 60 * 1000));
+        Date minDate = new Date(System.currentTimeMillis() - (60 * 1000));
         StatusHistoryDTO connHistory = StatusHistoryUtil.createStatusHistoryDTO(
                 statusRepository.getConnectionStatusHistory(conn.getIdentifier(), minDate, null, Integer.MAX_VALUE));
         List<StatusSnapshotDTO> aggregateSnapshots = connHistory.getAggregateSnapshots();
@@ -128,7 +168,6 @@ public class CachingStatusAnalyticEngine implements StatusAnalytics {
                 Long snapQueuedCount = snap.getStatusMetrics().get(ConnectionStatusDescriptor.QUEUED_COUNT.getField());
                 long snapTime = snap.getTimestamp().getTime();
                 regression.addData(snapTime, snapQueuedCount);
-                LOG.info("Connection " + conn.getIdentifier() + " statistics: ("+snapTime+","+snapQueuedCount+")");
             }
 
             if (regression.getSlope() <= 0 && !conn.getFlowFileQueue().isFull()) {
@@ -141,20 +180,5 @@ public class CachingStatusAnalyticEngine implements StatusAnalytics {
 
     }
 
-    public void refreshModel() {
-        ProcessGroup rootGroup = controller.getFlowManager().getRootGroup();
-        List<Connection> allConnections = rootGroup.findAllConnections();
-        cache.invalidateAll();
-        for (Connection conn : allConnections) {
-            SimpleRegression regression = getBackPressureRegressionModel(conn);
-            if(regression != null) {
-                cache.put(conn.getIdentifier(), regression);
-            }
-        }
-    }
 
-    @Override
-    public long getMinTimeToBackpressureMillis() {
-        return 0;
-    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java
index 024c138..5a873d5 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java
@@ -172,22 +172,22 @@ public class StatusAnalyticEngine implements StatusAnalytics {
 
     // TODO - populate the prediction fields. Do we need to pass in connection ID?
     @Override
-    public long getTimeToCountBackpressureMillis() {
+    public long getTimeToCountBackpressureMillis(String connectionId) {
         return 0;
     }
 
     @Override
-    public long getTimeToBytesBackpressureMillis() {
+    public long getTimeToBytesBackpressureMillis(String connectionId) {
         return 0;
     }
 
     @Override
-    public long getNextIntervalBytes() {
+    public long getNextIntervalBytes(String connectionId) {
         return 0;
     }
 
     @Override
-    public int getNextIntervalCount() {
+    public int getNextIntervalCount(String connectionId) {
         return 0;
     }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java
index f943856..87fcd4d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java
@@ -65,10 +65,12 @@ import org.apache.nifi.remote.RemoteGroupPort;
 public class StandardEventAccess implements UserAwareEventAccess {
     private final FlowFileEventRepository flowFileEventRepository;
     private final FlowController flowController;
+    private final StatusAnalytics statusAnalytics;
 
     public StandardEventAccess(final FlowController flowController, final FlowFileEventRepository flowFileEventRepository) {
         this.flowController = flowController;
         this.flowFileEventRepository = flowFileEventRepository;
+        this.statusAnalytics = flowController.getStatusAnalytics();
     }
 
     /**
@@ -339,12 +341,11 @@ public class StandardEventAccess implements UserAwareEventAccess {
                 bytesTransferred += connectionStatusReport.getContentSizeIn() + connectionStatusReport.getContentSizeOut();
             }
 
-            final StatusAnalytics statusAnalytics = flowController.getStatusAnalytics();
             if (statusAnalytics != null) {
-                connStatus.setPredictedTimeToBytesBackpressureMillis(statusAnalytics.getTimeToBytesBackpressureMillis());
-                connStatus.setPredictedTimeToCountBackpressureMillis(statusAnalytics.getTimeToCountBackpressureMillis());
-                connStatus.setNextPredictedQueuedBytes(statusAnalytics.getNextIntervalBytes());
-                connStatus.setNextPredictedQueuedCount(statusAnalytics.getNextIntervalCount());
+                connStatus.setPredictedTimeToBytesBackpressureMillis(statusAnalytics.getTimeToBytesBackpressureMillis(conn.getIdentifier()));
+                connStatus.setPredictedTimeToCountBackpressureMillis(statusAnalytics.getTimeToCountBackpressureMillis(conn.getIdentifier()));
+                connStatus.setNextPredictedQueuedBytes(statusAnalytics.getNextIntervalBytes(conn.getIdentifier()));
+                connStatus.setNextPredictedQueuedCount(statusAnalytics.getNextIntervalCount(conn.getIdentifier()));
             }
 
             if (isConnectionAuthorized) {


[nifi] 09/21: NIFI-6510 Revert "DFA-9 Remove redundant connection prediction interfaces as we can just use ConnectionStatusAnalytics directly"

Posted by ai...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

aichrist pushed a commit to branch analytics-framework
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit d1a9a923bba2e8483d5ab48bf3dc8b05ba3a5b74
Author: Matthew Burgess <ma...@apache.org>
AuthorDate: Wed Jul 24 13:06:51 2019 -0400

    NIFI-6510 Revert "DFA-9 Remove redundant connection prediction interfaces as we can just use ConnectionStatusAnalytics directly"
    
    This reverts commit 5b9fead1471059098c0e98343fb337070f1c75c1.
---
 .../status/analytics/StatusAnalytics.java          | 28 ++++++++++++++++++++++
 .../analytics/CachingStatusAnalyticEngine.java     | 20 ++++++++++++++++
 .../status/analytics/StatusAnalyticEngine.java     | 22 +++++++++++++++++
 .../apache/nifi/reporting/StandardEventAccess.java | 10 ++++----
 4 files changed, 74 insertions(+), 6 deletions(-)

diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java
index 131531f..45e1c12 100644
--- a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java
+++ b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalytics.java
@@ -28,4 +28,32 @@ public interface StatusAnalytics {
      * @return A ConnectionStatusAnalytics object
      */
     ConnectionStatusAnalytics getConnectionStatusAnalytics(String connectionId);
+
+    /**
+     * Returns the predicted time (in milliseconds) when backpressure is expected to be applied to this connection, based on the total number of bytes in the queue.
+     * @return milliseconds until backpressure is predicted to occur, based on the total number of bytes in the queue.
+     * @param connectionId
+     */
+    long getTimeToBytesBackpressureMillis(String connectionId);
+
+    /**
+     * Returns the predicted time (in milliseconds) when backpressure is expected to be applied to this connection, based on the number of objects in the queue.
+     * @return milliseconds until backpressure is predicted to occur, based on the number of objects in the queue.
+     * @param connectionId
+     */
+    long getTimeToCountBackpressureMillis(String connectionId);
+
+    /**
+     * Returns the predicted total number of bytes in the queue to occur at the next configured interval (5 mins in the future, e.g.).
+     * @return milliseconds until backpressure is predicted to occur, based on the total number of bytes in the queue.
+     * @param connectionId
+     */
+    long getNextIntervalBytes(String connectionId);
+
+    /**
+     * Returns the predicted number of objects in the queue to occur at the next configured interval (5 mins in the future, e.g.).
+     * @return milliseconds until backpressure is predicted to occur, based on the number of bytes in the queue.
+     * @param connectionId
+     */
+    int getNextIntervalCount(String connectionId);
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingStatusAnalyticEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingStatusAnalyticEngine.java
index 015d6f8..864a5d4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingStatusAnalyticEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingStatusAnalyticEngine.java
@@ -51,6 +51,26 @@ public class CachingStatusAnalyticEngine implements StatusAnalytics {
         return cachedResult;
     }
 
+    @Override
+    public long getTimeToBytesBackpressureMillis(String connectionId) {
+        return 0;
+    }
+
+    @Override
+    public long getTimeToCountBackpressureMillis(String connectionId) {
+        return getConnectionStatusAnalytics(connectionId).getTimeToCountBackpressureMillis();
+    }
+
+    @Override
+    public long getNextIntervalBytes(String connectionId) {
+        return 0;
+    }
+
+    @Override
+    public int getNextIntervalCount(String connectionId) {
+        return 0;
+    }
+
     protected ConnectionStatusAnalytics calculate(SimpleRegression regression, Connection conn){
         long backPressureObjectThreshold = conn.getFlowFileQueue().getBackPressureObjectThreshold();
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java
index 56c263e..5a873d5 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticEngine.java
@@ -158,6 +158,7 @@ public class StatusAnalyticEngine implements StatusAnalytics {
     public long getMinTimeToBackpressureMillis() {
         ProcessGroup rootGroup = controller.getFlowManager().getRootGroup();
         List<Connection> allConnections = rootGroup.findAllConnections();
+        rootGroup.findConnection("asdf");
         long minTimeToBackpressure = Long.MAX_VALUE;
 
         for (Connection conn : allConnections) {
@@ -168,4 +169,25 @@ public class StatusAnalyticEngine implements StatusAnalytics {
         LOG.info("Min time to backpressure is: " + Long.toString(minTimeToBackpressure));
         return minTimeToBackpressure;
     }
+
+    // TODO - populate the prediction fields. Do we need to pass in connection ID?
+    @Override
+    public long getTimeToCountBackpressureMillis(String connectionId) {
+        return 0;
+    }
+
+    @Override
+    public long getTimeToBytesBackpressureMillis(String connectionId) {
+        return 0;
+    }
+
+    @Override
+    public long getNextIntervalBytes(String connectionId) {
+        return 0;
+    }
+
+    @Override
+    public int getNextIntervalCount(String connectionId) {
+        return 0;
+    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java
index aeb9559..87fcd4d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/reporting/StandardEventAccess.java
@@ -51,7 +51,6 @@ import org.apache.nifi.controller.status.ProcessorStatus;
 import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
 import org.apache.nifi.controller.status.RunStatus;
 import org.apache.nifi.controller.status.TransmissionStatus;
-import org.apache.nifi.controller.status.analytics.ConnectionStatusAnalytics;
 import org.apache.nifi.controller.status.analytics.StatusAnalytics;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
@@ -343,11 +342,10 @@ public class StandardEventAccess implements UserAwareEventAccess {
             }
 
             if (statusAnalytics != null) {
-                ConnectionStatusAnalytics connectionStatusAnalytics = statusAnalytics.getConnectionStatusAnalytics(conn.getIdentifier());
-                connStatus.setPredictedTimeToBytesBackpressureMillis(connectionStatusAnalytics.getTimeToBytesBackpressureMillis());
-                connStatus.setPredictedTimeToCountBackpressureMillis(connectionStatusAnalytics.getTimeToCountBackpressureMillis());
-                connStatus.setNextPredictedQueuedBytes(connectionStatusAnalytics.getNextIntervalBytes());
-                connStatus.setNextPredictedQueuedCount(connectionStatusAnalytics.getNextIntervalCount());
+                connStatus.setPredictedTimeToBytesBackpressureMillis(statusAnalytics.getTimeToBytesBackpressureMillis(conn.getIdentifier()));
+                connStatus.setPredictedTimeToCountBackpressureMillis(statusAnalytics.getTimeToCountBackpressureMillis(conn.getIdentifier()));
+                connStatus.setNextPredictedQueuedBytes(statusAnalytics.getNextIntervalBytes(conn.getIdentifier()));
+                connStatus.setNextPredictedQueuedCount(statusAnalytics.getNextIntervalCount(conn.getIdentifier()));
             }
 
             if (isConnectionAuthorized) {