NIFI-6510 - Analytics framework (#3681)

* NIFI-6510 Implement initial analytic engine

* NIFI-6510 Implemented basic linear regression model for queue counts

* NIFI-6510 Initial analytics REST endpoint and supporting objects

* NIFI-6510 Connect the dots for StatusAnalytics -> API

* 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-6510 Updated objects and interfaces to reflect 4 prediction metrics

(cherry picked from commit 050e0fc)

(cherry picked from commit 9fd365f)

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

* NIFI-6510 Split StatusAnalytics interface into Engine and per-Connection versions

* NIFI-6510 Remove redundant connection prediction interfaces as we can just use ConnectionStatusAnalytics directly

* NIFI-6510 Revert "DFA-9 Remove redundant connection prediction interfaces as we can just use ConnectionStatusAnalytics directly"

This reverts commit 5b9fead1471059098c0e98343fb337070f1c75c1.

* NIFI-6510 Added prediction fields for use by UI, still need to be populated

* 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-6510 Add prediction percent values and predicted interval seconds

(cherry picked from commit e60015d)

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

* NIFI-6510 Added tests for engine

(cherry picked from commit 6d7a13b)

* NIFI-6150 Added tests for connection status analytics class, corrected variable names

(cherry picked from commit 58c7c81)

* NIFI-6150 Make checkstyle happy

(cherry picked from commit b6e35ac)

* NIFI-6150 Fixed NaN check and refactored time prediction. Switched to use non caching engine for testing

* NIFI-6510 Fixed checkstyle issue in TestConnectionStatusAnalytics

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

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

* NIFI-6566 - Refactor to decouple model instance from status analytics object. Also allow configurable model from nifi.properties

NIFI-6566 - changes to allow scoring configurations for model in nifi.properties

NIFI-6566 - added default implementation value to NiFiProperties

NIFI-6566 - correction to default variable name in NiFiProperties, removed unnecessary init method from ConnectionStatusAnalytics

Signed-off-by: Matthew Burgess <mattyb149@apache.org>

This closes #3663

* NIFI-6585 - Refactored tests to use mocked models and extract functions.  Added check in ConnectionStatusAnalytics to confirm expected model by type

* NIFI-6586 - documentation and comments

This closes NIFI-6586

Signed-off-by: Andrew I. Christianson <andy@andyic.org>

* NIFI-6568 - Surface time-to-back-pressure and initial predictions in the UI
* Add multi-line tooltips with detail for connection queue back pressure graphics.
* Add estimated time to back pressure to connections summary table.
* Add back pressure prediction ticks.
* add moment.js to format predicted time to back pressure
* tweak summary table headings to match data displayed. re-order connection summary columns

* NIFI-6568 - Properly sort the min estimated time to back pressure in the connection summary table. Also added a js doc comment.

* NIFI-6510 - add an enable/disable property for analytics

* NIFI-6510 - documentation updates for enable/disable property

* NIFI-6510 - UI: handle the scenario where backpressure predictions are disabled (#3685)

* NIFI-6510 - admin guide updates to further describe model functionality

* NIFI-6510 - code quality fixes (if statement and constructor)

* NIFI-6510 - log warnings when properties could not be retrieved. fixed incorrect property retrieval for score threshold

* NIFI-6510 Extract out predictions into their own DTO

* NIFI-6510 Optimize imports

* NIFI-6510 Fix formatting

* NIFI-6510 Optimize imports

* NIFI-6510 Optimize imports

* NIFI-6510 - Notice updates for Commons math and Caffeine

* NIFI-6510 - UI updates to account for minor API changes for back pressure predictions (#3697)

* NIFI-6510 - Fix issue displaying estimated time to back pressure in connection summary table when only one of the predictions is known.

Signed-off-by: Matthew Burgess <mattyb149@apache.org>

This closes #3705

* NIFI-6510 Rip out useless members

* NIFI-6510 - dto updates to check for -1 value

* NIFI-6510 - checkstyle fix

* NIFI-6510 - rolled back last change and applied minNonNegative method

* NIFI-6510 Rip out useless members
This commit is contained in:
Andy I. Christianson 2019-09-09 15:37:11 +00:00 committed by GitHub
parent 625d4a13ca
commit 8a8b9c1d08
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
71 changed files with 3991 additions and 231 deletions

View File

@ -16,6 +16,7 @@
*/
package org.apache.nifi.controller.status;
import org.apache.nifi.controller.status.analytics.ConnectionStatusPredictions;
import org.apache.nifi.processor.DataUnit;
/**
@ -30,6 +31,7 @@ public class ConnectionStatus implements Cloneable {
private String destinationId;
private String destinationName;
private String backPressureDataSizeThreshold;
private ConnectionStatusPredictions predictions;
private long backPressureBytesThreshold;
private long backPressureObjectThreshold;
private int inputCount;
@ -122,6 +124,14 @@ public class ConnectionStatus implements Cloneable {
setBackPressureBytesThreshold(DataUnit.parseDataSize(backPressureDataSizeThreshold, DataUnit.B).longValue());
}
public ConnectionStatusPredictions getPredictions() {
return predictions;
}
public void setPredictions(ConnectionStatusPredictions predictions) {
this.predictions = predictions;
}
public long getBackPressureObjectThreshold() {
return backPressureObjectThreshold;
}
@ -202,6 +212,11 @@ public class ConnectionStatus implements Cloneable {
clonedObj.sourceName = sourceName;
clonedObj.destinationId = destinationId;
clonedObj.destinationName = destinationName;
if (predictions != null) {
clonedObj.setPredictions(predictions.clone());
}
clonedObj.backPressureDataSizeThreshold = backPressureDataSizeThreshold;
clonedObj.backPressureObjectThreshold = backPressureObjectThreshold;
clonedObj.maxQueuedBytes = maxQueuedBytes;

View File

@ -0,0 +1,117 @@
/*
* 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 class ConnectionStatusPredictions implements Cloneable {
private long predictionIntervalMillis;
private int nextPredictedQueuedCount;
private long nextPredictedQueuedBytes;
private long predictedTimeToCountBackpressureMillis;
private long predictedTimeToBytesBackpressureMillis;
private int predictedPercentCount = 0;
private int predictedPercentBytes = 0;
public long getPredictionIntervalMillis() {
return predictionIntervalMillis;
}
public void setPredictionIntervalMillis(long predictionIntervalMillis) {
this.predictionIntervalMillis = predictionIntervalMillis;
}
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;
}
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 ConnectionStatusPredictions clone() {
final ConnectionStatusPredictions clonedObj = new ConnectionStatusPredictions();
clonedObj.nextPredictedQueuedBytes = nextPredictedQueuedBytes;
clonedObj.nextPredictedQueuedCount = nextPredictedQueuedCount;
clonedObj.predictedTimeToBytesBackpressureMillis = predictedTimeToBytesBackpressureMillis;
clonedObj.predictedTimeToCountBackpressureMillis = predictedTimeToCountBackpressureMillis;
clonedObj.predictedPercentCount = predictedPercentCount;
clonedObj.predictedPercentBytes = predictedPercentBytes;
return clonedObj;
}
@Override
public String toString() {
StringBuilder builder = new StringBuilder();
builder.append("ConnectionStatusPredictions [id=");
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(", predictedPercentCount=");
builder.append(predictedPercentCount);
builder.append(", predictedPercentBytes=");
builder.append(predictedPercentBytes);
builder.append("]");
return builder.toString();
}
}

View File

@ -0,0 +1,62 @@
/*
* 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;
/**
* QueryWindow is used to track time intervals for querying for observations
*/
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;
}
}

View File

@ -0,0 +1,46 @@
/*
* 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.Map;
/**
* The StatusAnalytics interface offers methods for accessing predicted and other values for a single component (Connection instance, e.g.).
*/
public interface StatusAnalytics {
/**
* Get the Query Window used by the analytics instance
* @return queryWindow
*/
QueryWindow getQueryWindow();
/**
* Get available predictions where the key (String) in the map is the name of the prediction and value (Long)
* is the value for the prediction
* @return map
*/
Map<String,Long> getPredictions();
/**
* Return if analytics object supports online learning
* @return boolean
*/
boolean supportsOnlineLearning();
}

View File

@ -0,0 +1,28 @@
/*
* 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 {
/**
* Retrieve status analytics object for given component
* @param componentId identifier for component
* @return componenet specific status analytics object
*/
StatusAnalytics getStatusAnalytics(String componentId);
}

View File

@ -0,0 +1,64 @@
/*
* 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.Map;
import java.util.stream.Stream;
public interface StatusAnalyticsModel {
/**
* Train model with provided observations (features, labels/targets)
* @param features Stream of feature observation values
* @param labels target observation values
*/
void learn(Stream<Double[]> features, Stream<Double> labels);
/**
* Return a prediction given observation values
* @param feature feature observation values values
* @return prediction of target/label
*/
Double predict(Double[] feature);
/**
* Predict a feature given a known target and known predictor values (if multiple predictors are included with model)
* @param predictVariableIndex index of feature that we would like to predict (index should align with order provided in model learn method)
* @param knownVariablesWithIndex a map of known predictor values with their indexes if available
* @param label known target value
* @return prediction for variable
*/
Double predictVariable(Integer predictVariableIndex, Map<Integer,Double> knownVariablesWithIndex, Double label);
/**
* Indicate if model supports online learning (e.g. can learn new observation samples to create a model)
* @return boolean indicating online learning support
*/
Boolean supportsOnlineLearning();
/**
* Returns a map of scores relevant to model (e.g. rSquared, Confidence Intervals, etc.)
* @return Map of score names with values
*/
Map<String,Double> getScores();
/**
* Resets a model by clearing observations and other calculations
*/
void clear();
}

View File

@ -2583,6 +2583,31 @@ This product bundles 'lodash' which is available under an MIT license.
licenses; we recommend you read them, as their terms may differ from the
terms above.
This product bundles 'moment' which is available under an MIT license.
Copyright (c) JS Foundation and other contributors
Permission is hereby granted, free of charge, to any person
obtaining a copy of this software and associated documentation
files (the "Software"), to deal in the Software without
restriction, including without limitation the rights to use,
copy, modify, merge, publish, distribute, sublicense, and/or sell
copies of the Software, and to permit persons to whom the
Software is furnished to do so, subject to the following
conditions:
The above copyright notice and this permission notice shall be
included in all copies or substantial portions of the Software.
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES
OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT
HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY,
WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
OTHER DEALINGS IN THE SOFTWARE.
The binary distribution of this product bundles 'normalize.css'
NORMALIZE.CSS LICENSE

View File

@ -238,6 +238,13 @@ 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_ENABLED = "nifi.analytics.predict.enabled";
public static final String ANALYTICS_PREDICTION_INTERVAL = "nifi.analytics.predict.interval";
public static final String ANALYTICS_CONNECTION_MODEL_IMPLEMENTATION = "nifi.analytics.connection.model.implementation";
public static final String ANALYTICS_CONNECTION_MODEL_SCORE_NAME= "nifi.analytics.connection.model.score.name";
public static final String ANALYTICS_CONNECTION_MODEL_SCORE_THRESHOLD = "nifi.analytics.connection.model.score.threshold";
// defaults
public static final Boolean DEFAULT_AUTO_RESUME_STATE = true;
public static final String DEFAULT_AUTHORIZER_CONFIGURATION_FILE = "conf/authorizers.xml";
@ -308,6 +315,12 @@ 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_ENABLED = "false";
public static final String DEFAULT_ANALYTICS_PREDICTION_INTERVAL = "3 mins";
public final static String DEFAULT_ANALYTICS_CONNECTION_MODEL_IMPLEMENTATION = "org.apache.nifi.controller.status.analytics.models.OrdinaryLeastSquares";
public static final String DEFAULT_ANALYTICS_CONNECTION_SCORE_NAME = "rSquared";
public static final double DEFAULT_ANALYTICS_CONNECTION_SCORE_THRESHOLD = .90;
/**
* Retrieves the property value for the given property key.

View File

@ -120,3 +120,7 @@ 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
nifi.analytics.connection.model.implementation=org.apache.nifi.controller.status.analytics.models.OrdinaryLeastSquares

View File

@ -2379,6 +2379,21 @@ root@kdc:~# ll /http*
root@kdc:~#
....
[[analytics_framework]]
== Analytics Framework
NiFi has an internal analytics framework which can be enabled to predict back pressure occurrence, given the configured settings for threshold on a queue. The model used by default for prediction is an ordinary least squares (OLS) linear regression. It uses recent observations from a queue (either number of objects or content size over time) and calculates a regression line for that data. The line's equation is then used to determine the next value that will be reached within a given time interval (e.g. number of objects in queue in the next 5 minutes). Below is an example graph of the linear regression model for Queue/Object Count over time which is used for predictions:
image:back_pressure_prediction_model_example.png["Back pressure prediction based on Queue/Object Count"]
In order to generate predictions, local status snapshot history is queried to obtain enough data to generate a model. By default component status snapshots are captured every minute. Internal models need at least 2 or more observations to generate a prediction, therefore it may take up to 2 or more minutes for predictions to be available by default. If predictions are needed sooner than what is provided by default, the timing of snapshots can be adjusted using the `nifi.components.status.snapshot.frequency` value in nifi.properties.
NiFi evaluates the model's effectiveness before sending prediction information by using the model's R-Squared score by default. One important note: R-Square is a measure of how close the regression line fits the observation data vs. how accurate the prediction will be; therefore there may be some measure of error. If the R-Squared score for the calculated model meets the configured threshold (as defined by `nifi.analytics.connection.model.score.threshold`) then the model will be used for prediction. Otherwise the model will not be used and predictions will not be available until a model is generated with a score that exceeds the threshold. Default R-Squared threshold value is `.9` however this can be tuned based on prediction requirements.
The prediction interval `nifi.analytics.predict.interval` can be configured to project out further when back pressure will occur. Predictions further out in time require more observations stored locally to generate an effective model. This may also require tuning of the model's scoring threshold value to select a score which can offer reasonable predictions.
See <<analytics_properties>> for complete information on configuring analytic properties.
[[system_properties]]
== System Properties
The _nifi.properties_ file in the `conf` directory is the main configuration file for controlling how NiFi runs. This section provides an overview of the properties in this file and includes some notes on how to configure it in a way that will make upgrading easier. *After making changes to this file, restart NiFi in order
@ -3317,6 +3332,21 @@ 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 predictive analytics capability, such as backpressure prediction, and should be configured the same way on all nodes.
|====
|*Property*|*Description*
|`nifi.analytics.predict.enabled`|This indicates whether prediction should be enabled for the cluster. The default is `false`.
|`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`.
|`nifi.analytics.connection.model.implementation`|This is the implementation class for the status analytics model used to make connection predictions. The default value is `org.apache.nifi.controller.status.analytics.models.OrdinaryLeastSquares`.
|`nifi.analytics.connection.model.score.name`|This is the name of the scoring type that should be used to evaluate model. The default value is `rSquared`.
|`nifi.analytics.connection.model.score.threshold`|This is the threshold for the scoring value (where model score should be above given threshold). The default value is `.9`.
|====
[[custom_properties]]
=== Custom Properties

Binary file not shown.

After

Width:  |  Height:  |  Size: 20 KiB

View File

@ -1685,6 +1685,8 @@ The FlowFiles enqueued in a Connection can also be deleted when necessary. The r
via `Empty queue` in the Connection's context menu. This action can also be performed if the source and destination
are actively running.
If the analytics prediction feature is enabled, hovering over the queue will also reveal predicted statistics on when the queue may encounter back pressure, either due to the object count or content size meeting the current threshold
settings. Predictions will only be available when NiFi has enough data in it's internal repository and if it's model is accurate enough to broadcast a prediction.
[[Summary_Page]]
=== Summary Page

View File

@ -853,3 +853,28 @@ This product bundles 'lodash' which is available under an MIT license.
maintained libraries used by this software which have their own
licenses; we recommend you read them, as their terms may differ from the
terms above.
This product bundles 'moment' which is available under an MIT license.
Copyright (c) JS Foundation and other contributors
Permission is hereby granted, free of charge, to any person
obtaining a copy of this software and associated documentation
files (the "Software"), to deal in the Software without
restriction, including without limitation the rights to use,
copy, modify, merge, publish, distribute, sublicense, and/or sell
copies of the Software, and to permit persons to whom the
Software is furnished to do so, subject to the following
conditions:
The above copyright notice and this permission notice shall be
included in all copies or substantial portions of the Software.
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES
OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT
HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY,
WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
OTHER DEALINGS IN THE SOFTWARE.

View File

@ -81,6 +81,18 @@ The following binary components are provided under the Apache Software License v
This product includes software from the Spring Framework,
under the Apache License 2.0 (see: StringUtils.containsWhitespace())
(ASLv2) Apache Commons Math
The following NOTICE information applies:
Apache Commons Math
Copyright 2001-2016 The Apache Software Foundation
This product includes software developed at
The Apache Software Foundation (http://www.apache.org/).
This product includes software developed for Orekit by
CS Systèmes d'Information (http://www.c-s.fr/)
Copyright 2010-2012 CS Systèmes d'Information
(ASLv2) Spring Framework
The following NOTICE information applies:
Spring Framework 4.1.6.RELEASE
@ -156,6 +168,11 @@ The following binary components are provided under the Apache Software License v
* Brian Langel
(ASLv2) Caffeine
The following NOTICE information applies:
Caffeine (caching library)
Copyright Ben Manes
************************
Common Development and Distribution License 1.1
************************

View File

@ -0,0 +1,94 @@
/*
* 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 Date statsLastRefreshed;
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 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.setId(getId());
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;
}
}

View File

@ -0,0 +1,130 @@
/*
* 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 javax.xml.bind.annotation.XmlType;
import io.swagger.annotations.ApiModelProperty;
/**
* DTO for serializing the statistics of a connection.
*/
@XmlType(name = "connectionStatisticsSnapshot")
public class ConnectionStatisticsSnapshotDTO implements Cloneable {
private String id;
private Long predictedMillisUntilCountBackpressure = 0L;
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 */
/**
* @return The connection id
*/
@ApiModelProperty("The id of the connection.")
public String getId() {
return id;
}
public void setId(String id) {
this.id = id;
}
@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;
}
@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();
other.setId(getId());
other.setPredictedMillisUntilCountBackpressure(getPredictedMillisUntilCountBackpressure());
other.setPredictedMillisUntilBytesBackpressure(getPredictedMillisUntilBytesBackpressure());
other.setPredictedCountAtNextInterval(getPredictedCountAtNextInterval());
other.setPredictedBytesAtNextInterval(getPredictedBytesAtNextInterval());
other.setPredictedPercentCount(getPredictedPercentCount());
other.setPredictedPercentBytes(getPredictedPercentBytes());
other.setPredictionIntervalMillis(getPredictionIntervalMillis());
return other;
}
}

