mirror of https://github.com/apache/nifi.git
NIFI-7408 - added percent used metrics for connections
NIFI-7408 - return double value for utilization Signed-off-by: Matthew Burgess <mattyb149@apache.org> This closes #4240
This commit is contained in:
parent
659a383723
commit
7784178abd
|
@ -171,6 +171,20 @@ public class NiFiMetricsRegistry extends AbstractMetricsRegistry {
|
|||
"source_id", "source_name", "destination_id", "destination_name")
|
||||
.register(registry));
|
||||
|
||||
nameToGaugeMap.put("PERCENT_USED_BYTES", Gauge.build()
|
||||
.name("nifi_percent_used_bytes")
|
||||
.help("The percentage of connection that is filled based on content bytes")
|
||||
.labelNames("instance", "component_type", "component_name", "component_id", "parent_id",
|
||||
"source_id", "source_name", "destination_id", "destination_name")
|
||||
.register(registry));
|
||||
|
||||
nameToGaugeMap.put("PERCENT_USED_COUNT", Gauge.build()
|
||||
.name("nifi_percent_used_count")
|
||||
.help("The percentage of connection that is filled based on object count")
|
||||
.labelNames("instance", "component_type", "component_name", "component_id", "parent_id",
|
||||
"source_id", "source_name", "destination_id", "destination_name")
|
||||
.register(registry));
|
||||
|
||||
// Port metrics
|
||||
nameToGaugeMap.put("IS_TRANSMITTING", Gauge.build()
|
||||
.name("nifi_transmitting")
|
||||
|
|
|
@ -223,6 +223,12 @@ public class PrometheusMetricsUtil {
|
|||
instanceId, connComponentType, connComponentName, connComponentId, parentId, sourceId, sourceName, destinationId, destinationName);
|
||||
nifiMetricsRegistry.setDataPoint(connectionStatus.getBackPressureObjectThreshold(), "BACKPRESSURE_OBJECT_THRESHOLD",
|
||||
instanceId, connComponentType, connComponentName, connComponentId, parentId, sourceId, sourceName, destinationId, destinationName);
|
||||
|
||||
nifiMetricsRegistry.setDataPoint(getUtilization(connectionStatus.getQueuedBytes(), connectionStatus.getBackPressureBytesThreshold()),
|
||||
"PERCENT_USED_BYTES", instanceId, connComponentType, connComponentName, connComponentId, parentId, sourceId, sourceName, destinationId, destinationName);
|
||||
nifiMetricsRegistry.setDataPoint(getUtilization(connectionStatus.getQueuedCount(), connectionStatus.getBackPressureObjectThreshold()),
|
||||
"PERCENT_USED_COUNT", instanceId, connComponentType, connComponentName, connComponentId, parentId, sourceId, sourceName, destinationId, destinationName);
|
||||
|
||||
boolean isBackpressureEnabled = (connectionStatus.getBackPressureObjectThreshold() > 0 && connectionStatus.getBackPressureObjectThreshold() <= connectionStatus.getQueuedCount())
|
||||
|| (connectionStatus.getBackPressureBytesThreshold() > 0 && connectionStatus.getBackPressureBytesThreshold() <= connectionStatus.getMaxQueuedBytes());
|
||||
nifiMetricsRegistry.setDataPoint(isBackpressureEnabled ? 1 : 0, "IS_BACKPRESSURE_ENABLED",
|
||||
|
@ -364,6 +370,10 @@ public class PrometheusMetricsUtil {
|
|||
return connectionAnalyticsMetricsRegistry.getRegistry();
|
||||
}
|
||||
|
||||
private static double getUtilization(final double used, final double total) {
|
||||
return (used / total) * 100;
|
||||
}
|
||||
|
||||
public static CollectorRegistry createBulletinMetrics(BulletinMetricsRegistry bulletinMetricsRegistry, String instId, String compType, String compId, String pgId, String nodeAddr,
|
||||
String cat, String srcName, String srcId, String lvl) {
|
||||
final String instanceId = StringUtils.isEmpty(instId) ? DEFAULT_LABEL_STRING : instId;
|
||||
|
|
Loading…
Reference in New Issue