View File

@ -0,0 +1,112 @@
/*
* 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 status predictions of a connection.
*/
@XmlType(name = "connectionStatusSnapshot")
public class ConnectionStatusPredictionsSnapshotDTO implements Cloneable {
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;
@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 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;
}
public void setPredictedBytesAtNextInterval(Long predictedBytesAtNextInterval) {
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 ConnectionStatusPredictionsSnapshotDTO clone() {
final ConnectionStatusPredictionsSnapshotDTO other = new ConnectionStatusPredictionsSnapshotDTO();
other.setPredictedMillisUntilCountBackpressure(getPredictedMillisUntilCountBackpressure());
other.setPredictedMillisUntilBytesBackpressure(getPredictedMillisUntilBytesBackpressure());
other.setPredictionIntervalSeconds(getPredictionIntervalSeconds());
other.setPredictedCountAtNextInterval(getPredictedCountAtNextInterval());
other.setPredictedBytesAtNextInterval(getPredictedBytesAtNextInterval());
other.setPredictedPercentBytes(getPredictedPercentBytes());
other.setPredictedPercentCount(getPredictedPercentCount());
return other;
}
}

View File

@ -34,7 +34,7 @@ public class ConnectionStatusSnapshotDTO implements Cloneable {
private String sourceName;
private String destinationId;
private String destinationName;
private ConnectionStatusPredictionsSnapshotDTO predictions;
private Integer flowFilesIn = 0;
private Long bytesIn = 0L;
private String input;
@ -181,6 +181,18 @@ public class ConnectionStatusSnapshotDTO implements Cloneable {
this.destinationName = destinationName;
}
/**
* @return predictions for this connection
*/
@ApiModelProperty("Predictions, if available, for this connection (null if not available)")
public ConnectionStatusPredictionsSnapshotDTO getPredictions() {
return predictions;
}
public void setPredictions(ConnectionStatusPredictionsSnapshotDTO predictions) {
this.predictions = predictions;
}
/**
* @return input for this connection
*/
@ -282,6 +294,10 @@ public class ConnectionStatusSnapshotDTO implements Cloneable {
other.setSourceId(getSourceId());
other.setSourceName(getSourceName());
if (predictions != null) {
other.setPredictions(predictions.clone());
}
other.setFlowFilesIn(getFlowFilesIn());
other.setBytesIn(getBytesIn());
other.setInput(getInput());

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -0,0 +1,75 @@
/*
* 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;
/**
* 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;
}
}

View File

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

View File

@ -37,6 +37,7 @@ import org.apache.nifi.web.api.dto.diagnostics.JVMDiagnosticsSnapshotDTO;
import org.apache.nifi.web.api.dto.diagnostics.JVMFlowDiagnosticsSnapshotDTO;
import org.apache.nifi.web.api.dto.diagnostics.JVMSystemDiagnosticsSnapshotDTO;
import org.apache.nifi.web.api.dto.status.ConnectionStatusDTO;
import org.apache.nifi.web.api.dto.status.ConnectionStatusPredictionsSnapshotDTO;
import org.apache.nifi.web.api.dto.status.ConnectionStatusSnapshotDTO;
import org.apache.nifi.web.api.dto.status.ControllerStatusDTO;
import org.apache.nifi.web.api.dto.status.NodeConnectionStatusSnapshotDTO;
@ -500,9 +501,56 @@ public class StatusMerger {
target.setPercentUseCount(Math.max(target.getPercentUseCount(), toMerge.getPercentUseCount()));
}
// Merge predicted values (minimum time to backpressure, maximum percent at next interval
ConnectionStatusPredictionsSnapshotDTO targetPredictions = target.getPredictions();
ConnectionStatusPredictionsSnapshotDTO toMergePredictions = toMerge.getPredictions();
if (targetPredictions == null) {
target.setPredictions(toMergePredictions);
} else if (toMergePredictions != null) {
if (targetPredictions.getPredictionIntervalSeconds() == null) {
targetPredictions.setPredictionIntervalSeconds(toMergePredictions.getPredictionIntervalSeconds());
}
if (targetPredictions.getPredictedMillisUntilBytesBackpressure() == null) {
targetPredictions.setPredictedMillisUntilBytesBackpressure(toMergePredictions.getPredictedMillisUntilBytesBackpressure());
} else if (toMergePredictions.getPredictedMillisUntilBytesBackpressure() != null) {
targetPredictions.setPredictedMillisUntilBytesBackpressure(minNonNegative(targetPredictions.getPredictedMillisUntilBytesBackpressure(),
toMergePredictions.getPredictedMillisUntilBytesBackpressure()));
}
if (targetPredictions.getPredictedMillisUntilCountBackpressure() == null) {
targetPredictions.setPredictedMillisUntilCountBackpressure(toMergePredictions.getPredictedMillisUntilCountBackpressure());
} else if (toMergePredictions.getPredictedMillisUntilCountBackpressure() != null) {
targetPredictions.setPredictedMillisUntilCountBackpressure(minNonNegative(targetPredictions.getPredictedMillisUntilCountBackpressure(),
toMergePredictions.getPredictedMillisUntilCountBackpressure()));
}
if (targetPredictions.getPredictedPercentBytes() == null) {
targetPredictions.setPredictedPercentBytes(toMergePredictions.getPredictedPercentBytes());
} else if (toMerge.getPercentUseBytes() != null) {
targetPredictions.setPredictedPercentBytes(Math.max(targetPredictions.getPredictedPercentBytes(),
toMergePredictions.getPredictedPercentBytes()));
}
if (targetPredictions.getPredictedPercentCount() == null) {
targetPredictions.setPredictedPercentCount(toMergePredictions.getPredictedPercentCount());
} else if (toMergePredictions.getPredictedPercentCount() != null) {
targetPredictions.setPredictedPercentCount(Math.max(targetPredictions.getPredictedPercentCount(),
toMergePredictions.getPredictedPercentCount()));
}
}
updatePrettyPrintedFields(target);
}
private static long minNonNegative(long a, long b){
if(a < 0){
return b;
}else if(b < 0){
return a;
}else{
return Math.min(a, b);
}
}
public static void updatePrettyPrintedFields(final ConnectionStatusSnapshotDTO target) {
target.setQueued(prettyPrint(target.getFlowFilesQueued(), target.getBytesQueued()));
target.setQueuedCount(formatCount(target.getFlowFilesQueued()));
@ -520,7 +568,8 @@ public class StatusMerger {
merge(target.getRemoteProcessGroupStatusSnapshot(), target.getCanRead(), toMerge.getRemoteProcessGroupStatusSnapshot(), toMerge.getCanRead());
}
public static void merge(final RemoteProcessGroupStatusSnapshotDTO target, final boolean targetReadablePermission, final RemoteProcessGroupStatusSnapshotDTO toMerge,
public static void merge(final RemoteProcessGroupStatusSnapshotDTO target, final boolean targetReadablePermission,
final RemoteProcessGroupStatusSnapshotDTO toMerge,
final boolean toMergeReadablePermission) {
if (target == null || toMerge == null) {
return;

View File

@ -133,6 +133,11 @@
<groupId>commons-io</groupId>
<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>
@ -236,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>

View File

@ -16,13 +16,45 @@
*/
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;
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;
@ -117,6 +149,11 @@ 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.StatusAnalyticsEngine;
import org.apache.nifi.controller.status.analytics.StatusAnalyticsModel;
import org.apache.nifi.controller.status.analytics.StatusAnalyticsModelMapFactory;
import org.apache.nifi.controller.status.analytics.StatusMetricExtractFunction;
import org.apache.nifi.controller.status.history.ComponentStatusRepository;
import org.apache.nifi.controller.status.history.GarbageCollectionHistory;
import org.apache.nifi.controller.status.history.GarbageCollectionStatus;
@ -139,6 +176,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;
@ -175,6 +213,7 @@ import org.apache.nifi.util.ComponentIdGenerator;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.util.ReflectionUtils;
import org.apache.nifi.util.Tuple;
import org.apache.nifi.util.concurrency.TimedLock;
import org.apache.nifi.web.api.dto.PositionDTO;
import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
@ -182,38 +221,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
@ -348,6 +355,8 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
// guarded by rwLock
private NodeConnectionStatus connectionStatus;
private StatusAnalyticsEngine analyticsEngine;
// guarded by rwLock
private String instanceId;
@ -588,8 +597,46 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
zooKeeperStateServer = null;
}
eventAccess = new StandardEventAccess(this, flowFileEventRepository);
componentStatusRepository = createComponentStatusRepository();
final boolean analyticsEnabled = Boolean.parseBoolean(nifiProperties.getProperty(NiFiProperties.ANALYTICS_PREDICTION_ENABLED, NiFiProperties.DEFAULT_ANALYTICS_PREDICTION_ENABLED));
if(analyticsEnabled) {
// 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) {
LOG.warn("Analytics is enabled however could not retrieve value for "+ NiFiProperties.ANALYTICS_PREDICTION_INTERVAL + ". This property has been set to '"
+ NiFiProperties.DEFAULT_ANALYTICS_PREDICTION_INTERVAL + "'");
predictionIntervalMillis = FormatUtils.getTimeDuration(NiFiProperties.DEFAULT_ANALYTICS_PREDICTION_INTERVAL, TimeUnit.MILLISECONDS);
}
// Determine score name to use for evaluating model performance
String modelScoreName = nifiProperties.getProperty(NiFiProperties.ANALYTICS_CONNECTION_MODEL_SCORE_NAME, NiFiProperties.DEFAULT_ANALYTICS_CONNECTION_SCORE_NAME);
// Determine score threshold to use when evaluating acceptable model
Double modelScoreThreshold;
try {
modelScoreThreshold = Double.valueOf(nifiProperties.getProperty(NiFiProperties.ANALYTICS_CONNECTION_MODEL_SCORE_THRESHOLD,
Double.toString(NiFiProperties.DEFAULT_ANALYTICS_CONNECTION_SCORE_THRESHOLD)));
} catch (final Exception e) {
LOG.warn("Analytics is enabled however could not retrieve value for "+ NiFiProperties.ANALYTICS_CONNECTION_MODEL_SCORE_THRESHOLD + ". This property has been set to '"
+ NiFiProperties.DEFAULT_ANALYTICS_CONNECTION_SCORE_THRESHOLD + "'.");
modelScoreThreshold = NiFiProperties.DEFAULT_ANALYTICS_CONNECTION_SCORE_THRESHOLD;
}
final Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = StatusAnalyticsModelMapFactory
.getConnectionStatusModelMap(extensionManager, nifiProperties);
analyticsEngine = new CachingConnectionStatusAnalyticsEngine(flowManager, componentStatusRepository, flowFileEventRepository, modelMap,
predictionIntervalMillis, modelScoreName, modelScoreThreshold);
}
eventAccess = new StandardEventAccess(this, flowFileEventRepository);
timerDrivenEngineRef.get().scheduleWithFixedDelay(new Runnable() {
@Override
public void run() {
@ -1026,8 +1073,6 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
}
public KerberosConfig createKerberosConfig(final NiFiProperties nifiProperties) {
final String principal = nifiProperties.getKerberosServicePrincipal();
final String keytabLocation = nifiProperties.getKerberosServiceKeytabLocation();
@ -1372,6 +1417,10 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
return eventAccess;
}
public StatusAnalyticsEngine getStatusAnalyticsEngine() {
return analyticsEngine;
}
/**
* Sets the root group to the given group
*

View File

@ -0,0 +1,73 @@
/*
* 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.Map;
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.apache.nifi.util.Tuple;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.github.benmanes.caffeine.cache.Cache;
import com.github.benmanes.caffeine.cache.Caffeine;
/**
* <p>
* An implementation of {@link StatusAnalyticsEngine} that supports caching of ConnectionStatusAnalytics objects.
* Caching engine in use is an in-memory cache where the caching policy is to expire 30 minutes after initial write to cache.
* </p>
*/
public class CachingConnectionStatusAnalyticsEngine extends ConnectionStatusAnalyticsEngine{
private volatile Cache<String, StatusAnalytics> cache;
private static final Logger LOG = LoggerFactory.getLogger(CachingConnectionStatusAnalyticsEngine.class);
public CachingConnectionStatusAnalyticsEngine(FlowManager flowManager, ComponentStatusRepository statusRepository,
FlowFileEventRepository flowFileEventRepository, Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap,
long predictionIntervalMillis, String scoreName, double scoreThreshold) {
super(flowManager,statusRepository,flowFileEventRepository,modelMap,predictionIntervalMillis,scoreName,scoreThreshold);
this.cache = Caffeine.newBuilder()
.expireAfterWrite(30, TimeUnit.MINUTES)
.build();
}
/**
* Return a connection status analytics instance
* @param identifier connection identifier
* @return StatusAnalytics analytics object for given connection with id
*/
@Override
public StatusAnalytics getStatusAnalytics(String identifier) {
StatusAnalytics connectionStatusAnalytics = cache.getIfPresent(identifier);
if (connectionStatusAnalytics == null) {
LOG.debug("Creating new status analytics object for connection id: {}", identifier);
connectionStatusAnalytics = super.getStatusAnalytics(identifier);
cache.put(identifier, connectionStatusAnalytics);
} else {
LOG.debug("Pulled existing analytics from cache for connection id: {}", identifier);
((ConnectionStatusAnalytics)connectionStatusAnalytics).refresh();
}
return connectionStatusAnalytics;
}
}

View File

@ -0,0 +1,390 @@
/*
* 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.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.history.ComponentStatusRepository;
import org.apache.nifi.controller.status.history.StatusHistory;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.processor.DataUnit;
import org.apache.nifi.util.Tuple;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.primitives.Doubles;
/**
* <p>
* An implementation of {@link StatusAnalytics} that is provides Connection related analysis/prediction for a given connection instance
* </p>
*/
public class ConnectionStatusAnalytics implements StatusAnalytics {
private static final Logger LOG = LoggerFactory.getLogger(ConnectionStatusAnalytics.class);
private final Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> 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 long intervalMillis = 3L * 60 * 1000; // Default is 3 minutes
private String scoreName = "rSquared";
private double scoreThreshold = .90;
public ConnectionStatusAnalytics(ComponentStatusRepository componentStatusRepository, FlowManager flowManager, FlowFileEventRepository flowFileEventRepository,
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap, String connectionIdentifier, Boolean supportOnlineLearning) {
this.componentStatusRepository = componentStatusRepository;
this.flowManager = flowManager;
this.flowFileEventRepository = flowFileEventRepository;
this.modelMap = modelMap;
this.connectionIdentifier = connectionIdentifier;
this.supportOnlineLearning = supportOnlineLearning;
}
/**
* Retrieve observations and train available model(s)
*/
public void refresh() {
if (supportOnlineLearning && this.queryWindow != null) {
//Obtain latest observations when available, extend window if needed to obtain minimum observations
this.queryWindow = new QueryWindow(extendWindow ? queryWindow.getStartTimeMillis() : queryWindow.getEndTimeMillis(), System.currentTimeMillis());
} else {
this.queryWindow = new QueryWindow(System.currentTimeMillis() - getIntervalTimeMillis(), System.currentTimeMillis());
}
modelMap.forEach((metric, modelFunction) -> {
StatusAnalyticsModel model = modelFunction.getKey();
StatusMetricExtractFunction 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);
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 StatusAnalyticsModel getModel(String modelType){
if(modelMap.containsKey(modelType)){
return modelMap.get(modelType).getKey();
}else{
throw new IllegalArgumentException("Model cannot be found for provided type: " + modelType);
}
}
/**
* 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 StatusAnalyticsModel bytesModel = getModel("queuedBytes");
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);
if (validModel(bytesModel) && flowFileEvent != null) {
Map<Integer, Double> predictFeatures = new HashMap<>();
Double inOutRatio = (flowFileEvent.getContentSizeOut() / (double) flowFileEvent.getContentSizeIn());
predictFeatures.put(1, inOutRatio);
return convertTimePrediction(bytesModel.predictVariable(0, predictFeatures, backPressureBytes), System.currentTimeMillis());
} else {
return -1L;
}
}
/**
* 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 StatusAnalyticsModel countModel = getModel("queuedCount");
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();
if (validModel(countModel) && flowFileEvent != null) {
Map<Integer, Double> predictFeatures = new HashMap<>();
Double inOutRatio = (flowFileEvent.getFlowFilesOut() / (double) flowFileEvent.getFlowFilesIn());
predictFeatures.put(1, inOutRatio);
return convertTimePrediction(countModel.predictVariable(0, predictFeatures, backPressureCountThreshold), System.currentTimeMillis());
} else {
return -1L;
}
}
/**
* 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 StatusAnalyticsModel bytesModel = getModel("queuedBytes");
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 convertCountPrediction(bytesModel.predict(predictFeatures.toArray(new Double[2])));
} else {
return -1L;
}
}
/**
* 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 Long getNextIntervalCount() {
final StatusAnalyticsModel countModel = getModel("queuedCount");
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 convertCountPrediction(countModel.predict(predictFeatures.toArray(new Double[2])));
} else {
return -1L;
}
}
/**
* Returns the predicted object count percentage in queue when compared to threshold (maximum value allowed) at the next configured interval
*
* @return percentage of bytes used at next interval
*/
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();
if (nextIntervalCount > -1L) {
return Math.min(100, Math.round((nextIntervalCount / backPressureCountThreshold) * 100));
} else {
return -1L;
}
}
/**
* Returns the predicted bytes percentage in queue when compared to threshold (maximum value allowed) at the next configured interval
*
* @return percentage of bytes used at next interval
*/
public Long 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);
final long nextIntervalBytes = getNextIntervalBytes();
if (nextIntervalBytes > -1L) {
return Math.min(100, Math.round((getNextIntervalBytes() / backPressureBytes) * 100));
} else {
return -1L;
}
}
public Long getIntervalTimeMillis() {
return intervalMillis;
}
public void setIntervalTimeMillis(long intervalTimeMillis) {
this.intervalMillis = intervalTimeMillis;
}
public String getScoreName() {
return scoreName;
}
public void setScoreName(String scoreName) {
this.scoreName = scoreName;
}
public double getScoreThreshold() {
return scoreThreshold;
}
public void setScoreThreshold(double scoreThreshold) {
this.scoreThreshold = scoreThreshold;
}
@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", getNextIntervalCount());
predictions.put("nextIntervalPercentageUseCount", getNextIntervalPercentageUseCount());
predictions.put("nextIntervalPercentageUseBytes", getNextIntervalPercentageUseBytes());
predictions.put("intervalTimeMillis", getIntervalTimeMillis());
predictions.forEach((key, value) -> {
LOG.debug("Prediction model for connection id {}: {}={} ", connectionIdentifier, key, value);
});
return predictions;
}
@Override
public boolean supportsOnlineLearning() {
return supportOnlineLearning;
}
private Connection getConnection() {
final ProcessGroup rootGroup = flowManager.getRootGroup();
Optional<Connection> connection = rootGroup.findAllConnections().stream().filter(c -> c.getIdentifier().equals(this.connectionIdentifier)).findFirst();
return connection.orElse(null);
}
private FlowFileEvent getStatusReport() {
RepositoryStatusReport statusReport = flowFileEventRepository.reportTransferEvents(System.currentTimeMillis());
return statusReport.getReportEntry(this.connectionIdentifier);
}
/**
* Convert time into valid prediction value (-1 indicates no prediction available).
* Valid time translates to prediction equal to current time or in the future
* @param prediction prediction value
* @param timeMillis time in milliseconds
* @return valid model boolean
*/
private Long convertTimePrediction(Double prediction, Long timeMillis) {
if (Double.isNaN(prediction) || Double.isInfinite(prediction) || prediction < timeMillis) {
return -1L;
} else {
return Math.max(0, Math.round(prediction) - timeMillis);
}
}
/**
* Convert count into valid prediction value (-1 indicates no prediction available)
* @param prediction prediction value
* @return prediction prediction value converted into valid value for consumption
*/
private Long convertCountPrediction(Double prediction) {
if (Double.isNaN(prediction) || Double.isInfinite(prediction) || prediction < 0) {
return -1L;
} else {
return Math.max(0, Math.round(prediction));
}
}
/**
* Check if model is valid for prediction based on score
* @param model StatusAnalyticsModel object
* @return valid model boolean
*/
private boolean validModel(StatusAnalyticsModel model) {
Double score = getScore(model);
if (score == null || (Doubles.isFinite(score) && !Double.isNaN(score) && score < scoreThreshold)) {
if (supportOnlineLearning && model.supportsOnlineLearning()) {
model.clear();
}
return false;
} else {
return true;
}
}
/**
* Get specific score from trained model
* @param model StatusAnalyticsModel object
* @return scoreValue
*/
private Double getScore(StatusAnalyticsModel model) {
if (model != null && model.getScores() != null) {
return model.getScores().get(scoreName);
} else {
return null;
}
}
}

View File

@ -0,0 +1,69 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.status.analytics;
import java.util.Map;
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.util.Tuple;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* <p>
* An implementation of {@link StatusAnalyticsEngine} that supports creation of ConnectionStatusAnalytics objects
* </p>
*/
public class ConnectionStatusAnalyticsEngine implements StatusAnalyticsEngine {
private static final Logger LOG = LoggerFactory.getLogger(ConnectionStatusAnalyticsEngine.class);
protected final ComponentStatusRepository statusRepository;
protected final FlowManager flowManager;
protected final FlowFileEventRepository flowFileEventRepository;
protected final Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap;
protected final long predictionIntervalMillis;
protected final String scoreName;
protected final double scoreThreshold;
public ConnectionStatusAnalyticsEngine(FlowManager flowManager, ComponentStatusRepository statusRepository, FlowFileEventRepository flowFileEventRepository,
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap, long predictionIntervalMillis, String scoreName, double scoreThreshold) {
this.flowManager = flowManager;
this.statusRepository = statusRepository;
this.flowFileEventRepository = flowFileEventRepository;
this.predictionIntervalMillis = predictionIntervalMillis;
this.modelMap = modelMap;
this.scoreName = scoreName;
this.scoreThreshold = scoreThreshold;
}
/**
* Return a connection status analytics instance
* @param identifier connection identifier
* @return status analytics object
*/
@Override
public StatusAnalytics getStatusAnalytics(String identifier) {
ConnectionStatusAnalytics connectionStatusAnalytics = new ConnectionStatusAnalytics(statusRepository, flowManager, flowFileEventRepository, modelMap, identifier, false);
connectionStatusAnalytics.setIntervalTimeMillis(predictionIntervalMillis);
connectionStatusAnalytics.setScoreName(scoreName);
connectionStatusAnalytics.setScoreThreshold(scoreThreshold);
connectionStatusAnalytics.refresh();
return connectionStatusAnalytics;
}
}

View File

@ -0,0 +1,126 @@
/*
* 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.Random;
import org.apache.nifi.controller.status.history.StatusHistoryUtil;
import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.NarThreadContextClassLoader;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.util.Tuple;
import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
import org.apache.nifi.web.api.dto.status.StatusSnapshotDTO;
/**
* <p>
* This factory supports the creation of models and their associated extraction functions
* </p>
*/
public class StatusAnalyticsModelMapFactory {
private final static String QUEUED_COUNT_METRIC = "queuedCount";
private final static String QUEUED_BYTES_METRIC = "queuedBytes";
private final static String INPUT_COUNT_METRIC = "inputCount";
private final static String INPUT_BYTES_METRIC = "inputBytes";
private final static String OUTPUT_COUNT_METRIC = "outputCount";
private final static String OUTPUT_BYTES_METRIC = "outputBytes";
/**
* Return mapping of models and extraction functions for connection status analytics prediction instances
* @param extensionManager Extension Manager object for instantiating classes
* @param niFiProperties NiFi Properties object
* @return
*/
public static Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> getConnectionStatusModelMap(ExtensionManager extensionManager, NiFiProperties niFiProperties){
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = new HashMap<>();
StatusMetricExtractFunction extract = getConnectionStatusExtractFunction();
Tuple<StatusAnalyticsModel, StatusMetricExtractFunction> countModelFunction = new Tuple<>(createModelInstance(extensionManager, niFiProperties), extract);
Tuple<StatusAnalyticsModel, StatusMetricExtractFunction> byteModelFunction = new Tuple<>(createModelInstance(extensionManager, niFiProperties), extract);
modelMap.put(QUEUED_COUNT_METRIC, countModelFunction);
modelMap.put(QUEUED_BYTES_METRIC, byteModelFunction);
return modelMap;
}
/**
* Create a connection model instance using configurations set in NiFi properties
* @param extensionManager Extension Manager object for instantiating classes
* @param nifiProperties NiFi Properties object
* @return statusAnalyticsModel
*/
private static StatusAnalyticsModel createModelInstance(ExtensionManager extensionManager, NiFiProperties nifiProperties) {
final String implementationClassName = nifiProperties.getProperty(NiFiProperties.ANALYTICS_CONNECTION_MODEL_IMPLEMENTATION, NiFiProperties.DEFAULT_ANALYTICS_CONNECTION_MODEL_IMPLEMENTATION);
if (implementationClassName == null) {
throw new RuntimeException("Cannot create Analytics Model because the NiFi Properties is missing the following property: "
+ NiFiProperties.ANALYTICS_CONNECTION_MODEL_IMPLEMENTATION);
}
try {
return NarThreadContextClassLoader.createInstance(extensionManager, implementationClassName, StatusAnalyticsModel.class, nifiProperties);
} catch (final Exception e) {
throw new RuntimeException(e);
}
}
/**
* Get a connection status extract function instance
* @return StatusMetricExtractFunction
*/
private static StatusMetricExtractFunction getConnectionStatusExtractFunction() {
return (metric, statusHistory) -> {
List<Double> values = 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(QUEUED_COUNT_METRIC)) {
Long inputCount = snap.getStatusMetrics().get(INPUT_COUNT_METRIC);
Long outputCount = snap.getStatusMetrics().get(OUTPUT_COUNT_METRIC);
Double inOutRatio = ((double) outputCount / (double) inputCount) + randomError;
featureArray.add(Double.isNaN(inOutRatio) ? randomError : inOutRatio);
} else {
Long inputBytes = snap.getStatusMetrics().get(INPUT_BYTES_METRIC);
Long outputBytes = snap.getStatusMetrics().get(OUTPUT_BYTES_METRIC);
Double inOutRatio = ((double) outputBytes / (double) inputBytes) + randomError;
featureArray.add(Double.isNaN(inOutRatio) ? randomError : inOutRatio);
}
values.add((double) snapValue);
features.add(featureArray.toArray(new Double[featureArray.size()]));
}
return new Tuple<>(features.stream(), values.stream());
};
}
}

View File

@ -0,0 +1,33 @@
/*
* 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.nifi.controller.status.history.StatusHistory;
import org.apache.nifi.util.Tuple;
public interface StatusMetricExtractFunction {
/**
* Searches through status history to extract observations for a given metric
* @param metric metric value that should be extracted
* @param statusHistory StatusHistory object used to search metrics
* @return a Tuple with extracted observations (features and targets/labels)
*/
Tuple<Stream<Double[]>, Stream<Double>> extractMetric(String metric, StatusHistory statusHistory);
}

View File

@ -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.models;
import java.util.Map;
import java.util.stream.Stream;
import org.apache.nifi.controller.status.analytics.StatusAnalyticsModel;
/**
* <p>
* An abstract class for implementations of {@link StatusAnalyticsModel} which makes bivariate models compatible with analytics interface
* </p>
*/
public abstract class BivariateStatusAnalyticsModel implements StatusAnalyticsModel {
@Override
public abstract void learn(Stream<Double[]> features, Stream<Double> labels);
@Override
public Double predict(Double[] feature){
return predictY(feature[0]);
}
@Override
public Double predictVariable(Integer predictVariableIndex, Map<Integer, Double> knownVariablesWithIndex, Double label) {
return predictY(label);
}
@Override
public Boolean supportsOnlineLearning() {
return false;
}
public abstract Double predictX(Double y);
public abstract Double predictY(Double x);
@Override
public abstract Map<String,Double> getScores();
@Override
public void clear() {
}
}

View File

@ -0,0 +1,108 @@
/*
* 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.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.controller.status.analytics.StatusAnalyticsModel;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* <p>
* An implementation of the {@link StatusAnalyticsModel} that uses Ordinary Least Squares computation for regression.
* This model support multiple regression
* </p>
*/
public class OrdinaryLeastSquares implements StatusAnalyticsModel {
private static final Logger LOG = LoggerFactory.getLogger(OrdinaryLeastSquares.class);
private OLSMultipleLinearRegression olsModel;
private double[] coefficients;
public OrdinaryLeastSquares() {
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) {
return null;
} else {
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;
}
}
@Override
public Double predictVariable(Integer predictVariableIndex, Map<Integer, Double> knownVariablesWithIndex, Double label) {
if (coefficients == null) {
return null;
} else {
final double intercept = olsModel.isNoIntercept() ? 0 : coefficients[0];
final double predictorCoeff = coefficients[predictVariableIndex + 1];
double sumX = 0;
if (knownVariablesWithIndex.size() > 0) {
sumX = knownVariablesWithIndex.entrySet().stream().map(featureTuple -> coefficients[olsModel.isNoIntercept()
? featureTuple.getKey() : featureTuple.getKey() + 1] * featureTuple.getValue())
.collect(Collectors.summingDouble(Double::doubleValue));
}
return (label - intercept - sumX) / predictorCoeff;
}
}
@Override
public Map<String, Double> getScores() {
if (coefficients == null) {
return null;
} else {
Map<String, Double> scores = new HashMap<>();
scores.put("rSquared", olsModel.calculateRSquared());
scores.put("totalSumOfSquares", olsModel.calculateTotalSumOfSquares());
return scores;
}
}
@Override
public Boolean supportsOnlineLearning() {
return false;
}
@Override
public void clear() {
}
}

View File

@ -0,0 +1,104 @@
/*
* 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.Map;
import java.util.stream.Stream;
import org.apache.commons.lang3.ArrayUtils;
import org.apache.commons.math3.stat.regression.RegressionResults;
import org.apache.nifi.controller.status.analytics.StatusAnalyticsModel;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* <p>
* An implementation of the {@link StatusAnalyticsModel} that uses the SimpleRegression class from math for computation of regression. It only supports Bivariates
* for x,y (multiple predictors are not supported by SimpleRegression). Online learning is supported for collecting multiple samples at different points in time
*
* </p>
*/
public class SimpleRegression extends BivariateStatusAnalyticsModel {
private static final Logger LOG = LoggerFactory.getLogger(SimpleRegression.class);
private final org.apache.commons.math3.stat.regression.SimpleRegression regression;
private final Boolean supportOnlineLearning;
private RegressionResults results;
public SimpleRegression() {
this(true);
}
public SimpleRegression(Boolean supportOnlineLearning) {
this.regression = new org.apache.commons.math3.stat.regression.SimpleRegression();
this.supportOnlineLearning = supportOnlineLearning;
}
@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);
if (!supportOnlineLearning) {
regression.clear();
}
regression.addObservations(featuresMatrix, labelArray);
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 predictX(Double y) {
return (y - regression.getIntercept()) / regression.getSlope();
}
@Override
public Double predictY(Double x) {
return regression.getSlope() * x + regression.getIntercept();
}
@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 supportOnlineLearning;
}
}

View File

@ -16,6 +16,14 @@
*/
package org.apache.nifi.reporting;
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;
import org.apache.commons.collections4.Predicate;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.action.Action;
@ -44,6 +52,9 @@ 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.ConnectionStatusPredictions;
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;
@ -54,20 +65,15 @@ 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;
private final StatusAnalyticsEngine statusAnalyticsEngine;
public StandardEventAccess(final FlowController flowController, final FlowFileEventRepository flowFileEventRepository) {
this.flowController = flowController;
this.flowFileEventRepository = flowFileEventRepository;
this.statusAnalyticsEngine = flowController.getStatusAnalyticsEngine();
}
/**
@ -338,6 +344,24 @@ public class StandardEventAccess implements UserAwareEventAccess {
bytesTransferred += connectionStatusReport.getContentSizeIn() + connectionStatusReport.getContentSizeOut();
}
if (statusAnalyticsEngine != null) {
StatusAnalytics statusAnalytics = statusAnalyticsEngine.getStatusAnalytics(conn.getIdentifier());
if (statusAnalytics != null) {
Map<String,Long> predictionValues = statusAnalytics.getPredictions();
ConnectionStatusPredictions predictions = new ConnectionStatusPredictions();
connStatus.setPredictions(predictions);
predictions.setPredictedTimeToBytesBackpressureMillis(predictionValues.get("timeToBytesBackpressureMillis"));
predictions.setPredictedTimeToCountBackpressureMillis(predictionValues.get("timeToCountBackpressureMillis"));
predictions.setNextPredictedQueuedBytes(predictionValues.get("nextIntervalBytes"));
predictions.setNextPredictedQueuedCount(predictionValues.get("nextIntervalCount").intValue());
predictions.setPredictedPercentCount(predictionValues.get("nextIntervalPercentageUseCount").intValue());
predictions.setPredictedPercentBytes(predictionValues.get("nextIntervalPercentageUseBytes").intValue());
predictions.setPredictionIntervalMillis(predictionValues.get("intervalTimeMillis"));
}
}else{
connStatus.setPredictions(null);
}
if (isConnectionAuthorized) {
if (StringUtils.isNotBlank(conn.getName())) {
connStatus.setName(conn.getName());

View File

@ -0,0 +1,16 @@
# 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.
org.apache.nifi.controller.status.analytics.models.OrdinaryLeastSquares
org.apache.nifi.controller.status.analytics.models.SimpleRegression

View File

@ -0,0 +1,52 @@
/*
* 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.assertEquals;
import static org.junit.Assert.assertNotEquals;
import java.util.Map;
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.util.Tuple;
import org.junit.Test;
public class TestCachingConnectionStatusAnalyticsEngine extends TestStatusAnalyticsEngine {
@Override
public StatusAnalyticsEngine getStatusAnalyticsEngine(FlowManager flowManager, FlowFileEventRepository flowFileEventRepository,
ComponentStatusRepository componentStatusRepository,
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap,
long predictIntervalMillis, String scoreName, double scoreThreshold) {
return new CachingConnectionStatusAnalyticsEngine(flowManager, componentStatusRepository, flowFileEventRepository, modelMap, predictIntervalMillis, scoreName, scoreThreshold);
}
@Test
public void testCachedStatusAnalytics() {
StatusAnalyticsEngine statusAnalyticsEngine = new CachingConnectionStatusAnalyticsEngine(flowManager, statusRepository, flowFileEventRepository, modelMap,
DEFAULT_PREDICT_INTERVAL_MILLIS, DEFAULT_SCORE_NAME, DEFAULT_SCORE_THRESHOLD);
StatusAnalytics statusAnalyticsA = statusAnalyticsEngine.getStatusAnalytics("A");
StatusAnalytics statusAnalyticsB = statusAnalyticsEngine.getStatusAnalytics("B");
StatusAnalytics statusAnalyticsTest = statusAnalyticsEngine.getStatusAnalytics("A");
assertEquals(statusAnalyticsA, statusAnalyticsTest);
assertNotEquals(statusAnalyticsB, statusAnalyticsTest);
}
}

View File

@ -0,0 +1,433 @@
/*
* 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.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
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;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.commons.lang3.time.DateUtils;
import org.apache.nifi.bundle.Bundle;
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;
import org.apache.nifi.controller.status.history.StatusHistory;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
import org.apache.nifi.nar.SystemBundle;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.util.Tuple;
import org.junit.Test;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
public class TestConnectionStatusAnalytics {
private static final Set<MetricDescriptor<?>> CONNECTION_METRICS = Arrays.stream(ConnectionStatusDescriptor.values())
.map(ConnectionStatusDescriptor::getDescriptor)
.collect(Collectors.toSet());
protected ConnectionStatusAnalytics getConnectionStatusAnalytics(Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap) {
ComponentStatusRepository statusRepository = Mockito.mock(ComponentStatusRepository.class);
FlowManager flowManager;
flowManager = Mockito.mock(FlowManager.class);
final Map<String, String> otherProps = new HashMap<>();
final String propsFile = "src/test/resources/conf/nifi.properties";
NiFiProperties nifiProperties = NiFiProperties.createBasicNiFiProperties(propsFile, otherProps);
// use the system bundle
Bundle systemBundle = SystemBundle.create(nifiProperties);
StandardExtensionDiscoveringManager extensionManager = new StandardExtensionDiscoveringManager();
extensionManager.discoverExtensions(systemBundle, Collections.emptySet());
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 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);
when(flowFileQueue.getBackPressureDataSizeThreshold()).thenReturn("100MB");
when(flowFileQueue.getBackPressureObjectThreshold()).thenReturn(100L);
when(connection.getIdentifier()).thenReturn(connectionIdentifier);
when(connection.getFlowFileQueue()).thenReturn(flowFileQueue);
when(processGroup.findAllConnections()).thenReturn(connections);
when(flowManager.getRootGroup()).thenReturn(processGroup);
when(flowFileEvent.getContentSizeIn()).thenReturn(10L);
when(flowFileEvent.getContentSizeOut()).thenReturn(10L);
when(flowFileEvent.getFlowFilesIn()).thenReturn(10);
when(flowFileEvent.getFlowFilesOut()).thenReturn(10);
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,flowFileEventRepository,
modelMap, connectionIdentifier, false);
connectionStatusAnalytics.refresh();
return connectionStatusAnalytics;
}
public Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> getModelMap( String predictionType, Double score,
Double targetPrediction, Double variablePrediction){
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = new HashMap<>();
StatusAnalyticsModel model = Mockito.mock(StatusAnalyticsModel.class);
StatusMetricExtractFunction extractFunction = Mockito.mock(StatusMetricExtractFunction.class);
Tuple<StatusAnalyticsModel,StatusMetricExtractFunction> modelTuple = new Tuple<>(model,extractFunction);
modelMap.put(predictionType,modelTuple);
Map<String,Double> scores = new HashMap<>();
scores.put("rSquared",score);
Double[][] features = new Double[1][1];
Double[] target = new Double[1];
when(extractFunction.extractMetric(anyString(),any(StatusHistory.class))).then(new Answer<Tuple<Stream<Double[]>,Stream<Double>>>() {
@Override
public Tuple<Stream<Double[]>, Stream<Double>> answer(InvocationOnMock invocationOnMock) throws Throwable {
return new Tuple<>(Stream.of(features), Stream.of(target));
}
});
when(model.getScores()).thenReturn(scores);
when(model.predict(any(Double[].class))).thenReturn(targetPrediction);
when(model.predictVariable(anyInt(),any(),any())).thenReturn(variablePrediction);
return modelMap;
}
@Test
public void testInvalidModelLowScore() {
Date now = new Date();
Long tomorrowMillis = DateUtils.addDays(now,1).toInstant().toEpochMilli();
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("queuedCount",.5,100.0,tomorrowMillis.doubleValue());
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
Long countTime = connectionStatusAnalytics.getTimeToCountBackpressureMillis();
assertNotNull(countTime);
assert (countTime == -1);
}
@Test
public void testInvalidModelNaNScore() {
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("queuedCount",Double.NaN,Double.NaN,Double.NaN);
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
Long countTime = connectionStatusAnalytics.getTimeToCountBackpressureMillis();
assertNotNull(countTime);
assert (countTime == -1);
}
@Test
public void testInvalidModelInfiniteScore() {
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("queuedCount",Double.POSITIVE_INFINITY,Double.POSITIVE_INFINITY,Double.POSITIVE_INFINITY);
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
Long countTime = connectionStatusAnalytics.getTimeToCountBackpressureMillis();
assertNotNull(countTime);
assert (countTime == -1);
}
@Test
public void testGetIntervalTimeMillis() {
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("queuedCount",.9,100.0,100.0);
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
Long interval = connectionStatusAnalytics.getIntervalTimeMillis();
assertNotNull(interval);
assert (interval == 180000);
}
@Test
public void testGetTimeToCountBackpressureMillis() {
Date now = new Date();
Long tomorrowMillis = DateUtils.addDays(now,1).toInstant().toEpochMilli();
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("queuedCount",.9,100.0,tomorrowMillis.doubleValue());
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
Long countTime = connectionStatusAnalytics.getTimeToCountBackpressureMillis();
assertNotNull(countTime);
assert (countTime > 0);
}
@Test
public void testCannotPredictTimeToCountNaN() {
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("queuedCount",.9,Double.NaN,Double.NaN);
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
Long countTime = connectionStatusAnalytics.getTimeToCountBackpressureMillis();
assertNotNull(countTime);
assert (countTime == -1);
}
@Test
public void testCannotPredictTimeToCountInfinite() {
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("queuedCount",.9,Double.POSITIVE_INFINITY,Double.POSITIVE_INFINITY);
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
Long countTime = connectionStatusAnalytics.getTimeToCountBackpressureMillis();
assertNotNull(countTime);
assert (countTime == -1);
}
@Test
public void testCannotPredictTimeToCountNegative() {
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("queuedCount",.9,-1.0,-1.0);
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
Long countTime = connectionStatusAnalytics.getTimeToCountBackpressureMillis();
assertNotNull(countTime);
assert (countTime == -1);
}
@Test
public void testMissingModelGetTimeToCount() {
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("fakeModel",Double.POSITIVE_INFINITY,Double.POSITIVE_INFINITY,Double.POSITIVE_INFINITY);
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
try {
connectionStatusAnalytics.getTimeToCountBackpressureMillis();
fail();
}catch(IllegalArgumentException iae){
assertTrue(true);
}
}
@Test
public void testGetTimeToBytesBackpressureMillis() {
Date now = new Date();
Long tomorrowMillis = DateUtils.addDays(now,1).toInstant().toEpochMilli();
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("queuedBytes",.9,100.0,tomorrowMillis.doubleValue());
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
Long countTime = connectionStatusAnalytics.getTimeToBytesBackpressureMillis();
assertNotNull(countTime);
assert (countTime > 0);
}
@Test
public void testCannotPredictTimeToBytesNaN() {
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("queuedBytes",.9,Double.NaN,Double.NaN);
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
Long countTime = connectionStatusAnalytics.getTimeToBytesBackpressureMillis();
assertNotNull(countTime);
assert (countTime == -1);
}
@Test
public void testCannotPredictTimeToBytesInfinite() {
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("queuedBytes",.9,Double.POSITIVE_INFINITY,Double.POSITIVE_INFINITY);
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
Long countTime = connectionStatusAnalytics.getTimeToBytesBackpressureMillis();
assertNotNull(countTime);
assert (countTime == -1);
}
@Test
public void testCannotPredictTimeToBytesNegative() {
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("queuedBytes",.9,-1.0,-1.0);
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
Long countTime = connectionStatusAnalytics.getTimeToBytesBackpressureMillis();
assertNotNull(countTime);
assert (countTime == -1);
}
@Test
public void testMissingModelGetTimeToBytes() {
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("fakeModel",Double.POSITIVE_INFINITY,Double.POSITIVE_INFINITY,Double.POSITIVE_INFINITY);
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
try {
connectionStatusAnalytics.getTimeToBytesBackpressureMillis();
fail();
}catch(IllegalArgumentException iae){
assertTrue(true);
}
}
@Test
public void testGetNextIntervalBytes() {
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("queuedBytes",.9,1.0,1.0);
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
Long nextIntervalBytes = connectionStatusAnalytics.getNextIntervalBytes();
assertNotNull(nextIntervalBytes);
assert (nextIntervalBytes > 0);
}
@Test
public void testCannotPredictNextIntervalBytesNegative() {
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("queuedBytes",.9,-1.0,-1.0);
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
Long nextIntervalBytes = connectionStatusAnalytics.getNextIntervalBytes();
assertNotNull(nextIntervalBytes);
assert (nextIntervalBytes == -1);
}
@Test
public void testCannotPredictNextIntervalBytesNaN() {
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("queuedBytes",.9,Double.NaN,Double.NaN);
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
Long nextIntervalBytes = connectionStatusAnalytics.getNextIntervalBytes();
assertNotNull(nextIntervalBytes);
assert (nextIntervalBytes == -1);
}
@Test
public void testCannotPredictNextIntervalBytesInfinity() {
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("queuedBytes",.9,Double.POSITIVE_INFINITY,Double.POSITIVE_INFINITY);
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
Long nextIntervalBytes = connectionStatusAnalytics.getNextIntervalBytes();
assertNotNull(nextIntervalBytes);
assert (nextIntervalBytes == -1);
}
@Test
public void testMissingModelNextIntervalBytes() {
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("fakeModel",Double.POSITIVE_INFINITY,Double.POSITIVE_INFINITY,Double.POSITIVE_INFINITY);
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
try {
connectionStatusAnalytics.getNextIntervalBytes();
fail();
}catch(IllegalArgumentException iae){
assertTrue(true);
}
}
@Test
public void testGetNextIntervalCount() {
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("queuedCount",.9,1.0,1.0);
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
Long nextIntervalBytes = connectionStatusAnalytics.getNextIntervalCount();
assertNotNull(nextIntervalBytes);
assert (nextIntervalBytes > 0);
}
@Test
public void testCannotPredictNextIntervalCountNegative() {
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("queuedCount",.9,-1.0,-1.0);
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
Long nextIntervalBytes = connectionStatusAnalytics.getNextIntervalCount();
assertNotNull(nextIntervalBytes);
assert (nextIntervalBytes == -1);
}
@Test
public void testCannotPredictNextIntervalCountNaN() {
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("queuedCount",.9,Double.NaN,Double.NaN);
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
Long nextIntervalBytes = connectionStatusAnalytics.getNextIntervalCount();
assertNotNull(nextIntervalBytes);
assert (nextIntervalBytes == -1);
}
@Test
public void testCannotPredictNextIntervalCountInfinity() {
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("queuedCount",.9,Double.POSITIVE_INFINITY,Double.POSITIVE_INFINITY);
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
Long nextIntervalBytes = connectionStatusAnalytics.getNextIntervalCount();
assertNotNull(nextIntervalBytes);
assert (nextIntervalBytes == -1);
}
@Test
public void testMissingModelGetNextIntervalCount() {
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("fakeModel",Double.POSITIVE_INFINITY,Double.POSITIVE_INFINITY,Double.POSITIVE_INFINITY);
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
try {
connectionStatusAnalytics.getNextIntervalCount();
fail();
}catch(IllegalArgumentException iae){
assertTrue(true);
}
}
@Test
public void testGetNextIntervalPercentageUseCount() {
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("queuedCount",.9,50.0,1.0);
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
Long percentage = connectionStatusAnalytics.getNextIntervalPercentageUseCount();
assertNotNull(percentage);
assert (percentage == 50);
}
@Test
public void testGetNextIntervalPercentageUseBytes() {
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = getModelMap("queuedBytes",.9,10000000.0,1.0);
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap);
Long percentage = connectionStatusAnalytics.getNextIntervalPercentageUseBytes();
assertNotNull(percentage);
assert (percentage == 10);
}
@Test
public void testGetScores() {
Date now = new Date();
Long tomorrowMillis = DateUtils.addDays(now,1).toInstant().toEpochMilli();
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> bytesModelMap = getModelMap("queuedBytes",.9,10000000.0,tomorrowMillis.doubleValue());
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> countModelMap = getModelMap("queuedCount",.9,50.0,tomorrowMillis.doubleValue());
countModelMap.putAll(bytesModelMap);
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(countModelMap);
Map<String,Long> scores = connectionStatusAnalytics.getPredictions();
assertNotNull(scores);
assertFalse(scores.isEmpty());
assertTrue(scores.get("nextIntervalPercentageUseCount").equals(50L));
assertTrue(scores.get("nextIntervalBytes").equals(10000000L));
assertTrue(scores.get("timeToBytesBackpressureMillis") > 0);
assertTrue(scores.get("nextIntervalCount").equals(50L));
assertTrue(scores.get("nextIntervalPercentageUseBytes").equals(10L));
assertTrue(scores.get("intervalTimeMillis").equals(180000L));
assertTrue(scores.get("timeToCountBackpressureMillis") > 0);
}
@Test
public void testGetScoresWithBadModel() {
Date now = new Date();
Long tomorrowMillis = DateUtils.addDays(now,1).toInstant().toEpochMilli();
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> bytesModelMap = getModelMap("queuedBytes",.9,10000000.0,tomorrowMillis.doubleValue());
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> countModelMap = getModelMap("queuedCount",.1,50.0,tomorrowMillis.doubleValue());
countModelMap.putAll(bytesModelMap);
ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(countModelMap);
Map<String,Long> scores = connectionStatusAnalytics.getPredictions();
assertNotNull(scores);
assertFalse(scores.isEmpty());
assertTrue(scores.get("nextIntervalPercentageUseCount").equals(-1L));
assertTrue(scores.get("nextIntervalBytes").equals(10000000L));
assertTrue(scores.get("timeToBytesBackpressureMillis") > 0);
assertTrue(scores.get("nextIntervalCount").equals(-1L));
assertTrue(scores.get("nextIntervalPercentageUseBytes").equals(10L));
assertTrue(scores.get("intervalTimeMillis").equals(180000L));
assertTrue(scores.get("timeToCountBackpressureMillis") == -1);
}
}

View File

@ -0,0 +1,35 @@
/*
* 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.Map;
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.util.Tuple;
public class TestConnectionStatusAnalyticsEngine extends TestStatusAnalyticsEngine {
@Override
public StatusAnalyticsEngine getStatusAnalyticsEngine(FlowManager flowManager, FlowFileEventRepository flowFileEventRepository,
ComponentStatusRepository statusRepository, Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap,
long predictIntervalMillis, String scoreName, double scoreThreshold) {
return new ConnectionStatusAnalyticsEngine(flowManager, statusRepository, flowFileEventRepository,modelMap, DEFAULT_PREDICT_INTERVAL_MILLIS, scoreName, scoreThreshold);
}
}

View File

@ -0,0 +1,101 @@
/*
* 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 java.util.HashMap;
import java.util.Map;
import java.util.stream.Stream;
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;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.util.Tuple;
import org.junit.Before;
import org.junit.Test;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
public abstract class TestStatusAnalyticsEngine {
static final long DEFAULT_PREDICT_INTERVAL_MILLIS = 3L * 60 * 1000;
static final String DEFAULT_SCORE_NAME = "rSquared";
static final double DEFAULT_SCORE_THRESHOLD = .9;
protected ComponentStatusRepository statusRepository;
protected FlowManager flowManager;
protected FlowFileEventRepository flowFileEventRepository;
protected Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap;
@Before
public void setup() {
statusRepository = Mockito.mock(ComponentStatusRepository.class);
flowManager = Mockito.mock(FlowManager.class);
modelMap = new HashMap<>();
StatusAnalyticsModel countModel = Mockito.mock(StatusAnalyticsModel.class);
StatusAnalyticsModel byteModel = Mockito.mock(StatusAnalyticsModel.class);
StatusMetricExtractFunction extractFunction = Mockito.mock(StatusMetricExtractFunction.class);
Tuple<StatusAnalyticsModel,StatusMetricExtractFunction> countTuple = new Tuple<>(countModel,extractFunction);
Tuple<StatusAnalyticsModel,StatusMetricExtractFunction> byteTuple = new Tuple<>(byteModel,extractFunction);
modelMap.put("queuedCount",countTuple);
modelMap.put("queuedBytes",byteTuple);
Double[][] features = new Double[1][1];
Double[] target = new Double[1];
ProcessGroup processGroup = Mockito.mock(ProcessGroup.class);
StatusHistory statusHistory = Mockito.mock(StatusHistory.class);
StatusSnapshot statusSnapshot = Mockito.mock(StatusSnapshot.class);
when(extractFunction.extractMetric(anyString(),any(StatusHistory.class))).then(new Answer<Tuple<Stream<Double[]>,Stream<Double>>>() {
@Override
public Tuple<Stream<Double[]>, Stream<Double>> answer(InvocationOnMock invocationOnMock) throws Throwable {
return new Tuple<>(Stream.of(features), Stream.of(target));
}
});
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,flowFileEventRepository, statusRepository, modelMap, DEFAULT_PREDICT_INTERVAL_MILLIS,
DEFAULT_SCORE_NAME, DEFAULT_SCORE_THRESHOLD);
StatusAnalytics statusAnalytics = statusAnalyticsEngine.getStatusAnalytics("1");
assertNotNull(statusAnalytics);
}
public abstract StatusAnalyticsEngine getStatusAnalyticsEngine(FlowManager flowManager, FlowFileEventRepository flowFileEventRepository,
ComponentStatusRepository componentStatusRepository, Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap,
long predictIntervalMillis, String scoreName, double scoreThreshold);
}

View File

@ -0,0 +1,66 @@
/*
* 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.assertEquals;
import static org.testng.Assert.assertNotNull;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import org.apache.nifi.bundle.Bundle;
import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
import org.apache.nifi.nar.SystemBundle;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.util.Tuple;
import org.junit.Before;
import org.junit.Test;
public class TestStatusAnalyticsModelMapFactory {
protected NiFiProperties nifiProperties;
protected ExtensionManager extensionManager;
@Before
public void setup() {
final Map<String, String> otherProps = new HashMap<>();
final String propsFile = "src/test/resources/conf/nifi.properties";
nifiProperties = NiFiProperties.createBasicNiFiProperties(propsFile, otherProps);
// use the system bundle
Bundle systemBundle = SystemBundle.create(nifiProperties);
extensionManager = new StandardExtensionDiscoveringManager();
((StandardExtensionDiscoveringManager) extensionManager).discoverExtensions(systemBundle, Collections.emptySet());
}
@Test
public void getConnectionStatusModelMap() {
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap = StatusAnalyticsModelMapFactory
.getConnectionStatusModelMap(extensionManager, nifiProperties);
assertNotNull(modelMap.get("queuedCount"));
assertNotNull(modelMap.get("queuedBytes"));
StatusAnalyticsModel countModel = modelMap.get("queuedCount").getKey();
StatusAnalyticsModel bytesModel = modelMap.get("queuedBytes").getKey();
assertNotNull(countModel);
assertNotNull(bytesModel);
assertEquals(countModel.getClass().getName(),"org.apache.nifi.controller.status.analytics.models.OrdinaryLeastSquares");
assertEquals(bytesModel.getClass().getName(),"org.apache.nifi.controller.status.analytics.models.OrdinaryLeastSquares");
}
}

View File

@ -0,0 +1,156 @@
/*
* 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.Date;
import java.util.HashMap;
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.junit.Test;
public class TestOrdinaryLeastSqaures {
@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};
OrdinaryLeastSquares model = new OrdinaryLeastSquares();
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};
OrdinaryLeastSquares model = new OrdinaryLeastSquares();
model.learn(Stream.of(features), Stream.of(labels));
Map<Integer,Double> predictorVars = new HashMap<>();
predictorVars.put(1,200/800.0);
Double target = model.predictVariable(0,predictorVars, 750.0);
Double rSquared = model.getScores().get("rSquared");
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};
OrdinaryLeastSquares model = new OrdinaryLeastSquares();
Double[] predictor = {timestamp + 5000, outputCount/inputCount};
model.learn(Stream.of(features), Stream.of(labels));
Double target = model.predict(predictor);
Double rSquared = model.getScores().get("rSquared");
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};
OrdinaryLeastSquares ordinaryLeastSquares = new OrdinaryLeastSquares();
SimpleRegression simpleRegression = new SimpleRegression(false);
ordinaryLeastSquares.learn(Stream.of(features), Stream.of(labels));
simpleRegression.learn(Stream.of(features), Stream.of(labels));
double olsR2 = ordinaryLeastSquares.getScores().get("rSquared");
double srR2 = simpleRegression.getScores().get("rSquared");
assert(!Double.isNaN(olsR2));
assert(!Double.isNaN(srR2));
Map<String,Double> olsScores = ordinaryLeastSquares.getScores();
Map<String,Double> srScores = simpleRegression.getScores();
System.out.print(olsScores.toString());
System.out.print(srScores.toString());
assert(olsR2 > srR2);
}
}

View File

@ -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 TestSimpleRegression {
@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};
SimpleRegression model = new SimpleRegression(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};
SimpleRegression model = new SimpleRegression(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};
SimpleRegression model = new SimpleRegression(false);
Double[] predictor = {timestamp + 5000};
model.learn(Stream.of(features), Stream.of(labels));
Double target = model.predict(predictor);
Double rSquared = model.getScores().get("rSquared");
Double minCount = -1265.0;
Double maxCount = 3235.0;
assert(rSquared > .60);
assert(target >= minCount && target <= maxCount);
}
}

View File

@ -119,3 +119,9 @@ 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
nifi.analytics.connection.model.implementation=org.apache.nifi.controller.status.analytics.models.OrdinaryLeastSquares
nifi.analytics.connection.model.score.name=rSquared
nifi.analytics.connection.model.score.threshold=.9

View File

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

View File

@ -16,32 +16,6 @@
*/
package org.apache.nifi.nar;
import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
import org.apache.nifi.authentication.LoginIdentityProvider;
import org.apache.nifi.authorization.AccessPolicyProvider;
import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.UserGroupProvider;
import org.apache.nifi.bundle.Bundle;
import org.apache.nifi.bundle.BundleCoordinate;
import org.apache.nifi.components.ConfigurableComponent;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.state.StateProvider;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.repository.ContentRepository;
import org.apache.nifi.controller.repository.FlowFileRepository;
import org.apache.nifi.controller.repository.FlowFileSwapManager;
import org.apache.nifi.controller.status.history.ComponentStatusRepository;
import org.apache.nifi.flowfile.FlowFilePrioritizer;
import org.apache.nifi.init.ConfigurableComponentInitializer;
import org.apache.nifi.init.ConfigurableComponentInitializerFactory;
import org.apache.nifi.processor.Processor;
import org.apache.nifi.provenance.ProvenanceRepository;
import org.apache.nifi.reporting.InitializationException;
import org.apache.nifi.reporting.ReportingTask;
import org.apache.nifi.util.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.net.URL;
import java.net.URLClassLoader;
@ -58,6 +32,33 @@ import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.stream.Collectors;
import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
import org.apache.nifi.authentication.LoginIdentityProvider;
import org.apache.nifi.authorization.AccessPolicyProvider;
import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.UserGroupProvider;
import org.apache.nifi.bundle.Bundle;
import org.apache.nifi.bundle.BundleCoordinate;
import org.apache.nifi.components.ConfigurableComponent;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.state.StateProvider;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.repository.ContentRepository;
import org.apache.nifi.controller.repository.FlowFileRepository;
import org.apache.nifi.controller.repository.FlowFileSwapManager;
import org.apache.nifi.controller.status.analytics.StatusAnalyticsModel;
import org.apache.nifi.controller.status.history.ComponentStatusRepository;
import org.apache.nifi.flowfile.FlowFilePrioritizer;
import org.apache.nifi.init.ConfigurableComponentInitializer;
import org.apache.nifi.init.ConfigurableComponentInitializerFactory;
import org.apache.nifi.processor.Processor;
import org.apache.nifi.provenance.ProvenanceRepository;
import org.apache.nifi.reporting.InitializationException;
import org.apache.nifi.reporting.ReportingTask;
import org.apache.nifi.util.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Scans through the classpath to load all FlowFileProcessors, FlowFileComparators, and ReportingTasks using the service provider API and running through all classloaders (root, NARs).
*
@ -95,6 +96,7 @@ public class StandardExtensionDiscoveringManager implements ExtensionDiscovering
definitionMap.put(FlowFileSwapManager.class, new HashSet<>());
definitionMap.put(ContentRepository.class, new HashSet<>());
definitionMap.put(StateProvider.class, new HashSet<>());
definitionMap.put(StatusAnalyticsModel.class, new HashSet<>());
}
@Override

View File

@ -208,6 +208,13 @@
<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.enabled>false</nifi.analytics.predict.enabled>
<nifi.analytics.predict.interval>3 mins</nifi.analytics.predict.interval>
<nifi.analytics.connection.model.implementation>org.apache.nifi.controller.status.analytics.models.OrdinaryLeastSquares</nifi.analytics.connection.model.implementation>
<nifi.analytics.connection.model.score.name>rSquared</nifi.analytics.connection.model.score.name>
<nifi.analytics.connection.model.score.threshold>.90</nifi.analytics.connection.model.score.threshold>
</properties>
<build>
<plugins>

View File

@ -256,3 +256,10 @@ 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=
# analytics properties #
nifi.analytics.predict.enabled=${nifi.analytics.predict.enabled}
nifi.analytics.predict.interval=${nifi.analytics.predict.interval}
nifi.analytics.connection.model.implementation=${nifi.analytics.connection.model.implementation}
nifi.analytics.connection.model.score.name=${nifi.analytics.connection.model.score.name}
nifi.analytics.connection.model.score.threshold=${nifi.analytics.connection.model.score.threshold}

View File

@ -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;
@ -661,6 +662,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.
*

View File

@ -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;
@ -217,6 +245,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;
@ -236,6 +265,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;
@ -308,33 +338,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.
@ -3234,6 +3238,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(connection, controllerFacade.getConnectionStatusAnalytics(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);

View File

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

View File

@ -106,6 +106,8 @@ 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.ConnectionStatusPredictions;
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;
@ -200,7 +202,10 @@ 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.ConnectionStatusPredictionsSnapshotDTO;
import org.apache.nifi.web.api.dto.status.ConnectionStatusSnapshotDTO;
import org.apache.nifi.web.api.dto.status.PortStatusDTO;
import org.apache.nifi.web.api.dto.status.PortStatusSnapshotDTO;
@ -1177,11 +1182,37 @@ public final class DtoFactory {
snapshot.setFlowFilesOut(connectionStatus.getOutputCount());
snapshot.setBytesOut(connectionStatus.getOutputBytes());
ConnectionStatusPredictions predictions = connectionStatus.getPredictions();
ConnectionStatusPredictionsSnapshotDTO predictionsDTO = null;
if (predictions != null) {
predictionsDTO = new ConnectionStatusPredictionsSnapshotDTO();
}
if (connectionStatus.getBackPressureObjectThreshold() > 0) {
snapshot.setPercentUseCount(Math.min(100, StatusMerger.getUtilization(connectionStatus.getQueuedCount(), connectionStatus.getBackPressureObjectThreshold())));
if (predictionsDTO != null) {
snapshot.setPredictions(predictionsDTO);
predictionsDTO.setPredictionIntervalSeconds(((Long) (predictions.getPredictionIntervalMillis() / 1000L)).intValue());
predictionsDTO.setPredictedMillisUntilCountBackpressure(predictions.getPredictedTimeToCountBackpressureMillis());
predictionsDTO.setPredictedCountAtNextInterval(predictions.getNextPredictedQueuedCount());
predictionsDTO.setPredictedPercentCount(predictions.getPredictedPercentCount());
predictionsDTO.setPredictedPercentBytes(predictions.getPredictedPercentBytes());
predictionsDTO.setPredictionIntervalSeconds(((Long) (predictions.getPredictionIntervalMillis() / 1000L)).intValue());
}
}
if (connectionStatus.getBackPressureBytesThreshold() > 0) {
snapshot.setPercentUseBytes(Math.min(100, StatusMerger.getUtilization(connectionStatus.getQueuedBytes(), connectionStatus.getBackPressureBytesThreshold())));
if (predictionsDTO != null) {
snapshot.setPredictions(predictionsDTO);
predictionsDTO.setPredictionIntervalSeconds(((Long) (predictions.getPredictionIntervalMillis() / 1000L)).intValue());
predictionsDTO.setPredictedMillisUntilBytesBackpressure(predictions.getPredictedTimeToBytesBackpressureMillis());
predictionsDTO.setPredictedBytesAtNextInterval(predictions.getNextPredictedQueuedBytes());
predictionsDTO.setPredictedPercentCount(predictions.getPredictedPercentCount());
predictionsDTO.setPredictedPercentBytes(predictions.getPredictedPercentBytes());
predictionsDTO.setPredictionIntervalSeconds(((Long) (predictions.getPredictionIntervalMillis() / 1000L)).intValue());
}
}
StatusMerger.updatePrettyPrintedFields(snapshot);
@ -1189,6 +1220,29 @@ public final class DtoFactory {
return connectionStatusDto;
}
public ConnectionStatisticsDTO createConnectionStatisticsDto(final Connection connection, final StatusAnalytics statusAnalytics) {
final ConnectionStatisticsDTO connectionStatisticsDTO = new ConnectionStatisticsDTO();
connectionStatisticsDTO.setId(connection.getIdentifier());
connectionStatisticsDTO.setStatsLastRefreshed(new Date());
final ConnectionStatisticsSnapshotDTO snapshot = new ConnectionStatisticsSnapshotDTO();
connectionStatisticsDTO.setAggregateSnapshot(snapshot);
snapshot.setId(connection.getIdentifier());
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;
}
public ProcessorStatusDTO createProcessorStatusDto(final ProcessorStatus procStatus) {
final ProcessorStatusDTO dto = new ProcessorStatusDTO();
dto.setId(procStatus.getId());

View File

@ -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;
@ -138,6 +142,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());

View File

@ -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,6 +80,8 @@ 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.StatusAnalyticsEngine;
import org.apache.nifi.controller.status.history.ComponentStatusRepository;
import org.apache.nifi.diagnostics.SystemDiagnostics;
import org.apache.nifi.flowfile.FlowFilePrioritizer;
@ -110,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);
@ -684,6 +687,37 @@ public class ControllerFacade implements Authorizable {
return status;
}
/**
* Gets status analytics for the specified connection.
*
* @param connectionId connection id
* @return the statistics for the specified connection
*/
public StatusAnalytics getConnectionStatusAnalytics(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));
}
// get from flow controller
final StatusAnalyticsEngine statusAnalyticsEngine = flowController.getStatusAnalyticsEngine();
if (statusAnalyticsEngine == null) {
throw new ResourceNotFoundException(String.format("Unable to provide analytics for connection with id '%s'. Analytics may not be enabled", connectionId));
}
return statusAnalyticsEngine.getStatusAnalytics(connectionId);
}
/**
* Gets the status for the specified input port.
*

View File

@ -239,6 +239,10 @@
<include>lodash-core/distrib/lodash-core.min.js</include>
<include>lodash-core/distrib/README.md</include>
<include>moment/min/moment.min.js</include>
<include>moment/README.md</include>
<include>moment/LICENSE</include>
</includes>
</resource>
</resources>

View File

@ -88,9 +88,9 @@
}
},
"commander": {
"version": "2.16.0",
"resolved": "https://registry.npmjs.org/commander/-/commander-2.16.0.tgz",
"integrity": "sha512-sVXqklSaotK9at437sFlFpyOcJonxe0yST/AG9DkQKUdIE6IqGIMv4SfAQSKaJbSdVEJYItASCrBiVQHq1HQew=="
"version": "2.20.0",
"resolved": "https://registry.npmjs.org/commander/-/commander-2.20.0.tgz",
"integrity": "sha512-7j2y+40w61zy6YC2iRNpUe/NwhNyoXrYpHMrSunaMG64nRnaf96zO/KMQR4OyN/UnE5KLyEBnKHd4aG3rskjpQ=="
},
"d3": {
"version": "4.13.0",
@ -189,8 +189,8 @@
"resolved": "https://registry.npmjs.org/d3-dsv/-/d3-dsv-1.0.8.tgz",
"integrity": "sha512-IVCJpQ+YGe3qu6odkPQI0KPqfxkhbP/oM1XhhE/DFiYmcXKfCRub4KXyiuehV1d4drjWVXHUWx4gHqhdZb6n/A==",
"requires": {
"commander": "2.16.0",
"iconv-lite": "0.4.23",
"commander": "2.20.0",
"iconv-lite": "0.4.24",
"rw": "1.3.3"
}
},
@ -372,9 +372,9 @@
"integrity": "sha1-ZxRKUmDDT8PMpnfQQdr1L+e3iy8="
},
"iconv-lite": {
"version": "0.4.23",
"resolved": "https://registry.npmjs.org/iconv-lite/-/iconv-lite-0.4.23.tgz",
"integrity": "sha512-neyTUVFtahjf0mB3dZT77u+8O0QB89jFdnBkd5P1JgYPbPaia3gXXOVL2fq8VyU2gMMD7SaN7QukTB/pmXYvDA==",
"version": "0.4.24",
"resolved": "https://registry.npmjs.org/iconv-lite/-/iconv-lite-0.4.24.tgz",
"integrity": "sha512-v3MXnZAcvnywkTUEZomIActle7RXXeedOR31wwl7VlyoXO4Qi9arvSenNQWne1TcRwhCL1HwLI21bEqdpj8/rA==",
"requires": {
"safer-buffer": "2.1.2"
}
@ -432,6 +432,11 @@
"resolved": "https://registry.npmjs.org/lodash-core/-/lodash-core-4.17.11.tgz",
"integrity": "sha512-8ilprNE67U1REh0wQHL0z37qXdsxuFXjvxehg79Mh/MQgNJ+C1muXtysSKpt9sCxXZUSiwifEC82Vtzf2GSSKQ=="
},
"moment": {
"version": "2.24.0",
"resolved": "https://registry.npmjs.org/moment/-/moment-2.24.0.tgz",
"integrity": "sha512-bV7f+6l2QigeBBZSM/6yTNq4P2fNpSWj/0e7jQcy87A8e7o2nAfP/34/2ky5Vw4B9S446EtIhodAzkFCcR4dQg=="
},
"nomnom": {
"version": "1.8.1",
"resolved": "https://registry.npmjs.org/nomnom/-/nomnom-1.8.1.tgz",

View File

@ -35,6 +35,7 @@
"reset.css": "2.0.2",
"jquery-form": "3.50.0",
"lodash-core": "4.17.11",
"moment": "2.24.0",
"url-search-params": "0.6.1",
"jsonlint": "1.6.2",
"qtip2": "3.0.3",

View File

@ -771,3 +771,28 @@ This product bundles 'lodash' which is available under an MIT license.
maintained libraries used by this software which have their own
licenses; we recommend you read them, as their terms may differ from the
terms above.
This product bundles 'moment' which is available under an MIT license.
Copyright (c) JS Foundation and other contributors
Permission is hereby granted, free of charge, to any person
obtaining a copy of this software and associated documentation
files (the "Software"), to deal in the Software without
restriction, including without limitation the rights to use,
copy, modify, merge, publish, distribute, sublicense, and/or sell
copies of the Software, and to permit persons to whom the
Software is furnished to do so, subject to the following
conditions:
The above copyright notice and this permission notice shall be
included in all copies or substantial portions of the Software.
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES
OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT
HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY,
WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
OTHER DEALINGS IN THE SOFTWARE.

View File

@ -39,6 +39,7 @@
<script type="text/javascript" src="js/jquery/jquery.each.js"></script>
<script type="text/javascript" src="assets/jquery-ui-dist/jquery-ui.min.js"></script>
<script type="text/javascript" src="assets/lodash-core/distrib/lodash-core.min.js"></script>
<script type="text/javascript" src="assets/moment/min/moment.min.js"></script>
<script type="text/javascript" src="assets/qtip2/dist/jquery.qtip.min.js"></script>
<script type="text/javascript" src="assets/angular/angular.min.js"></script>
<script type="text/javascript" src="assets/angular-messages/angular-messages.min.js"></script>

View File

@ -62,6 +62,7 @@
<script type="text/javascript" src="js/jquery/modal/jquery.modal.js?${project.version}"></script>
<script type="text/javascript" src="assets/jquery-minicolors/jquery.minicolors.min.js"></script>
<script type="text/javascript" src="assets/lodash-core/distrib/lodash-core.min.js"></script>
<script type="text/javascript" src="assets/moment/min/moment.min.js"></script>
<script type="text/javascript" src="assets/qtip2/dist/jquery.qtip.min.js"></script>
<script type="text/javascript" src="assets/slickgrid/lib/jquery.event.drag-2.3.0.js"></script>
<script type="text/javascript" src="assets/slickgrid/plugins/slick.cellrangeselector.js"></script>
@ -132,6 +133,7 @@
<div id="port-tooltips"></div>
<div id="process-group-tooltips"></div>
<div id="remote-process-group-tooltips"></div>
<div id="connection-tooltips"></div>
</div>
<jsp:include page="/WEB-INF/partials/canvas/navigation.jsp"/>
<jsp:include page="/WEB-INF/partials/canvas/settings-content.jsp"/>

View File

@ -43,6 +43,7 @@
<script type="text/javascript" src="js/jquery/jquery.each.js"></script>
<script type="text/javascript" src="assets/jquery-ui-dist/jquery-ui.min.js"></script>
<script type="text/javascript" src="assets/lodash-core/distrib/lodash-core.min.js"></script>
<script type="text/javascript" src="assets/moment/min/moment.min.js"></script>
<script type="text/javascript" src="assets/qtip2/dist/jquery.qtip.min.js"></script>
<script type="text/javascript" src="assets/slickgrid/lib/jquery.event.drag-2.3.0.js"></script>
<script type="text/javascript" src="assets/slickgrid/plugins/slick.cellrangeselector.js"></script>

View File

@ -41,6 +41,7 @@
<script type="text/javascript" src="js/jquery/jquery.each.js"></script>
<script type="text/javascript" src="assets/jquery-ui-dist/jquery-ui.min.js"></script>
<script type="text/javascript" src="assets/lodash-core/distrib/lodash-core.min.js"></script>
<script type="text/javascript" src="assets/moment/min/moment.min.js"></script>
<script type="text/javascript" src="assets/qtip2/dist/jquery.qtip.min.js"></script>
<script type="text/javascript" src="assets/slickgrid/lib/jquery.event.drag-2.3.0.js"></script>
<script type="text/javascript" src="assets/slickgrid/plugins/slick.cellrangeselector.js"></script>

View File

@ -41,6 +41,7 @@
<script type="text/javascript" src="js/jquery/jquery.each.js"></script>
<script type="text/javascript" src="assets/jquery-ui-dist/jquery-ui.min.js"></script>
<script type="text/javascript" src="assets/lodash-core/distrib/lodash-core.min.js"></script>
<script type="text/javascript" src="assets/moment/min/moment.min.js"></script>
<script type="text/javascript" src="assets/qtip2/dist/jquery.qtip.min.js"></script>
<script type="text/javascript" src="assets/slickgrid/lib/jquery.event.drag-2.3.0.js"></script>
<script type="text/javascript" src="assets/slickgrid/plugins/slick.cellrangeselector.js"></script>

View File

@ -40,6 +40,7 @@
<script type="text/javascript" src="assets/jquery-ui-dist/jquery-ui.min.js"></script>
<script type="text/javascript" src="js/nf/nf-namespace.js?${project.version}"></script>
<script type="text/javascript" src="assets/lodash-core/distrib/lodash-core.min.js"></script>
<script type="text/javascript" src="assets/moment/min/moment.min.js"></script>
${nf.login.script.tags}
</head>
<body class="login-body">

View File

@ -45,6 +45,7 @@
<script type="text/javascript" src="js/jquery/jquery.each.js"></script>
<script type="text/javascript" src="assets/jquery-ui-dist/jquery-ui.min.js"></script>
<script type="text/javascript" src="assets/lodash-core/distrib/lodash-core.min.js"></script>
<script type="text/javascript" src="assets/moment/min/moment.min.js"></script>
<script type="text/javascript" src="assets/qtip2/dist/jquery.qtip.min.js"></script>
<script type="text/javascript" src="assets/slickgrid/lib/jquery.event.drag-2.3.0.js"></script>
<script type="text/javascript" src="assets/slickgrid/plugins/slick.cellrangeselector.js"></script>

View File

@ -51,6 +51,7 @@
<script type="text/javascript" src="js/jquery/jquery.ellipsis.js"></script>
<script type="text/javascript" src="js/jquery/jquery.each.js"></script>
<script type="text/javascript" src="assets/lodash-core/distrib/lodash-core.min.js"></script>
<script type="text/javascript" src="assets/moment/min/moment.min.js"></script>
<script type="text/javascript" src="assets/qtip2/dist/jquery.qtip.min.js"></script>
<script type="text/javascript" src="assets/slickgrid/lib/jquery.event.drag-2.3.0.js"></script>
<script type="text/javascript" src="assets/slickgrid/plugins/slick.cellrangeselector.js"></script>

View File

@ -41,6 +41,7 @@
<script type="text/javascript" src="js/jquery/jquery.each.js"></script>
<script type="text/javascript" src="assets/jquery-ui-dist/jquery-ui.min.js"></script>
<script type="text/javascript" src="assets/lodash-core/distrib/lodash-core.min.js"></script>
<script type="text/javascript" src="assets/moment/min/moment.min.js"></script>
<script type="text/javascript" src="assets/qtip2/dist/jquery.qtip.min.js"></script>
<script type="text/javascript" src="assets/slickgrid/lib/jquery.event.drag-2.3.0.js"></script>
<script type="text/javascript" src="assets/slickgrid/plugins/slick.cellrangeselector.js"></script>

View File

@ -43,6 +43,7 @@
<script type="text/javascript" src="js/jquery/jquery.ellipsis.js"></script>
<script type="text/javascript" src="js/jquery/jquery.each.js"></script>
<script type="text/javascript" src="assets/lodash-core/distrib/lodash-core.min.js"></script>
<script type="text/javascript" src="assets/moment/min/moment.min.js"></script>
<script type="text/javascript" src="assets/qtip2/dist/jquery.qtip.min.js"></script>
<script type="text/javascript" src="assets/slickgrid/lib/jquery.event.drag-2.3.0.js"></script>
<script type="text/javascript" src="assets/slickgrid/plugins/slick.cellrangeselector.js"></script>

View File

@ -344,7 +344,24 @@ text.connection-from-run-status.is-missing-port, text.connection-to-run-status.i
}
g.connection rect.backpressure-tick {
fill: #3e3e3e;
fill: transparent;
}
g.connection rect.backpressure-tick.data-size-prediction.prediction-down,
g.connection rect.backpressure-tick.object-prediction.prediction-down {
fill: white;
}
g.connection rect.backpressure-tick.data-size-prediction,
g.connection rect.backpressure-tick.object-prediction {
fill: black;
}
g.connection rect.backpressure-tick.data-size-prediction.not-configured,
g.connection rect.backpressure-tick.object-prediction.not-configured,
g.connection rect.backpressure-tick.data-size-prediction.prediction-down.not-configured,
g.connection rect.backpressure-tick.object-prediction.prediction-down.not-configured {
fill: transparent;
}
g.connection rect.backpressure-tick.not-configured {
@ -355,6 +372,12 @@ g.connection rect.backpressure-object, g.connection rect.backpressure-data-size
fill: #d8d8d8;
}
/**/
g.connection rect.backpressure-object>title>tspan, g.connection rect.backpressure-data-size>title>tspan {
display: block;
}
/**/
g.connection rect.backpressure-object.not-configured, g.connection rect.backpressure-data-size.not-configured {
fill: transparent;
}

View File

@ -26,9 +26,10 @@
'nf.Storage',
'nf.ErrorHandler',
'nf.Client',
'nf.CanvasUtils'],
function ($, d3, nfCommon, nfDialog, nfStorage, nfErrorHandler, nfClient, nfCanvasUtils) {
return (nf.Connection = factory($, d3, nfCommon, nfDialog, nfStorage, nfErrorHandler, nfClient, nfCanvasUtils));
'nf.CanvasUtils',
'lodash-core'],
function ($, d3, nfCommon, nfDialog, nfStorage, nfErrorHandler, nfClient, nfCanvasUtils, _) {
return (nf.Connection = factory($, d3, nfCommon, nfDialog, nfStorage, nfErrorHandler, nfClient, nfCanvasUtils, _));
});
} else if (typeof exports === 'object' && typeof module === 'object') {
module.exports = (nf.Connection =
@ -39,7 +40,8 @@
require('nf.Storage'),
require('nf.ErrorHandler'),
require('nf.Client'),
require('nf.CanvasUtils')));
require('nf.CanvasUtils'),
require('lodash-code')));
} else {
nf.Connection = factory(root.$,
root.d3,
@ -48,9 +50,10 @@
root.nf.Storage,
root.nf.ErrorHandler,
root.nf.Client,
root.nf.CanvasUtils);
root.nf.CanvasUtils,
root._);
}
}(this, function ($, d3, nfCommon, nfDialog, nfStorage, nfErrorHandler, nfClient, nfCanvasUtils) {
}(this, function ($, d3, nfCommon, nfDialog, nfStorage, nfErrorHandler, nfClient, nfCanvasUtils, _) {
'use strict';
var nfSelectable;
@ -66,6 +69,9 @@
// width of a backpressure indicator - half of width, left/right padding, left/right border
var backpressureBarWidth = (dimensions.width / 2) - 15 - 2;
var backpressureCountOffset = 6;
var backpressureDataSizeOffset = (dimensions.width / 2) + 10 + 1;
// --------------------------
// Snap alignment for drag events
// --------------------------
@ -1238,92 +1244,121 @@
var yBackpressureOffset = rowHeight + HEIGHT_FOR_BACKPRESSURE - 4;
// backpressure object threshold
var backpressureObjectContainer = queued.append('g')
.attrs({
'transform': 'translate(' + backpressureCountOffset + ', ' + yBackpressureOffset + ')',
'class': 'backpressure-object-container'
});
// start
queued.append('rect')
backpressureObjectContainer.append('rect')
.attrs({
'class': 'backpressure-tick object',
'width': 1,
'height': 3,
'x': 5,
'y': yBackpressureOffset
'x': 0,
'y': 0
});
// bar
var backpressureCountOffset = 6;
queued.append('rect')
backpressureObjectContainer.append('rect')
.attrs({
'class': 'backpressure-object',
'width': backpressureBarWidth,
'height': 3,
'x': backpressureCountOffset,
'y': yBackpressureOffset
})
.append('title');
'x': 0,
'y': 0
});
// end
queued.append('rect')
backpressureObjectContainer.append('rect')
.attrs({
'class': 'backpressure-tick object',
'width': 1,
'height': 3,
'x': backpressureCountOffset + backpressureBarWidth,
'y': yBackpressureOffset
'x': backpressureBarWidth,
'y': 0
});
// percent full
queued.append('rect')
backpressureObjectContainer.append('rect')
.attrs({
'class': 'backpressure-percent object',
'width': 0,
'height': 3,
'x': backpressureCountOffset,
'y': yBackpressureOffset
'x': 0,
'y': 0
});
// prediction indicator
backpressureObjectContainer.append('rect')
.attrs({
'class': 'backpressure-tick object-prediction',
'width': 1,
'height': 3,
'x': backpressureBarWidth,
'y': 0
});
// backpressure data size threshold
var backpressureDataSizeContainer = queued.append('g')
.attrs({
'transform': 'translate(' + backpressureDataSizeOffset + ', ' + yBackpressureOffset + ')',
'class': 'backpressure-data-size-container'
});
// start
queued.append('rect')
backpressureDataSizeContainer.append('rect')
.attrs({
'class': 'backpressure-tick data-size',
'width': 1,
'height': 3,
'x': (dimensions.width / 2) + 10,
'y': yBackpressureOffset
'x': 0,
'y': 0
});
// bar
var backpressureDataSizeOffset = (dimensions.width / 2) + 10 + 1;
queued.append('rect')
backpressureDataSizeContainer.append('rect')
.attrs({
'class': 'backpressure-data-size',
'width': backpressureBarWidth,
'height': 3,
'x': backpressureDataSizeOffset,
'y': yBackpressureOffset
'x': 0,
'y': 0
})
.append('title');
// end
queued.append('rect')
backpressureDataSizeContainer.append('rect')
.attrs({
'class': 'backpressure-tick data-size',
'width': 1,
'height': 3,
'x': backpressureDataSizeOffset + backpressureBarWidth,
'y': yBackpressureOffset
'x': backpressureBarWidth,
'y': 0
});
// percent full
queued.append('rect')
backpressureDataSizeContainer.append('rect')
.attrs({
'class': 'backpressure-percent data-size',
'width': 0,
'height': 3,
'x': backpressureDataSizeOffset,
'y': yBackpressureOffset
'x': 0,
'y': 0
});
// prediction indicator
backpressureDataSizeContainer.append('rect')
.attrs({
'class': 'backpressure-tick data-size-prediction',
'width': 1,
'height': 3,
'x': backpressureBarWidth,
'y': 0
});
} else {
backgrounds.push(queued.select('rect.connection-label-background'));
borders.push(queued.select('rect.connection-label-border'));
@ -1441,6 +1476,10 @@
.classed('not-configured', function () {
return nfCommon.isUndefinedOrNull(d.status.aggregateSnapshot.percentUseCount);
});
connectionLabelContainer.selectAll('rect.backpressure-tick.object-prediction')
.classed('not-configured', function () {
return nfCommon.isUndefinedOrNull(d.status.aggregateSnapshot.percentUseCount);
});
// update backpressure data size fill
connectionLabelContainer.select('rect.backpressure-data-size')
@ -1451,6 +1490,10 @@
.classed('not-configured', function () {
return nfCommon.isUndefinedOrNull(d.status.aggregateSnapshot.percentUseBytes);
});
connectionLabelContainer.selectAll('rect.backpressure-tick.data-size-prediction')
.classed('not-configured', function () {
return nfCommon.isUndefinedOrNull(d.status.aggregateSnapshot.percentUseBytes);
});
if (d.permissions.canWrite) {
// only support dragging the label when appropriate
@ -1476,6 +1519,114 @@
});
};
var isAtBackPressure = function (d) {
var percentUseCount = _.get(d, 'status.aggregateSnapshot.percentUseCount', 0);
var percentUseBytes = _.get(d, 'status.aggregateSnapshot.percentUseBytes', 0);
return Math.max(percentUseCount, percentUseBytes) >= 100;
};
/**
* Gets the tooltip content for the back pressure count metric
* @param d
*/
var getBackPressureCountTip = function (d) {
var tooltipContent;
var percentUseCount = _.get(d, 'status.aggregateSnapshot.percentUseCount');
if (_.isNumber(percentUseCount)) {
var objectThreshold = _.get(d, 'component.backPressureObjectThreshold');
var predictions = _.get(d, 'status.aggregateSnapshot.predictions');
var tooltipLines = ['Queue: ' + _.clamp(percentUseCount, 0, 100) + '% full (based on ' + objectThreshold + ' object threshold)'];
if (!_.isNil(predictions)) {
var predictedPercentCount = _.get(predictions, 'predictedPercentCount', -1);
var timeToBackPressure = _.get(predictions, 'predictedMillisUntilCountBackpressure', -1);
// only show predicted percent if it is non-negative
var predictionIntervalSeconds = _.get(predictions, 'predictionIntervalSeconds', 60 * 5);
if (_.isNumber(predictedPercentCount) && predictedPercentCount > -1) {
tooltipLines.push('Predicted queue (next ' + (predictionIntervalSeconds / 60 ) + ' mins): ' + _.clamp(predictedPercentCount, 0, 100) + '%')
} else {
tooltipLines.push('Predicted queue (next ' + (predictionIntervalSeconds / 60 ) + ' mins): NA' )
}
// only show an estimate if it is valid (non-negative but less than the max number supported)
if (_.isNumber(timeToBackPressure) && _.inRange(timeToBackPressure, 0, Number.MAX_SAFE_INTEGER) && !isAtBackPressure(d)) {
var duration = nfCommon.formatPredictedDuration(timeToBackPressure);
tooltipLines.push('Estimated time to back pressure: ' + duration);
} else {
tooltipLines.push('Estimated time to back pressure: ' + (isAtBackPressure(d) ? 'now' : 'NA'));
}
} else {
tooltipLines.push('Queue Prediction is not configured')
}
if (_.isEmpty(tooltipLines)) {
return '';
} else if (_.size(tooltipLines) === 1) {
return tooltipLines[0];
} else {
tooltipContent = nfCommon.formatUnorderedList(tooltipLines)
}
} else {
tooltipContent = 'Back Pressure Object Threshold is not configured';
}
return tooltipContent;
};
/**
* Gets the tooltip content for the back pressure size metric
* @param d
*/
var getBackPressureSizeTip = function (d) {
var tooltipContent;
var percentUseBytes = _.get(d, 'status.aggregateSnapshot.percentUseBytes');
if (_.isNumber(percentUseBytes)) {
var dataSizeThreshold = _.get(d, 'component.backPressureDataSizeThreshold');
var predictions = _.get(d, 'status.aggregateSnapshot.predictions');
var tooltipLines = ['Queue: ' + _.clamp(percentUseBytes, 0, 100) + '% full (based on ' + dataSizeThreshold + ' data size threshold)'];
if (!_.isNil(predictions)) {
var predictedPercentBytes = _.get(predictions, 'predictedPercentBytes', -1);
var timeToBackPressure = _.get(predictions, 'predictedMillisUntilBytesBackpressure', -1);
// only show predicted percent if it is non-negative
var predictionIntervalSeconds = _.get(predictions, 'predictionIntervalSeconds', 60 * 5);
if (_.isNumber(predictedPercentBytes) && predictedPercentBytes > -1) {
tooltipLines.push('Predicted queue (next ' + (predictionIntervalSeconds / 60) + ' mins): ' + _.clamp(predictedPercentBytes, 0, 100) + '%')
} else {
tooltipLines.push('Predicted queue (next ' + (predictionIntervalSeconds / 60 ) + ' mins): NA' )
}
// only show an estimate if it is valid (non-negative but less than the max number supported)
if (_.isNumber(timeToBackPressure) && _.inRange(timeToBackPressure, 0, Number.MAX_SAFE_INTEGER) && !isAtBackPressure(d)) {
var duration = nfCommon.formatPredictedDuration(timeToBackPressure);
tooltipLines.push('Estimated time to back pressure: ' + duration);
} else {
tooltipLines.push('Estimated time to back pressure: ' + (isAtBackPressure(d) ? 'now' : 'NA'));
}
} else {
tooltipLines.push('Queue Prediction is not configured')
}
if (_.isEmpty(tooltipLines)) {
return '';
} else if (_.size(tooltipLines) === 1) {
return tooltipLines[0];
} else {
tooltipContent = nfCommon.formatUnorderedList(tooltipLines)
}
} else {
tooltipContent = 'Back Pressure Data Size Threshold is not configured';
}
return tooltipContent;
};
/**
* Updates the stats of the connections in the specified selection.
*
@ -1517,12 +1668,52 @@
deferred.resolve();
});
updated.select('rect.backpressure-data-size').select('title').text(function (d) {
if (nfCommon.isDefinedAndNotNull(d.status.aggregateSnapshot.percentUseBytes)) {
return 'Queue is ' + d.status.aggregateSnapshot.percentUseBytes + '% full based on Back Pressure Data Size Threshold';
var backpressurePercentDataSizePrediction = updated.select('rect.backpressure-tick.data-size-prediction');
backpressurePercentDataSizePrediction.transition()
.duration(400)
.attrs({
'x': function (d) {
// clamp the prediction between 0 and 100 percent
var predicted = _.get(d, 'status.aggregateSnapshot.predictions.predictedPercentBytes', 0);
return (backpressureBarWidth * _.clamp(predicted, 0, 100)) / 100;
},
'display': function (d) {
var predicted = _.get(d, 'status.aggregateSnapshot.predictions.predictedPercentBytes', -1);
if (predicted >= 0) {
return 'unset';
} else {
return 'Back Pressure Data Size Threshold is not configured';
// don't show it if there not a valid prediction
return 'none';
}
}
}).on('end', function () {
backpressurePercentDataSizePrediction.classed('prediction-down', function (d) {
var actual = _.get(d, 'status.aggregateSnapshot.predictions.percentUseBytes', 0);
var predicted = _.get(d, 'status.aggregateSnapshot.predictions.predictedPercentBytes', 0);
return predicted < actual;
})
});
updated.select('g.backpressure-data-size-container')
.each(function(d) {
var tip = d3.select('#back-pressure-size-tip-' + d.id);
// create a DOM element for the tooltip if ones does not already exist
if (tip.empty()) {
tip = d3.select('#connection-tooltips')
.append('div')
.attr('id', function() {
return 'back-pressure-size-tip-' + d.id
})
.attr('class', 'tooltip nifi-tooltip');
}
// update the tooltip
tip.html(function() {
return $('<div></div>').append(getBackPressureSizeTip(d)).html();
});
nfCanvasUtils.canvasTooltip(tip, d3.select(this));
});
}).promise();
@ -1557,12 +1748,53 @@
deferred.resolve();
});
updated.select('rect.backpressure-object').select('title').text(function (d) {
if (nfCommon.isDefinedAndNotNull(d.status.aggregateSnapshot.percentUseCount)) {
return 'Queue is ' + d.status.aggregateSnapshot.percentUseCount + '% full based on Back Pressure Object Threshold';
var backpressurePercentObjectPrediction = updated.select('rect.backpressure-tick.object-prediction');
backpressurePercentObjectPrediction.transition()
.duration(400)
.attrs({
'x': function (d) {
// clamp the prediction between 0 and 100 percent
var predicted = _.get(d, 'status.aggregateSnapshot.predictions.predictedPercentCount', 0);
return (backpressureBarWidth * _.clamp(predicted, 0, 100)) / 100;
},
'display': function (d) {
var predicted = _.get(d, 'status.aggregateSnapshot.predictions.predictedPercentCount', -1);
if (predicted >= 0) {
return 'unset';
} else {
return 'Back Pressure Object Threshold is not configured';
// don't show it if there not a valid prediction
return 'none';
}
}
}).on('end', function () {
backpressurePercentObjectPrediction.classed('prediction-down', function (d) {
var actual = _.get(d, 'status.aggregateSnapshot.percentUseCount', 0);
var predicted = _.get(d, 'status.aggregateSnapshot.predictions.predictedPercentCount', 0);
return predicted < actual;
})
});
updated.select('g.backpressure-object-container')
.each(function(d) {
var tip = d3.select('#back-pressure-count-tip-' + d.id);
// create a DOM element for the tooltip if ones does not already exist
if (tip.empty()) {
tip = d3.select('#connection-tooltips')
.append('div')
.attr('id', function() {
return 'back-pressure-count-tip-' + d.id
})
.attr('class', 'tooltip nifi-tooltip');
}
// update the tooltip
tip.html(function() {
return $('<div></div>').append(getBackPressureCountTip(d)).html();
});
nfCanvasUtils.canvasTooltip(tip, d3.select(this));
});
}).promise();
@ -1627,7 +1859,15 @@
});
// remove the connection
removed.remove();
removed.call(removeTooltips).remove();
};
var removeTooltips = function (removed) {
removed.each(function (d) {
// remove any associated tooltips
$('#back-pressure-size-tip-' + d.id).remove();
$('#back-pressure-count-tip-' + d.id).remove();
});
};
var nfConnection = {

View File

@ -23,22 +23,25 @@
define(['jquery',
'd3',
'nf.Storage',
'lodash-core'],
function ($, d3, nfStorage, _) {
return (nf.Common = factory($, d3, nfStorage, _));
'lodash-core',
'moment'],
function ($, d3, nfStorage, _, moment) {
return (nf.Common = factory($, d3, nfStorage, _, moment));
});
} else if (typeof exports === 'object' && typeof module === 'object') {
module.exports = (nf.Common = factory(require('jquery'),
require('d3'),
require('nf.Storage'),
require('lodash-core')));
require('lodash-core'),
require('moment')));
} else {
nf.Common = factory(root.$,
root.d3,
root.nf.Storage,
root._);
root._,
root.moment);
}
}(this, function ($, d3, nfStorage, _) {
}(this, function ($, d3, nfStorage, _, moment) {
'use strict';
$(document).ready(function () {
@ -1299,6 +1302,19 @@
}
},
/**
* Formats a number (in milliseconds) to a human-readable textual description.
*
* @param duration number of milliseconds representing the duration
* @return {string|*} a human-readable string
*/
formatPredictedDuration: function (duration) {
if (duration === 0) {
return 'now';
}
return moment.duration(duration, 'ms').humanize();
},
/**
* Constants for formatting data size.
*/

View File

@ -25,9 +25,10 @@
'nf.StatusHistory',
'nf.ProcessorDetails',
'nf.ConnectionDetails',
'nf.ng.Bridge'],
function ($, Slick, nfCommon, nfErrorHandler, nfStatusHistory, nfProcessorDetails, nfConnectionDetails, nfNgBridge) {
return (nf.SummaryTable = factory($, Slick, nfCommon, nfErrorHandler, nfStatusHistory, nfProcessorDetails, nfConnectionDetails, nfNgBridge));
'nf.ng.Bridge',
'lodash-core'],
function ($, Slick, nfCommon, nfErrorHandler, nfStatusHistory, nfProcessorDetails, nfConnectionDetails, nfNgBridge, _) {
return (nf.SummaryTable = factory($, Slick, nfCommon, nfErrorHandler, nfStatusHistory, nfProcessorDetails, nfConnectionDetails, nfNgBridge, _));
});
} else if (typeof exports === 'object' && typeof module === 'object') {
module.exports = (nf.SummaryTable =
@ -38,7 +39,8 @@
require('nf.StatusHistory'),
require('nf.ProcessorDetails'),
require('nf.ConnectionDetails'),
require('nf.ng.Bridge')));
require('nf.ng.Bridge'),
require('lodash-core')));
} else {
nf.SummaryTable = factory(root.$,
root.Slick,
@ -47,9 +49,10 @@
root.nf.StatusHistory,
root.nf.ProcessorDetails,
root.nf.ConnectionDetails,
root.nf.ng.Bridge);
root.nf.ng.Bridge,
root._);
}
}(this, function ($, Slick, nfCommon, nfErrorHandler, nfStatusHistory, nfProcessorDetails, nfConnectionDetails, nfNgBridge) {
}(this, function ($, Slick, nfCommon, nfErrorHandler, nfStatusHistory, nfProcessorDetails, nfConnectionDetails, nfNgBridge, _) {
'use strict';
/**
@ -64,6 +67,8 @@
}
};
var DATA_SEPARATOR = '&nbsp;&nbsp;|&nbsp;&nbsp;';
/**
* Goes to the specified component if possible.
*
@ -81,6 +86,59 @@
}
};
var backpressurePredictionFormatter = function (row, cell, value, columnDef, dataContext) {
var predictedMillisUntilBytesBackpressure = _.get(dataContext, 'predictions.predictedMillisUntilBytesBackpressure', -1);
var predictedMillisUntilCountBackpressure = _.get(dataContext, 'predictions.predictedMillisUntilCountBackpressure', -1);
var percentUseCount = _.get(dataContext, 'percentUseCount', 0);
var percentUseBytes = _.get(dataContext, 'percentUseBytes', 0);
var predictions = [
{ label: 'object', timeToBackPressure: predictedMillisUntilCountBackpressure },
{ label: 'size', timeToBackPressure: predictedMillisUntilBytesBackpressure },
];
var actualQueuePercents = [
{ label: 'object', percent: percentUseCount },
{ label: 'size', percent: percentUseBytes }
];
// if one of the queues is already at backpressure, return now as the prediction
var maxActual = _.maxBy(actualQueuePercents, 'percent');
if (maxActual.percent >= 100) {
// currently experiencing back pressure
return 'now (' + maxActual.label + ')';
}
// filter out the predictions that are unknown
var knownPredictions = predictions.filter(function(p) {
return p.timeToBackPressure >= 0;
});
if (_.isEmpty(knownPredictions)) {
// there is not a valid time-to-back-pressure prediction available
return 'NA';
}
// there is at least one valid prediction, return the minimum time to back pressure
var minPrediction = _.minBy(knownPredictions, 'timeToBackPressure');
var formatted = nfCommon.formatPredictedDuration(minPrediction.timeToBackPressure);
return nfCommon.escapeHtml(formatted) + ' (' + minPrediction.label + ')';
};
// define the column used to display backpressure predicted values (reused in both tables)
var backpressurePredictionColumn = {
id: 'backpressurePrediction',
field: 'backpressurePrediction',
name: 'Estimated Time to Back Pressure',
sortable: true,
defaultSortAsc: false,
formatter: backpressurePredictionFormatter,
resize: true,
toolTip: 'Estimated Time to Back Pressure'
};
/**
* Initializes the summary table.
*
@ -291,12 +349,12 @@
// formatter for io
var ioFormatter = function (row, cell, value, columnDef, dataContext) {
return nfCommon.escapeHtml(dataContext.read) + ' / ' + nfCommon.escapeHtml(dataContext.written);
return nfCommon.escapeHtml(dataContext.read) + DATA_SEPARATOR + nfCommon.escapeHtml(dataContext.written);
};
// formatter for tasks
var taskTimeFormatter = function (row, cell, value, columnDef, dataContext) {
return nfCommon.formatInteger(dataContext.tasks) + ' / ' + nfCommon.escapeHtml(dataContext.tasksDuration);
return nfCommon.formatInteger(dataContext.tasks) + DATA_SEPARATOR + nfCommon.escapeHtml(dataContext.tasksDuration);
};
// function for formatting the last accessed time
@ -309,7 +367,7 @@
var threadCounts = '';
var threadTip = '';
if (dataContext.terminatedThreadCount > 0) {
threadCounts = '(' + dataContext.activeThreadCount + ' / ' + dataContext.terminatedThreadCount + ')';
threadCounts = '(' + dataContext.activeThreadCount + DATA_SEPARATOR + dataContext.terminatedThreadCount + ')';
threadTip = 'Threads: (Active / Terminated)';
} else if (dataContext.activeThreadCount > 0) {
threadCounts = '(' + dataContext.activeThreadCount + ')';
@ -372,7 +430,7 @@
var inputColumn = {
id: 'input',
field: 'input',
name: '<span class="input-title">In</span>&nbsp;/&nbsp;<span class="input-size-title">Size</span>&nbsp;<span style="font-weight: normal; overflow: hidden;">5 min</span>',
name: '<span class="input-title">In</span>&nbsp;(<span class="input-size-title">Size</span>)&nbsp;<span style="font-weight: normal; overflow: hidden;">5 min</span>',
toolTip: 'Count / data size in the last 5 min',
sortable: true,
defaultSortAsc: false,
@ -382,7 +440,7 @@
var ioColumn = {
id: 'io',
field: 'io',
name: '<span class="read-title">Read</span>&nbsp;/&nbsp;<span class="written-title">Write</span>&nbsp;<span style="font-weight: normal; overflow: hidden;">5 min</span>',
name: '<span class="read-title">Read</span>' + DATA_SEPARATOR + '<span class="written-title">Write</span>&nbsp;<span style="font-weight: normal; overflow: hidden;">5 min</span>',
toolTip: 'Data size in the last 5 min',
formatter: ioFormatter,
sortable: true,
@ -392,7 +450,7 @@
var outputColumn = {
id: 'output',
field: 'output',
name: '<span class="output-title">Out</span>&nbsp;/&nbsp;<span class="output-size-title">Size</span>&nbsp;<span style="font-weight: normal; overflow: hidden;">5 min</span>',
name: '<span class="output-title">Out</span>&nbsp;(<span class="output-size-title">Size</span>)&nbsp;<span style="font-weight: normal; overflow: hidden;">5 min</span>',
toolTip: 'Count / data size in the last 5 min',
sortable: true,
defaultSortAsc: false,
@ -402,7 +460,7 @@
var tasksTimeColumn = {
id: 'tasks',
field: 'tasks',
name: '<span class="tasks-title">Tasks</span>&nbsp;/&nbsp;<span class="time-title">Time</span>&nbsp;<span style="font-weight: normal; overflow: hidden;">5 min</span>',
name: '<span class="tasks-title">Tasks</span>' + DATA_SEPARATOR + '<span class="time-title">Time</span>&nbsp;<span style="font-weight: normal; overflow: hidden;">5 min</span>',
toolTip: 'Count / duration in the last 5 min',
formatter: taskTimeFormatter,
sortable: true,
@ -708,23 +766,27 @@
return '<div class="pointer show-connection-details fa fa-info-circle" title="View Connection Details"></div>';
};
var formatPercent = function (value) {
return _.isNumber(value) && value >= 0 ? _.clamp(value, 0, 100) + '%' : 'NA';
};
var backpressureFormatter = function (row, cell, value, columnDef, dataContext) {
var percentUseCount = 'NA';
if (nfCommon.isDefinedAndNotNull(dataContext.percentUseCount)) {
percentUseCount = dataContext.percentUseCount + '%';
percentUseCount = formatPercent(dataContext.percentUseCount);
}
var percentUseBytes = 'NA';
if (nfCommon.isDefinedAndNotNull(dataContext.percentUseBytes)) {
percentUseBytes = dataContext.percentUseBytes + '%';
percentUseBytes = formatPercent(dataContext.percentUseBytes);
}
return nfCommon.escapeHtml(percentUseCount) + ' / ' + nfCommon.escapeHtml(percentUseBytes);
return nfCommon.escapeHtml(percentUseCount) + DATA_SEPARATOR + nfCommon.escapeHtml(percentUseBytes);
};
// define the input, read, written, and output columns (reused between both tables)
var queueColumn = {
id: 'queued',
field: 'queued',
name: '<span class="queued-title">Queue</span>&nbsp;/&nbsp;<span class="queued-size-title">Size</span>',
name: '<span class="queued-title">Queue</span>&nbsp;(<span class="queued-size-title">Size</span>)',
sortable: true,
defaultSortAsc: false,
resize: true,
@ -735,7 +797,7 @@
var backpressureColumn = {
id: 'backpressure',
field: 'backpressure',
name: '<span class="backpressure-object-title">Queue</span>&nbsp;/&nbsp;<span class="backpressure-data-size-title">Size</span> Threshold',
name: 'Threshold %: <span class="backpressure-object-title">Queue</span>&nbsp;&nbsp;|&nbsp;&nbsp;<span class="backpressure-data-size-title">Size</span>',
sortable: true,
defaultSortAsc: false,
formatter: backpressureFormatter,
@ -753,14 +815,6 @@
width: 50,
maxWidth: 50
},
{
id: 'sourceName',
field: 'sourceName',
name: 'Source Name',
sortable: true,
resizable: true,
formatter: nfCommon.genericValueFormatter
},
{
id: 'name',
field: 'name',
@ -769,18 +823,27 @@
resizable: true,
formatter: valueFormatter
},
queueColumn,
backpressureColumn,
backpressurePredictionColumn,
inputColumn,
{
id: 'destinationName',
field: 'destinationName',
name: 'Destination Name',
id: 'sourceName',
field: 'sourceName',
name: 'From Source',
sortable: true,
resizable: true,
formatter: nfCommon.genericValueFormatter
},
inputColumn,
queueColumn,
backpressureColumn,
outputColumn
outputColumn,
{
id: 'destinationName',
field: 'destinationName',
name: 'To Destination',
sortable: true,
resizable: true,
formatter: nfCommon.genericValueFormatter
}
];
// add an action column if appropriate
@ -952,9 +1015,10 @@
resizable: true,
formatter: nfCommon.genericValueFormatter
},
inputColumn,
queueColumn,
backpressureColumn,
backpressurePredictionColumn,
inputColumn,
outputColumn
];
@ -1032,7 +1096,7 @@
var transferredColumn = {
id: 'transferred',
field: 'transferred',
name: '<span class="transferred-title">Transferred</span>&nbsp;/&nbsp;<span class="transferred-size-title">Size</span>&nbsp;<span style="font-weight: normal; overflow: hidden;">5 min</span>',
name: '<span class="transferred-title">Transferred</span>&nbsp;(<span class="transferred-size-title">Size</span>)&nbsp;<span style="font-weight: normal; overflow: hidden;">5 min</span>',
toolTip: 'Count / data size transferred to and from connections in the last 5 min',
resizable: true,
defaultSortAsc: false,
@ -1042,7 +1106,7 @@
var sentColumn = {
id: 'sent',
field: 'sent',
name: '<span class="sent-title">Sent</span>&nbsp;/&nbsp;<span class="sent-size-title">Size</span>&nbsp;<span style="font-weight: normal; overflow: hidden;">5 min</span>',
name: '<span class="sent-title">Sent</span>&nbsp;(<span class="sent-size-title">Size</span>)&nbsp;<span style="font-weight: normal; overflow: hidden;">5 min</span>',
toolTip: 'Count / data size in the last 5 min',
sortable: true,
defaultSortAsc: false,
@ -1052,7 +1116,7 @@
var receivedColumn = {
id: 'received',
field: 'received',
name: '<span class="received-title">Received</span>&nbsp;/&nbsp;<span class="received-size-title">Size</span>&nbsp;<span style="font-weight: normal; overflow: hidden;">5 min</span>',
name: '<span class="received-title">Received</span>&nbsp;(<span class="received-size-title">Size</span>)&nbsp;<span style="font-weight: normal; overflow: hidden;">5 min</span>',
toolTip: 'Count / data size in the last 5 min',
sortable: true,
defaultSortAsc: false,
@ -2296,6 +2360,22 @@
var bPercentUseDataSize = nfCommon.isDefinedAndNotNull(b['percentUseBytes']) ? b['percentUseBytes'] : -1;
return aPercentUseDataSize - bPercentUseDataSize;
}
} else if (sortDetails.columnId === 'backpressurePrediction') {
// if the connection is at backpressure currently, "now" displays and not the estimate. Should account for that when sorting.
var aMaxCurrentUsage = Math.max(_.get(a, 'percentUseBytes', 0), _.get(a, 'percentUseCount', 0));
var bMaxCurrentUsage = Math.max(_.get(b, 'percentUseBytes', 0), _.get(b, 'percentUseCount', 0));
var aMinTime = Math.min(_.get(a, 'predictions.predictedMillisUntilBytesBackpressure', Number.MAX_VALUE), _.get(a, 'predictions.predictedMillisUntilCountBackpressure', Number.MAX_VALUE));
var bMinTime = Math.min(_.get(b, 'predictions.predictedMillisUntilBytesBackpressure', Number.MAX_VALUE), _.get(b, 'predictions.predictedMillisUntilCountBackpressure', Number.MAX_VALUE));
if (aMaxCurrentUsage >= 100) {
aMinTime = 0;
}
if (bMaxCurrentUsage >= 100) {
bMinTime = 0;
}
return aMinTime - bMinTime;
} else if (sortDetails.columnId === 'sent' || sortDetails.columnId === 'received' || sortDetails.columnId === 'input' || sortDetails.columnId === 'output' || sortDetails.columnId === 'transferred') {
var aSplit = a[sortDetails.columnId].split(/\(([^)]+)\)/);
var bSplit = b[sortDetails.columnId].split(/\(([^)]+)\)/);
@ -2349,6 +2429,9 @@
$('#' + tableId + ' span.queued-size-title').removeClass('sorted');
$('#' + tableId + ' span.backpressure-object-title').removeClass('sorted');
$('#' + tableId + ' span.backpressure-data-size-title').removeClass('sorted');
$('#' + tableId + ' span.backpressure-prediction-object-title').removeClass('sorted');
$('#' + tableId + ' span.backpressure-prediction-data-size-title').removeClass('sorted');
$('#' + tableId + ' span.backpressure-prediction-time-title').removeClass('sorted');
$('#' + tableId + ' span.input-title').removeClass('sorted');
$('#' + tableId + ' span.input-size-title').removeClass('sorted');
$('#' + tableId + ' span.output-title').removeClass('sorted');
@ -2776,10 +2859,40 @@
queuedSize: snapshot.queuedSize,
percentUseCount: snapshot.percentUseCount,
percentUseBytes: snapshot.percentUseBytes,
predictions: snapshot.predictions,
output: snapshot.output
});
});
// determine if the backpressure prediction column should be displayed or not
var anyPredictionsDisabled = _.some(clusterConnections, function (connectionItem) {
return _.isNil(connectionItem.predictions);
});
var currentConnectionColumns = clusterConnectionsGrid.getColumns();
if (anyPredictionsDisabled) {
var connectionColumnsNoPredictedBackPressure = currentConnectionColumns.filter(function (column) {
return column.id !== backpressurePredictionColumn.id;
});
clusterConnectionsGrid.setColumns(connectionColumnsNoPredictedBackPressure);
} else {
// make sure the prediction column is there
var backPressurePredictionColumnIndex = currentConnectionColumns.findIndex(function (column) {
return column.id === backpressurePredictionColumn.id;
});
// if it is not there, add it immediately after the backpressure column
if (backPressurePredictionColumnIndex < 0) {
var backpressureColumnIndex = currentConnectionColumns.findIndex(function (column) {
return column.id === 'backpressure';
});
if (backpressureColumnIndex < 0) {
currentConnectionColumns.push(backpressurePredictionColumn);
} else {
currentConnectionColumns.splice(backpressureColumnIndex + 1, 0, backpressurePredictionColumn);
}
clusterConnectionsGrid.setColumns(currentConnectionColumns);
}
}
// update the processors
clusterConnectionsData.setItems(clusterConnections);
clusterConnectionsData.reSort();
@ -3200,6 +3313,34 @@
// populate the tables
populateProcessGroupStatus(processorItems, connectionItems, processGroupItems, inputPortItems, outputPortItems, remoteProcessGroupItems, aggregateSnapshot, [aggregateSnapshot]);
var anyPredictionsDisabled = _.some(connectionItems, function (connectionItem) {
return _.isNil(connectionItem.predictions);
});
var currentConnectionColumns = connectionsGrid.getColumns();
if (anyPredictionsDisabled) {
var connectionColumnsNoPredictedBackPressure = currentConnectionColumns.filter(function (column) {
return column.id !== backpressurePredictionColumn.id;
});
connectionsGrid.setColumns(connectionColumnsNoPredictedBackPressure);
} else {
// make sure the prediction column is there
var backPressurePredictionColumnIndex = currentConnectionColumns.findIndex(function (column) {
return column.id === backpressurePredictionColumn.id;
});
// if it is not there, add it immediately after the backpressure column
if (backPressurePredictionColumnIndex < 0) {
var backpressureColumnIndex = currentConnectionColumns.findIndex(function (column) {
return column.id === 'backpressure';
});
if (backpressureColumnIndex < 0) {
currentConnectionColumns.push(backpressurePredictionColumn);
} else {
currentConnectionColumns.splice(backpressureColumnIndex + 1, 0, backpressurePredictionColumn);
}
connectionsGrid.setColumns(currentConnectionColumns);
}
}
// update the processors
processorsData.setItems(processorItems);
processorsData.reSort();